diff --git a/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml b/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml index 30c88e990067..b054121ec441 100644 --- a/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml +++ b/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml @@ -9,7 +9,9 @@ body: description: What Apache Iceberg version are you using? multiple: false options: - - "1.2.1 (latest release)" + - "1.3.1 (latest release)" + - "1.3.0" + - "1.2.1" - "1.2.0" - "1.1.0" - "1.0.0" diff --git a/.github/ISSUE_TEMPLATE/iceberg_question.yml b/.github/ISSUE_TEMPLATE/iceberg_question.yml index a11d467d0b35..98d857b0cdfb 100644 --- a/.github/ISSUE_TEMPLATE/iceberg_question.yml +++ b/.github/ISSUE_TEMPLATE/iceberg_question.yml @@ -5,7 +5,7 @@ labels: ["kind:question"] body: - type: markdown attributes: - value: "Feel free to ask your question on [Slack](https://join.slack.com/t/apache-iceberg/shared_invite/zt-1uva9gyp1-TrLQl7o~nZ5PsTVgl6uoEQ) as well." + value: "Feel free to ask your question on [Slack](https://join.slack.com/t/apache-iceberg/shared_invite/zt-1znkcg5zm-7_FE~pcox347XwZE3GNfPg) as well." - type: textarea attributes: label: Query engine diff --git a/.github/workflows/api-binary-compatibility.yml b/.github/workflows/api-binary-compatibility.yml index 7f44e5b6ae29..d14d38d37d90 100644 --- a/.github/workflows/api-binary-compatibility.yml +++ b/.github/workflows/api-binary-compatibility.yml @@ -52,7 +52,7 @@ jobs: java-version: 11 - run: | echo "Using the old version tag, as per git describe, of $(git describe)"; - - run: ./gradlew :iceberg-api:revapi --rerun-tasks + - run: ./gradlew revapi --rerun-tasks - uses: actions/upload-artifact@v3 if: failure() with: diff --git a/.github/workflows/recurring-jmh-benchmarks.yml b/.github/workflows/recurring-jmh-benchmarks.yml index c6a1e80fe7ea..b864dc11202d 100644 --- a/.github/workflows/recurring-jmh-benchmarks.yml +++ b/.github/workflows/recurring-jmh-benchmarks.yml @@ -27,6 +27,7 @@ on: jobs: run-benchmark: + if: github.repository_owner == 'apache' runs-on: ubuntu-22.04 strategy: fail-fast: false diff --git a/.github/workflows/stale.yml b/.github/workflows/stale.yml index cb95148ff82a..566ae2441ea0 100644 --- a/.github/workflows/stale.yml +++ b/.github/workflows/stale.yml @@ -28,6 +28,7 @@ permissions: jobs: stale: + if: github.repository_owner == 'apache' runs-on: ubuntu-22.04 steps: - uses: actions/stale@v8.0.0 diff --git a/.palantir/revapi.yml b/.palantir/revapi.yml index ef039675ac03..1ae35cf64d0a 100644 --- a/.palantir/revapi.yml +++ b/.palantir/revapi.yml @@ -737,6 +737,67 @@ acceptedBreaks: old: "method void org.apache.iceberg.rest.auth.OAuth2Util.AuthSession::(java.util.Map, java.lang.String, java.lang.String)" justification: "Removing deprecations for 1.3.0" + - code: "java.method.returnTypeTypeParametersChanged" + old: "method java.util.List\ + \ org.apache.iceberg.rest.requests.UpdateTableRequest::requirements()" + new: "method java.util.List org.apache.iceberg.rest.requests.UpdateTableRequest::requirements()" + justification: "Signature changed to an interface, but this is safe because\ + \ of type erasure and the original type is always returned" + "1.3.0": + org.apache.iceberg:iceberg-api: + - code: "java.class.removed" + old: "class org.apache.iceberg.actions.ImmutableDeleteOrphanFiles" + justification: "Moving from iceberg-api to iceberg-core" + - code: "java.class.removed" + old: "class org.apache.iceberg.actions.ImmutableDeleteReachableFiles" + justification: "Moving from iceberg-api to iceberg-core" + - code: "java.class.removed" + old: "class org.apache.iceberg.actions.ImmutableExpireSnapshots" + justification: "Moving from iceberg-api to iceberg-core" + - code: "java.class.removed" + old: "class org.apache.iceberg.actions.ImmutableMigrateTable" + justification: "Moving from iceberg-api to iceberg-core" + - code: "java.class.removed" + old: "class org.apache.iceberg.actions.ImmutableRewriteDataFiles" + justification: "Moving from iceberg-api to iceberg-core" + - code: "java.class.removed" + old: "class org.apache.iceberg.actions.ImmutableRewriteManifests" + justification: "Moving from iceberg-api to iceberg-core" + - code: "java.class.removed" + old: "class org.apache.iceberg.actions.ImmutableRewritePositionDeleteFiles" + justification: "Moving from iceberg-api to iceberg-core" + - code: "java.class.removed" + old: "class org.apache.iceberg.actions.ImmutableSnapshotTable" + justification: "Moving from iceberg-api to iceberg-core" + - code: "java.class.removed" + old: "class org.apache.iceberg.view.ImmutableSQLViewRepresentation" + justification: "Moving from iceberg-api to iceberg-core" + - code: "java.class.removed" + old: "class org.apache.iceberg.view.ImmutableViewHistoryEntry" + justification: "Moving from iceberg-api to iceberg-core" + - code: "java.class.removed" + old: "class org.apache.iceberg.view.ImmutableViewVersion" + justification: "Moving from iceberg-api to iceberg-core" + - code: "java.class.removed" + old: "interface org.apache.iceberg.view.SQLViewRepresentation" + justification: "Moving from iceberg-api to iceberg-core" + - code: "java.method.numberOfParametersChanged" + old: "method org.apache.iceberg.view.ViewBuilder org.apache.iceberg.view.ViewBuilder::withQuery(java.lang.String)" + new: "method T org.apache.iceberg.view.VersionBuilder::withQuery(java.lang.String,\ + \ java.lang.String) @ org.apache.iceberg.view.ViewBuilder" + justification: "Acceptable break due to updating View APIs and the View Spec" + - code: "java.method.removed" + old: "method org.apache.iceberg.view.ViewBuilder org.apache.iceberg.view.ViewBuilder::withDialect(java.lang.String)" + justification: "Acceptable break due to updating View APIs and the View Spec" + - code: "java.method.removed" + old: "method org.apache.iceberg.view.ViewBuilder org.apache.iceberg.view.ViewBuilder::withFieldAliases(java.util.List)" + justification: "Acceptable break due to updating View APIs and the View Spec" + - code: "java.method.removed" + old: "method org.apache.iceberg.view.ViewBuilder org.apache.iceberg.view.ViewBuilder::withFieldComments(java.util.List)" + justification: "Acceptable break due to updating View APIs and the View Spec" + - code: "java.method.removed" + old: "method org.apache.iceberg.view.ViewBuilder org.apache.iceberg.view.ViewBuilder::withQueryColumnNames(java.util.List)" + justification: "Acceptable break due to updating View APIs and the View Spec" apache-iceberg-0.14.0: org.apache.iceberg:iceberg-api: - code: "java.class.defaultSerializationChanged" 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 index 0cc76825c241..e98f228b6f37 100644 --- 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 @@ -28,12 +28,13 @@ 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.commons.io.input.BoundedInputStream; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.io.ByteStreams; import org.slf4j.Logger; @@ -276,4 +277,246 @@ 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 count The count of bytes read. + * @throws IOException Subclasses may throw. + * @since 2.12.0 + */ + protected void onMaxLength(final long maxLength, final long pCount) 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 75766a671490..cce4f32dd02b 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 @@ -33,10 +33,11 @@ import java.nio.file.attribute.BasicFileAttributes; import java.security.MessageDigest; import java.security.NoSuchAlgorithmException; +import java.util.Comparator; import java.util.List; import java.util.Locale; import java.util.Map; -import org.apache.commons.io.FileUtils; +import java.util.stream.Stream; import org.apache.directory.api.util.Hex; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -101,7 +102,7 @@ private static void inputStreamToFile(InputStream inputStream, File targetFile) void createBucket(String bucketName) throws IOException { File newBucket = new File(root, bucketName); - FileUtils.forceMkdir(newBucket); + Files.createDirectory(newBucket.toPath()); } Bucket getBucket(String bucketName) { @@ -122,7 +123,9 @@ void deleteBucket(String bucketName) throws IOException { 409, OSSErrorCode.BUCKET_NOT_EMPTY, "The bucket you tried to delete is not empty. "); } - FileUtils.deleteDirectory(dir); + try (Stream walk = Files.walk(dir.toPath())) { + walk.sorted(Comparator.reverseOrder()).map(Path::toFile).forEach(File::delete); + } } ObjectMetadata putObject( diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockRule.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockRule.java index 12c1e0128de5..d23d4f1d83f3 100644 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockRule.java +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockRule.java @@ -25,7 +25,6 @@ import java.io.UncheckedIOException; import java.nio.file.Files; import java.util.Map; -import org.apache.commons.io.FileUtils; import org.apache.iceberg.aliyun.oss.AliyunOSSTestRule; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.base.Strings; @@ -114,7 +113,8 @@ public AliyunOSSTestRule build() { if (Strings.isNullOrEmpty(rootDir)) { File dir = new File( - FileUtils.getTempDirectory(), "oss-mock-file-store-" + System.currentTimeMillis()); + System.getProperty("java.io.tmpdir"), + "oss-mock-file-store-" + System.currentTimeMillis()); rootDir = dir.getAbsolutePath(); props.put(AliyunOSSMockApp.PROP_ROOT_DIR, rootDir); } diff --git a/api/src/main/java/org/apache/iceberg/ContentFile.java b/api/src/main/java/org/apache/iceberg/ContentFile.java index 7971ad0ccb10..fd8f7de62f1e 100644 --- a/api/src/main/java/org/apache/iceberg/ContentFile.java +++ b/api/src/main/java/org/apache/iceberg/ContentFile.java @@ -63,7 +63,8 @@ public interface ContentFile { Map columnSizes(); /** - * Returns if collected, map from column ID to the count of its non-null values, null otherwise. + * Returns if collected, map from column ID to the count of its values (including null and NaN + * values), null otherwise. */ Map valueCounts(); diff --git a/api/src/main/java/org/apache/iceberg/FileScanTask.java b/api/src/main/java/org/apache/iceberg/FileScanTask.java index d99d924370ad..5fb4b55459e3 100644 --- a/api/src/main/java/org/apache/iceberg/FileScanTask.java +++ b/api/src/main/java/org/apache/iceberg/FileScanTask.java @@ -29,6 +29,11 @@ public interface FileScanTask extends ContentScanTask, SplittableScanT */ List deletes(); + /** Return the schema for this file scan task. */ + default Schema schema() { + throw new UnsupportedOperationException("Does not support schema getter"); + } + @Override default long sizeBytes() { return length() + deletes().stream().mapToLong(ContentFile::fileSizeInBytes).sum(); diff --git a/api/src/main/java/org/apache/iceberg/ManageSnapshots.java b/api/src/main/java/org/apache/iceberg/ManageSnapshots.java index 81caf3a58de3..2fa60472da5e 100644 --- a/api/src/main/java/org/apache/iceberg/ManageSnapshots.java +++ b/api/src/main/java/org/apache/iceberg/ManageSnapshots.java @@ -83,6 +83,19 @@ public interface ManageSnapshots extends PendingUpdate { */ ManageSnapshots cherrypick(long snapshotId); + /** + * Create a new branch. The branch will point to current snapshot if the current snapshot is not + * NULL. Otherwise, the branch will point to a newly created empty snapshot. + * + * @param name branch name + * @return this for method chaining + * @throws IllegalArgumentException if a branch with the given name already exists + */ + default ManageSnapshots createBranch(String name) { + throw new UnsupportedOperationException( + this.getClass().getName() + " doesn't implement createBranch(String)"); + } + /** * Create a new branch pointing to the given snapshot id. * diff --git a/api/src/main/java/org/apache/iceberg/actions/DeleteOrphanFiles.java b/api/src/main/java/org/apache/iceberg/actions/DeleteOrphanFiles.java index a57da3bed098..4e8f80fa833f 100644 --- a/api/src/main/java/org/apache/iceberg/actions/DeleteOrphanFiles.java +++ b/api/src/main/java/org/apache/iceberg/actions/DeleteOrphanFiles.java @@ -24,7 +24,6 @@ import java.util.function.Consumer; import org.apache.iceberg.io.SupportsBulkOperations; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.immutables.value.Value; /** * An action that deletes orphan metadata, data and delete files in a table. @@ -32,7 +31,6 @@ *

A file is considered orphan if it is not reachable by any valid snapshot. The set of actual * files is built by listing the underlying storage which makes this operation expensive. */ -@Value.Enclosing public interface DeleteOrphanFiles extends Action { /** * Passes a location which should be scanned for orphan files. @@ -141,7 +139,6 @@ default DeleteOrphanFiles equalAuthorities(Map newEqualAuthoriti } /** The action result that contains a summary of the execution. */ - @Value.Immutable interface Result { /** Returns locations of orphan files. */ Iterable orphanFileLocations(); diff --git a/api/src/main/java/org/apache/iceberg/actions/DeleteReachableFiles.java b/api/src/main/java/org/apache/iceberg/actions/DeleteReachableFiles.java index e8110284954f..d4ebb5a2007c 100644 --- a/api/src/main/java/org/apache/iceberg/actions/DeleteReachableFiles.java +++ b/api/src/main/java/org/apache/iceberg/actions/DeleteReachableFiles.java @@ -22,7 +22,6 @@ import java.util.function.Consumer; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.SupportsBulkOperations; -import org.immutables.value.Value; /** * An action that deletes all files referenced by a table metadata file. @@ -33,7 +32,6 @@ * *

Implementations may use a query engine to distribute parts of work. */ -@Value.Enclosing public interface DeleteReachableFiles extends Action { @@ -67,7 +65,6 @@ public interface DeleteReachableFiles DeleteReachableFiles io(FileIO io); /** The action result that contains a summary of the execution. */ - @Value.Immutable interface Result { /** Returns the number of deleted data files. */ diff --git a/api/src/main/java/org/apache/iceberg/actions/ExpireSnapshots.java b/api/src/main/java/org/apache/iceberg/actions/ExpireSnapshots.java index 286ab115d14f..610d2d97248e 100644 --- a/api/src/main/java/org/apache/iceberg/actions/ExpireSnapshots.java +++ b/api/src/main/java/org/apache/iceberg/actions/ExpireSnapshots.java @@ -22,7 +22,6 @@ import java.util.function.Consumer; import org.apache.iceberg.Snapshot; import org.apache.iceberg.io.SupportsBulkOperations; -import org.immutables.value.Value; /** * An action that expires snapshots in a table. @@ -30,7 +29,6 @@ *

Similar to {@link org.apache.iceberg.ExpireSnapshots} but may use a query engine to distribute * parts of the work. */ -@Value.Enclosing public interface ExpireSnapshots extends Action { /** * Expires a specific {@link Snapshot} identified by id. @@ -100,7 +98,6 @@ public interface ExpireSnapshots extends Action { /** * Sets table properties in the newly created Iceberg table. Any properties with the same key name @@ -53,7 +51,6 @@ default MigrateTable dropBackup() { } /** The action result that contains a summary of the execution. */ - @Value.Immutable interface Result { /** Returns the number of migrated data files. */ long migratedDataFilesCount(); 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 20e75159b1e8..854b0993513e 100644 --- a/api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java +++ b/api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java @@ -24,13 +24,11 @@ import org.apache.iceberg.StructLike; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.immutables.value.Value; /** * An action for rewriting data files according to a rewrite strategy. Generally used for optimizing * the sizing and layout of data files within a table. */ -@Value.Enclosing public interface RewriteDataFiles extends SnapshotUpdate { @@ -178,33 +176,27 @@ default RewriteDataFiles zOrder(String... columns) { * null then that particular file group failed. We should only have failed groups if partial * progress is enabled otherwise we will report a total failure for the job. */ - @Value.Immutable interface Result { List rewriteResults(); - @Value.Default default List rewriteFailures() { return ImmutableList.of(); } - @Value.Default default int addedDataFilesCount() { return rewriteResults().stream().mapToInt(FileGroupRewriteResult::addedDataFilesCount).sum(); } - @Value.Default default int rewrittenDataFilesCount() { return rewriteResults().stream() .mapToInt(FileGroupRewriteResult::rewrittenDataFilesCount) .sum(); } - @Value.Default default long rewrittenBytesCount() { return rewriteResults().stream().mapToLong(FileGroupRewriteResult::rewrittenBytesCount).sum(); } - @Value.Default default int failedDataFilesCount() { return rewriteFailures().stream().mapToInt(FileGroupFailureResult::dataFilesCount).sum(); } @@ -214,7 +206,6 @@ default int failedDataFilesCount() { * For a particular file group, the number of files which are newly created and the number of * files which were formerly part of the table but have been rewritten. */ - @Value.Immutable interface FileGroupRewriteResult { FileGroupInfo info(); @@ -222,14 +213,12 @@ interface FileGroupRewriteResult { int rewrittenDataFilesCount(); - @Value.Default default long rewrittenBytesCount() { return 0L; } } /** For a file group that failed to rewrite. */ - @Value.Immutable interface FileGroupFailureResult { FileGroupInfo info(); @@ -240,7 +229,6 @@ interface FileGroupFailureResult { * A description of a file group, when it was processed, and within which partition. For use * tracking rewrite operations and for returning results. */ - @Value.Immutable interface FileGroupInfo { /** returns which file group this is out of the total set of file groups for this rewrite */ diff --git a/api/src/main/java/org/apache/iceberg/actions/RewriteManifests.java b/api/src/main/java/org/apache/iceberg/actions/RewriteManifests.java index 836ec6d4a726..57b35aaf6596 100644 --- a/api/src/main/java/org/apache/iceberg/actions/RewriteManifests.java +++ b/api/src/main/java/org/apache/iceberg/actions/RewriteManifests.java @@ -20,10 +20,8 @@ import java.util.function.Predicate; import org.apache.iceberg.ManifestFile; -import org.immutables.value.Value; /** An action that rewrites manifests. */ -@Value.Enclosing public interface RewriteManifests extends SnapshotUpdate { /** @@ -57,7 +55,6 @@ public interface RewriteManifests RewriteManifests stagingLocation(String stagingLocation); /** The action result that contains a summary of the execution. */ - @Value.Immutable interface Result { /** Returns rewritten manifests. */ Iterable rewrittenManifests(); diff --git a/api/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteFiles.java b/api/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteFiles.java index 96548ae07bee..0c05433a8b96 100644 --- a/api/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteFiles.java +++ b/api/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteFiles.java @@ -22,14 +22,12 @@ import org.apache.iceberg.RewriteJobOrder; import org.apache.iceberg.StructLike; import org.apache.iceberg.expressions.Expression; -import org.immutables.value.Value; /** * An action for rewriting position delete files. * *

Generally used for optimizing the size and layout of position delete files within a table. */ -@Value.Enclosing public interface RewritePositionDeleteFiles extends SnapshotUpdate { @@ -93,7 +91,6 @@ public interface RewritePositionDeleteFiles RewritePositionDeleteFiles filter(Expression expression); /** The action result that contains a summary of the execution. */ - @Value.Immutable interface Result { List rewriteResults(); @@ -127,7 +124,6 @@ default long addedBytesCount() { * newly created and the number of files which were formerly part of the table but have been * rewritten. */ - @Value.Immutable interface FileGroupRewriteResult { /** Description of this position delete file group. */ FileGroupInfo info(); @@ -149,7 +145,6 @@ interface FileGroupRewriteResult { * A description of a position delete file group, when it was processed, and within which * partition. For use tracking rewrite operations and for returning results. */ - @Value.Immutable interface FileGroupInfo { /** * Returns which position delete file group this is out of the total set of file groups for this diff --git a/api/src/main/java/org/apache/iceberg/actions/SnapshotTable.java b/api/src/main/java/org/apache/iceberg/actions/SnapshotTable.java index d009ed0e18f1..37c600ab0392 100644 --- a/api/src/main/java/org/apache/iceberg/actions/SnapshotTable.java +++ b/api/src/main/java/org/apache/iceberg/actions/SnapshotTable.java @@ -19,10 +19,8 @@ package org.apache.iceberg.actions; import java.util.Map; -import org.immutables.value.Value; /** An action that creates an independent snapshot of an existing table. */ -@Value.Enclosing public interface SnapshotTable extends Action { /** * Sets the table identifier for the newly created Iceberg table. @@ -60,7 +58,6 @@ public interface SnapshotTable extends ActionNumbers are replaced with magnitude and type, string-like values are replaced by hashes, and + * date/time values are replaced by the type. + * + * @param struct a StructType to bind the expression + * @param expr an Expression to sanitize + * @param caseSensitive whether to bind case sensitively + * @return a sanitized Expression + */ + public static Expression sanitize( + Types.StructType struct, Expression expr, boolean caseSensitive) { + try { + Expression bound = Binder.bind(struct, expr, caseSensitive); + return ExpressionVisitors.visit(bound, new ExpressionSanitizer()); + } catch (RuntimeException e) { + // if the expression cannot be bound, sanitize the unbound version + return ExpressionVisitors.visit(expr, new ExpressionSanitizer()); + } + } + /** * Produces a sanitized expression string with the same structure, but with data values replaced * by descriptions. @@ -84,6 +108,29 @@ public static String toSanitizedString(Expression expr) { return ExpressionVisitors.visit(expr, new StringSanitizer()); } + /** + * Produces a sanitized expression string with the same structure, but with data values replaced + * by descriptions. + * + *

Numbers are replaced with magnitude and type, string-like values are replaced by hashes, and + * date/time values are replaced by the type. + * + * @param struct a StructType to bind the expression + * @param expr an Expression to sanitize + * @param caseSensitive whether to bind case sensitively + * @return a sanitized expression string + */ + public static String toSanitizedString( + Types.StructType struct, Expression expr, boolean caseSensitive) { + try { + Expression bound = Binder.bind(struct, expr, caseSensitive); + return ExpressionVisitors.visit(bound, new StringSanitizer()); + } catch (RuntimeException e) { + // if the expression cannot be bound, sanitize the unbound version + return ExpressionVisitors.visit(expr, new StringSanitizer()); + } + } + /** * Extracts an expression that references only the given column IDs from the given expression. * @@ -176,6 +223,28 @@ public static String describe(Term term) { } } + public static UnboundTerm unbind(BoundTerm term) { + if (term instanceof BoundTransform) { + BoundTransform bound = (BoundTransform) term; + return Expressions.transform(bound.ref().name(), bound.transform()); + } else if (term instanceof BoundReference) { + return Expressions.ref(((BoundReference) term).name()); + } + + throw new UnsupportedOperationException("Cannot unbind unsupported term: " + term); + } + + @SuppressWarnings("unchecked") + public static UnboundTerm unbind(Term term) { + if (term instanceof UnboundTerm) { + return (UnboundTerm) term; + } else if (term instanceof BoundTerm) { + return unbind((BoundTerm) term); + } + + throw new UnsupportedOperationException("Cannot unbind unsupported term: " + term); + } + private static class ExpressionSanitizer extends ExpressionVisitors.ExpressionVisitor { private final long now; @@ -214,8 +283,28 @@ public Expression or(Expression leftResult, Expression rightResult) { } @Override + @SuppressWarnings("unchecked") public Expression predicate(BoundPredicate pred) { - throw new UnsupportedOperationException("Cannot sanitize bound predicate: " + pred); + if (pred.isUnaryPredicate()) { + // unary predicates don't need to be sanitized + return new UnboundPredicate<>(pred.op(), unbind(pred.term())); + } else if (pred.isLiteralPredicate()) { + BoundLiteralPredicate bound = (BoundLiteralPredicate) pred; + return new UnboundPredicate<>( + pred.op(), + unbind(pred.term()), + (T) sanitize(bound.term().type(), bound.literal(), now, today)); + } else if (pred.isSetPredicate()) { + BoundSetPredicate bound = (BoundSetPredicate) pred; + Iterable iter = + () -> + bound.literalSet().stream() + .map(lit -> (T) sanitize(bound.term().type(), lit, now, today)) + .iterator(); + return new UnboundPredicate<>(pred.op(), unbind(pred.term()), iter); + } + + throw new UnsupportedOperationException("Cannot sanitize bound predicate type: " + pred.op()); } @Override @@ -286,9 +375,60 @@ public String or(String leftResult, String rightResult) { return "(" + leftResult + " OR " + rightResult + ")"; } + private String value(BoundLiteralPredicate pred) { + return sanitize(pred.term().type(), pred.literal().value(), nowMicros, today); + } + @Override public String predicate(BoundPredicate pred) { - throw new UnsupportedOperationException("Cannot sanitize bound predicate: " + pred); + String term = describe(pred.term()); + switch (pred.op()) { + case IS_NULL: + return term + " IS NULL"; + case NOT_NULL: + return term + " IS NOT NULL"; + case IS_NAN: + return "is_nan(" + term + ")"; + case NOT_NAN: + return "not_nan(" + term + ")"; + case LT: + return term + " < " + value((BoundLiteralPredicate) pred); + case LT_EQ: + return term + " <= " + value((BoundLiteralPredicate) pred); + case GT: + return term + " > " + value((BoundLiteralPredicate) pred); + case GT_EQ: + return term + " >= " + value((BoundLiteralPredicate) pred); + case EQ: + return term + " = " + value((BoundLiteralPredicate) pred); + case NOT_EQ: + return term + " != " + value((BoundLiteralPredicate) pred); + case IN: + return term + + " IN " + + abbreviateValues( + pred.asSetPredicate().literalSet().stream() + .map(lit -> sanitize(pred.term().type(), lit, nowMicros, today)) + .collect(Collectors.toList())) + .stream() + .collect(Collectors.joining(", ", "(", ")")); + case NOT_IN: + return term + + " NOT IN " + + abbreviateValues( + pred.asSetPredicate().literalSet().stream() + .map(lit -> sanitize(pred.term().type(), lit, nowMicros, today)) + .collect(Collectors.toList())) + .stream() + .collect(Collectors.joining(", ", "(", ")")); + case STARTS_WITH: + return term + " STARTS WITH " + value((BoundLiteralPredicate) pred); + case NOT_STARTS_WITH: + return term + " NOT STARTS WITH " + value((BoundLiteralPredicate) pred); + default: + throw new UnsupportedOperationException( + "Cannot sanitize unsupported predicate type: " + pred.op()); + } } @Override @@ -361,6 +501,34 @@ private static List abbreviateValues(List sanitizedValues) { return sanitizedValues; } + private static String sanitize(Type type, Object value, long now, int today) { + switch (type.typeId()) { + case INTEGER: + case LONG: + return sanitizeNumber((Number) value, "int"); + case FLOAT: + case DOUBLE: + return sanitizeNumber((Number) value, "float"); + case DATE: + return sanitizeDate((int) value, today); + case TIME: + return "(time)"; + case TIMESTAMP: + return sanitizeTimestamp((long) value, now); + case STRING: + return sanitizeString((CharSequence) value, now, today); + case BOOLEAN: + case UUID: + case DECIMAL: + case FIXED: + case BINARY: + // for boolean, uuid, decimal, fixed, and binary, match the string result + return sanitizeSimpleString(value.toString()); + } + throw new UnsupportedOperationException( + String.format("Cannot sanitize value for unsupported type %s: %s", type, value)); + } + private static String sanitize(Literal literal, long now, int today) { if (literal instanceof Literals.StringLiteral) { return sanitizeString(((Literals.StringLiteral) literal).value(), now, today); diff --git a/api/src/main/java/org/apache/iceberg/view/ReplaceViewVersion.java b/api/src/main/java/org/apache/iceberg/view/ReplaceViewVersion.java new file mode 100644 index 000000000000..b876933345ed --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/view/ReplaceViewVersion.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.view; + +import org.apache.iceberg.PendingUpdate; + +/** + * API for replacing a view's version. + * + *

Apply returns the updated view version for validation. + * + *

When committing, these changes will be applied to the current view metadata. Commit conflicts + * will be resolved by applying the pending changes to the new view metadata. + */ +public interface ReplaceViewVersion + extends PendingUpdate, VersionBuilder {} diff --git a/api/src/main/java/org/apache/iceberg/view/VersionBuilder.java b/api/src/main/java/org/apache/iceberg/view/VersionBuilder.java new file mode 100644 index 000000000000..0c69c8fc9e76 --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/view/VersionBuilder.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.view; + +import org.apache.iceberg.Schema; +import org.apache.iceberg.catalog.Namespace; + +public interface VersionBuilder { + /** + * Set the view schema. + * + * @param schema The schema to use for this view version + * @return this for method chaining + */ + T withSchema(Schema schema); + + /** + * Add a view representation for the given dialect and the SQL to the view. + * + * @param dialect The dialect of the view representation + * @param sql The SQL of the view representation + * @return this for method chaining + */ + T withQuery(String dialect, String sql); + + /** + * Set the default catalog to use for the view. + * + * @param catalog The default catalog to use when the SQL does not contain a catalog + * @return this for method chaining + */ + T withDefaultCatalog(String catalog); + + /** + * Set the default namespace to use for the view. + * + * @param namespace The default namespace to use when the SQL does not contain a namespace + * @return this for method chaining + */ + T withDefaultNamespace(Namespace namespace); +} diff --git a/api/src/main/java/org/apache/iceberg/view/View.java b/api/src/main/java/org/apache/iceberg/view/View.java index 0f4d1cc9c0e8..284c561b7848 100644 --- a/api/src/main/java/org/apache/iceberg/view/View.java +++ b/api/src/main/java/org/apache/iceberg/view/View.java @@ -83,4 +83,13 @@ public interface View { * @return a new {@link UpdateViewProperties} */ UpdateViewProperties updateProperties(); + + /** + * Create a new {@link ReplaceViewVersion} to replace the view's current version. + * + * @return a new {@link ReplaceViewVersion} + */ + default ReplaceViewVersion replaceVersion() { + throw new UnsupportedOperationException("Replacing a view's version is not supported"); + } } diff --git a/api/src/main/java/org/apache/iceberg/view/ViewBuilder.java b/api/src/main/java/org/apache/iceberg/view/ViewBuilder.java index 57f8a970867b..02620de72289 100644 --- a/api/src/main/java/org/apache/iceberg/view/ViewBuilder.java +++ b/api/src/main/java/org/apache/iceberg/view/ViewBuilder.java @@ -18,10 +18,7 @@ */ package org.apache.iceberg.view; -import java.util.List; import java.util.Map; -import org.apache.iceberg.Schema; -import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.ViewCatalog; /** @@ -29,70 +26,7 @@ * *

Call {@link ViewCatalog#buildView} to create a new builder. */ -public interface ViewBuilder { - /** - * Set the view schema. - * - * @param schema view schema - * @return this for method chaining - */ - ViewBuilder withSchema(Schema schema); - - /** - * Set the view query. - * - * @param query view query - * @return this for method chaining - */ - ViewBuilder withQuery(String query); - - /** - * Set the view SQL dialect. - * - * @param dialect view SQL dialect - * @return this for method chaining - */ - ViewBuilder withDialect(String dialect); - - /** - * Set the view default catalog. - * - * @param defaultCatalog view default catalog - * @return this for method chaining - */ - ViewBuilder withDefaultCatalog(String defaultCatalog); - - /** - * Set the view default namespace. - * - * @param defaultNamespace view default namespace - * @return this for method chaining - */ - ViewBuilder withDefaultNamespace(Namespace defaultNamespace); - - /** - * Set the view query column names. - * - * @param queryColumnNames view query column names - * @return this for method chaining - */ - ViewBuilder withQueryColumnNames(List queryColumnNames); - - /** - * Set the view field aliases. - * - * @param fieldAliases view field aliases - * @return this for method chaining - */ - ViewBuilder withFieldAliases(List fieldAliases); - - /** - * Set the view field comments. - * - * @param fieldComments view field comments - * @return this for method chaining - */ - ViewBuilder withFieldComments(List fieldComments); +public interface ViewBuilder extends VersionBuilder { /** * Add key/value properties to the view. diff --git a/api/src/main/java/org/apache/iceberg/view/ViewHistoryEntry.java b/api/src/main/java/org/apache/iceberg/view/ViewHistoryEntry.java index 351a27ce81c5..2840560655ed 100644 --- a/api/src/main/java/org/apache/iceberg/view/ViewHistoryEntry.java +++ b/api/src/main/java/org/apache/iceberg/view/ViewHistoryEntry.java @@ -18,15 +18,12 @@ */ package org.apache.iceberg.view; -import org.immutables.value.Value; - /** * View history entry. * *

An entry contains a change to the view state. At the given timestamp, the current version was * set to the given version ID. */ -@Value.Immutable public interface ViewHistoryEntry { /** Return the timestamp in milliseconds of the change */ long timestampMillis(); diff --git a/api/src/main/java/org/apache/iceberg/view/ViewVersion.java b/api/src/main/java/org/apache/iceberg/view/ViewVersion.java index 11dec1ff16c9..bf292f1073f4 100644 --- a/api/src/main/java/org/apache/iceberg/view/ViewVersion.java +++ b/api/src/main/java/org/apache/iceberg/view/ViewVersion.java @@ -20,8 +20,8 @@ import java.util.List; import java.util.Map; +import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.immutables.value.Value; /** * A version of the view at a point in time. @@ -30,7 +30,6 @@ * *

Versions are created by view operations, like Create and Replace. */ -@Value.Immutable public interface ViewVersion { /** Return this version's id. Version ids are monotonically increasing */ @@ -66,7 +65,6 @@ public interface ViewVersion { * * @return the string operation which produced the view version */ - @Value.Lazy default String operation() { return summary().get("operation"); } @@ -74,7 +72,16 @@ default String operation() { /** The query output schema at version create time, without aliases */ int schemaId(); - @Value.Check + /** The default catalog when the view is created. */ + default String defaultCatalog() { + return null; + } + + /** The default namespace to use when the SQL does not contain a namespace. */ + default Namespace defaultNamespace() { + return null; + } + default void check() { Preconditions.checkArgument( summary().containsKey("operation"), "Invalid view version summary, missing operation"); diff --git a/api/src/test/java/org/apache/iceberg/TestHelpers.java b/api/src/test/java/org/apache/iceberg/TestHelpers.java index 7e2a27d8b81c..153e2de7ea9a 100644 --- a/api/src/test/java/org/apache/iceberg/TestHelpers.java +++ b/api/src/test/java/org/apache/iceberg/TestHelpers.java @@ -28,8 +28,11 @@ import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.io.InputStream; import java.io.ObjectInputStream; import java.io.ObjectOutputStream; +import java.io.OutputStream; +import java.io.Serializable; import java.lang.invoke.SerializedLambda; import java.nio.ByteBuffer; import java.util.Arrays; @@ -41,6 +44,7 @@ import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.ExpressionVisitors; import org.apache.iceberg.expressions.UnboundPredicate; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.util.ByteBuffers; import org.assertj.core.api.Assertions; import org.objenesis.strategy.StdInstantiatorStrategy; @@ -168,6 +172,99 @@ public static void assertSameSchemaMap(Map map1, MapIf the call site incorrectly types the return value, a {@link ClassCastException} is thrown + * from the call site. Without Generics in this declaration, the call site must type cast and can + * cause the same ClassCastException. Note that in both cases, the ClassCastException is in the + * call site, not in this method. + * + *

This code is borrowed from `org.apache.commons:commons-lang3` + * + * @param the object type to be deserialized + * @param objectData the serialized object, must not be null + * @return the deserialized object + * @throws NullPointerException if {@code objectData} is {@code null} + * @throws IOException (runtime) if the serialization fails + */ + public static T deserialize(final byte[] objectData) + throws IOException, ClassNotFoundException { + Preconditions.checkNotNull(objectData, "objectData"); + return deserialize(new ByteArrayInputStream(objectData)); + } + + /** + * Deserializes an {@link Object} from the specified stream. + * + *

The stream will be closed once the object is written. This avoids the need for a finally + * clause, and maybe also exception handling, in the application code. + * + *

The stream passed in is not buffered internally within this method. This is the + * responsibility of your application if desired. + * + *

If the call site incorrectly types the return value, a {@link ClassCastException} is thrown + * from the call site. Without Generics in this declaration, the call site must type cast and can + * cause the same ClassCastException. Note that in both cases, the ClassCastException is in the + * call site, not in this method. + * + *

This code is borrowed from `org.apache.commons:commons-lang3` + * + * @param the object type to be deserialized + * @param inputStream the serialized object input stream, must not be null + * @return the deserialized object + * @throws NullPointerException if {@code inputStream} is {@code null} + * @throws IOException (runtime) if the serialization fails + * @throws ClassNotFoundException if Class is not found + */ + public static T deserialize(final InputStream inputStream) + throws IOException, ClassNotFoundException { + Preconditions.checkNotNull(inputStream, "inputStream"); + try (ObjectInputStream in = new ObjectInputStream(inputStream)) { + @SuppressWarnings("unchecked") + final T obj = (T) in.readObject(); + return obj; + } + } + /** + * Serializes an {@link Object} to a byte array for storage/serialization. + * + *

This code is borrowed from `org.apache.commons:commons-lang3` + * + * @param obj the object to serialize to bytes + * @return a byte[] with the converted Serializable + * @throws IOException (runtime) if the serialization fails + */ + public static byte[] serialize(final Serializable obj) throws IOException { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(512); + serialize(obj, baos); + return baos.toByteArray(); + } + + /** + * Serializes an {@link Object} to the specified stream. + * + *

The stream will be closed once the object is written. This avoids the need for a finally + * clause, and maybe also exception handling, in the application code. + * + *

The stream passed in is not buffered internally within this method. This is the + * responsibility of your application if desired. + * + *

This code is borrowed from `org.apache.commons:commons-lang3` + * + * @param obj the object to serialize to bytes, may be null + * @param outputStream the stream to write to, must not be null + * @throws NullPointerException if {@code outputStream} is {@code null} + * @throws IOException (runtime) if the serialization fails + */ + public static void serialize(final Serializable obj, final OutputStream outputStream) + throws IOException { + Preconditions.checkNotNull(outputStream, "outputStream"); + try (ObjectOutputStream out = new ObjectOutputStream(outputStream)) { + out.writeObject(obj); + } + } + public static class KryoHelpers { private KryoHelpers() {} diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestExpressionUtil.java b/api/src/test/java/org/apache/iceberg/expressions/TestExpressionUtil.java index 2bfa9bc20aca..9a27830543ad 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestExpressionUtil.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestExpressionUtil.java @@ -46,10 +46,14 @@ public class TestExpressionUtil { Types.NestedField.required(5, "date", Types.DateType.get()), Types.NestedField.required(6, "time", Types.DateType.get()), Types.NestedField.optional(7, "data", Types.StringType.get()), - Types.NestedField.optional(8, "measurement", Types.DoubleType.get())); + Types.NestedField.optional(8, "measurement", Types.DoubleType.get()), + Types.NestedField.optional(9, "test", Types.IntegerType.get())); private static final Types.StructType STRUCT = SCHEMA.asStruct(); + private static final Types.StructType FLOAT_TEST = + Types.StructType.of(Types.NestedField.optional(1, "test", Types.FloatType.get())); + @Test public void testUnchangedUnaryPredicates() { for (Expression unary : @@ -59,6 +63,7 @@ public void testUnchangedUnaryPredicates() { Expressions.isNaN("test"), Expressions.notNaN("test"))) { assertEquals(unary, ExpressionUtil.sanitize(unary)); + assertEquals(unary, ExpressionUtil.sanitize(FLOAT_TEST, unary, true)); } } @@ -68,9 +73,17 @@ public void testSanitizeIn() { Expressions.in("test", "(2-digit-int)", "(3-digit-int)"), ExpressionUtil.sanitize(Expressions.in("test", 34, 345))); + assertEquals( + Expressions.in("test", "(2-digit-int)", "(3-digit-int)"), + ExpressionUtil.sanitize(STRUCT, Expressions.in("test", 34, 345), true)); + assertThat(ExpressionUtil.toSanitizedString(Expressions.in("test", 34, 345))) .as("Sanitized string should be identical except for descriptive literal") .isEqualTo("test IN ((2-digit-int), (3-digit-int))"); + + assertThat(ExpressionUtil.toSanitizedString(STRUCT, Expressions.in("test", 34, 345), true)) + .as("Sanitized string should be identical except for descriptive literal") + .isEqualTo("test IN ((2-digit-int), (3-digit-int))"); } @Test @@ -126,6 +139,10 @@ public void testSanitizeNotIn() { assertThat(ExpressionUtil.toSanitizedString(Expressions.notIn("test", 34, 345))) .as("Sanitized string should be identical except for descriptive literal") .isEqualTo("test NOT IN ((2-digit-int), (3-digit-int))"); + + assertThat(ExpressionUtil.toSanitizedString(STRUCT, Expressions.notIn("test", 34, 345), true)) + .as("Sanitized string should be identical except for descriptive literal") + .isEqualTo("test NOT IN ((2-digit-int), (3-digit-int))"); } @Test @@ -161,9 +178,17 @@ public void testSanitizeLessThan() { Expressions.lessThan("test", "(2-digit-int)"), ExpressionUtil.sanitize(Expressions.lessThan("test", 34))); + assertEquals( + Expressions.lessThan("test", "(2-digit-int)"), + ExpressionUtil.sanitize(STRUCT, Expressions.lessThan("test", 34), true)); + assertThat(ExpressionUtil.toSanitizedString(Expressions.lessThan("test", 34))) .as("Sanitized string should be identical except for descriptive literal") .isEqualTo("test < (2-digit-int)"); + + assertThat(ExpressionUtil.toSanitizedString(STRUCT, Expressions.lessThan("test", 34), true)) + .as("Sanitized string should be identical except for descriptive literal") + .isEqualTo("test < (2-digit-int)"); } @Test @@ -172,9 +197,18 @@ public void testSanitizeLessThanOrEqual() { Expressions.lessThanOrEqual("test", "(2-digit-int)"), ExpressionUtil.sanitize(Expressions.lessThanOrEqual("test", 34))); + assertEquals( + Expressions.lessThanOrEqual("test", "(2-digit-int)"), + ExpressionUtil.sanitize(STRUCT, Expressions.lessThanOrEqual("test", 34), true)); + assertThat(ExpressionUtil.toSanitizedString(Expressions.lessThanOrEqual("test", 34))) .as("Sanitized string should be identical except for descriptive literal") .isEqualTo("test <= (2-digit-int)"); + + assertThat( + ExpressionUtil.toSanitizedString(STRUCT, Expressions.lessThanOrEqual("test", 34), true)) + .as("Sanitized string should be identical except for descriptive literal") + .isEqualTo("test <= (2-digit-int)"); } @Test @@ -183,9 +217,17 @@ public void testSanitizeGreaterThan() { Expressions.greaterThan("test", "(2-digit-int)"), ExpressionUtil.sanitize(Expressions.greaterThan("test", 34))); + assertEquals( + Expressions.greaterThan("test", "(2-digit-int)"), + ExpressionUtil.sanitize(STRUCT, Expressions.greaterThan("test", 34), true)); + assertThat(ExpressionUtil.toSanitizedString(Expressions.greaterThan("test", 34))) .as("Sanitized string should be identical except for descriptive literal") .isEqualTo("test > (2-digit-int)"); + + assertThat(ExpressionUtil.toSanitizedString(STRUCT, Expressions.greaterThan("test", 34), true)) + .as("Sanitized string should be identical except for descriptive literal") + .isEqualTo("test > (2-digit-int)"); } @Test @@ -194,9 +236,19 @@ public void testSanitizeGreaterThanOrEqual() { Expressions.greaterThanOrEqual("test", "(2-digit-int)"), ExpressionUtil.sanitize(Expressions.greaterThanOrEqual("test", 34))); + assertEquals( + Expressions.greaterThanOrEqual("test", "(2-digit-int)"), + ExpressionUtil.sanitize(STRUCT, Expressions.greaterThanOrEqual("test", 34), true)); + assertThat(ExpressionUtil.toSanitizedString(Expressions.greaterThanOrEqual("test", 34))) .as("Sanitized string should be identical except for descriptive literal") .isEqualTo("test >= (2-digit-int)"); + + assertThat( + ExpressionUtil.toSanitizedString( + STRUCT, Expressions.greaterThanOrEqual("test", 34), true)) + .as("Sanitized string should be identical except for descriptive literal") + .isEqualTo("test >= (2-digit-int)"); } @Test @@ -205,9 +257,17 @@ public void testSanitizeEqual() { Expressions.equal("test", "(2-digit-int)"), ExpressionUtil.sanitize(Expressions.equal("test", 34))); + assertEquals( + Expressions.equal("test", "(2-digit-int)"), + ExpressionUtil.sanitize(STRUCT, Expressions.equal("test", 34), true)); + assertThat(ExpressionUtil.toSanitizedString(Expressions.equal("test", 34))) .as("Sanitized string should be identical except for descriptive literal") .isEqualTo("test = (2-digit-int)"); + + assertThat(ExpressionUtil.toSanitizedString(STRUCT, Expressions.equal("test", 34), true)) + .as("Sanitized string should be identical except for descriptive literal") + .isEqualTo("test = (2-digit-int)"); } @Test @@ -216,9 +276,17 @@ public void testSanitizeNotEqual() { Expressions.notEqual("test", "(2-digit-int)"), ExpressionUtil.sanitize(Expressions.notEqual("test", 34))); + assertEquals( + Expressions.notEqual("test", "(2-digit-int)"), + ExpressionUtil.sanitize(STRUCT, Expressions.notEqual("test", 34), true)); + assertThat(ExpressionUtil.toSanitizedString(Expressions.notEqual("test", 34))) .as("Sanitized string should be identical except for descriptive literal") .isEqualTo("test != (2-digit-int)"); + + assertThat(ExpressionUtil.toSanitizedString(STRUCT, Expressions.notEqual("test", 34), true)) + .as("Sanitized string should be identical except for descriptive literal") + .isEqualTo("test != (2-digit-int)"); } @Test @@ -227,9 +295,18 @@ public void testSanitizeStartsWith() { Expressions.startsWith("test", "(hash-34d05fb7)"), ExpressionUtil.sanitize(Expressions.startsWith("test", "aaa"))); + assertEquals( + Expressions.startsWith("data", "(hash-34d05fb7)"), + ExpressionUtil.sanitize(STRUCT, Expressions.startsWith("data", "aaa"), true)); + assertThat(ExpressionUtil.toSanitizedString(Expressions.startsWith("test", "aaa"))) .as("Sanitized string should be identical except for descriptive literal") .isEqualTo("test STARTS WITH (hash-34d05fb7)"); + + assertThat( + ExpressionUtil.toSanitizedString(STRUCT, Expressions.startsWith("data", "aaa"), true)) + .as("Sanitized string should be identical except for descriptive literal") + .isEqualTo("data STARTS WITH (hash-34d05fb7)"); } @Test @@ -238,9 +315,19 @@ public void testSanitizeNotStartsWith() { Expressions.notStartsWith("test", "(hash-34d05fb7)"), ExpressionUtil.sanitize(Expressions.notStartsWith("test", "aaa"))); + assertEquals( + Expressions.notStartsWith("data", "(hash-34d05fb7)"), + ExpressionUtil.sanitize(STRUCT, Expressions.notStartsWith("data", "aaa"), true)); + assertThat(ExpressionUtil.toSanitizedString(Expressions.notStartsWith("test", "aaa"))) .as("Sanitized string should be identical except for descriptive literal") .isEqualTo("test NOT STARTS WITH (hash-34d05fb7)"); + + assertThat( + ExpressionUtil.toSanitizedString( + STRUCT, Expressions.notStartsWith("data", "aaa"), true)) + .as("Sanitized string should be identical except for descriptive literal") + .isEqualTo("data NOT STARTS WITH (hash-34d05fb7)"); } @Test @@ -249,11 +336,22 @@ public void testSanitizeTransformedTerm() { Expressions.equal(Expressions.truncate("test", 2), "(2-digit-int)"), ExpressionUtil.sanitize(Expressions.equal(Expressions.truncate("test", 2), 34))); + assertEquals( + Expressions.equal(Expressions.truncate("test", 2), "(2-digit-int)"), + ExpressionUtil.sanitize( + STRUCT, Expressions.equal(Expressions.truncate("test", 2), 34), true)); + assertThat( ExpressionUtil.toSanitizedString( Expressions.equal(Expressions.truncate("test", 2), 34))) .as("Sanitized string should be identical except for descriptive literal") .isEqualTo("truncate[2](test) = (2-digit-int)"); + + assertThat( + ExpressionUtil.toSanitizedString( + STRUCT, Expressions.equal(Expressions.truncate("test", 2), 34), true)) + .as("Sanitized string should be identical except for descriptive literal") + .isEqualTo("truncate[2](test) = (2-digit-int)"); } @Test @@ -262,9 +360,17 @@ public void testSanitizeLong() { Expressions.equal("test", "(2-digit-int)"), ExpressionUtil.sanitize(Expressions.equal("test", 34L))); + assertEquals( + Expressions.equal("id", "(2-digit-int)"), + ExpressionUtil.sanitize(STRUCT, Expressions.equal("id", 34L), true)); + assertThat(ExpressionUtil.toSanitizedString(Expressions.equal("test", 34L))) .as("Sanitized string should be identical except for descriptive literal") .isEqualTo("test = (2-digit-int)"); + + assertThat(ExpressionUtil.toSanitizedString(STRUCT, Expressions.equal("id", 34L), true)) + .as("Sanitized string should be identical except for descriptive literal") + .isEqualTo("id = (2-digit-int)"); } @Test @@ -273,9 +379,17 @@ public void testSanitizeFloat() { Expressions.equal("test", "(2-digit-float)"), ExpressionUtil.sanitize(Expressions.equal("test", 34.12F))); + assertEquals( + Expressions.equal("test", "(2-digit-float)"), + ExpressionUtil.sanitize(STRUCT, Expressions.equal("test", 34.12F), true)); + assertThat(ExpressionUtil.toSanitizedString(Expressions.equal("test", 34.12F))) .as("Sanitized string should be identical except for descriptive literal") .isEqualTo("test = (2-digit-float)"); + + assertThat(ExpressionUtil.toSanitizedString(STRUCT, Expressions.equal("test", 34.12F), true)) + .as("Sanitized string should be identical except for descriptive literal") + .isEqualTo("test = (2-digit-float)"); } @Test @@ -284,9 +398,17 @@ public void testSanitizeDouble() { Expressions.equal("test", "(2-digit-float)"), ExpressionUtil.sanitize(Expressions.equal("test", 34.12D))); + assertEquals( + Expressions.equal("measurement", "(2-digit-float)"), + ExpressionUtil.sanitize(STRUCT, Expressions.equal("measurement", 34.12D), true)); + assertThat(ExpressionUtil.toSanitizedString(Expressions.equal("test", 34.12D))) .as("Sanitized string should be identical except for descriptive literal") .isEqualTo("test = (2-digit-float)"); + + assertThat(ExpressionUtil.toSanitizedString(Expressions.equal("measurement", 34.12D))) + .as("Sanitized string should be identical except for descriptive literal") + .isEqualTo("measurement = (2-digit-float)"); } @Test @@ -295,9 +417,18 @@ public void testSanitizeDate() { Expressions.equal("test", "(date)"), ExpressionUtil.sanitize(Expressions.equal("test", "2022-04-29"))); + assertEquals( + Expressions.equal("date", "(date)"), + ExpressionUtil.sanitize(STRUCT, Expressions.equal("date", "2022-04-29"), true)); + assertThat(ExpressionUtil.toSanitizedString(Expressions.equal("test", "2022-04-29"))) .as("Sanitized string should be identical except for descriptive literal") .isEqualTo("test = (date)"); + + assertThat( + ExpressionUtil.toSanitizedString(STRUCT, Expressions.equal("date", "2022-04-29"), true)) + .as("Sanitized string should be identical except for descriptive literal") + .isEqualTo("date = (date)"); } @Test @@ -309,9 +440,18 @@ public void testSanitizeTime() { Expressions.equal("test", "(time)"), ExpressionUtil.sanitize(Expressions.equal("test", currentTime))); + assertEquals( + Expressions.equal("time", "(time)"), + ExpressionUtil.sanitize(STRUCT, Expressions.equal("time", currentTime), true)); + assertThat(ExpressionUtil.toSanitizedString(Expressions.equal("test", currentTime))) .as("Sanitized string should be identical except for descriptive literal") .isEqualTo("test = (time)"); + + assertThat( + ExpressionUtil.toSanitizedString(STRUCT, Expressions.equal("time", currentTime), true)) + .as("Sanitized string should be identical except for descriptive literal") + .isEqualTo("time = (time)"); } @Test @@ -326,9 +466,17 @@ public void testSanitizeTimestamp() { Expressions.equal("test", "(timestamp)"), ExpressionUtil.sanitize(Expressions.equal("test", timestamp))); + assertEquals( + Expressions.equal("ts", "(timestamp)"), + ExpressionUtil.sanitize(STRUCT, Expressions.equal("ts", timestamp), true)); + assertThat(ExpressionUtil.toSanitizedString(Expressions.equal("test", timestamp))) .as("Sanitized string should be identical except for descriptive literal") .isEqualTo("test = (timestamp)"); + + assertThat(ExpressionUtil.toSanitizedString(STRUCT, Expressions.equal("ts", timestamp), true)) + .as("Sanitized string should be identical except for descriptive literal") + .isEqualTo("ts = (timestamp)"); } } diff --git a/arrow/src/test/java/org/apache/iceberg/arrow/ArrowSchemaUtilTest.java b/arrow/src/test/java/org/apache/iceberg/arrow/ArrowSchemaUtilTest.java index 7568abd46009..8dc75c1a2db3 100644 --- a/arrow/src/test/java/org/apache/iceberg/arrow/ArrowSchemaUtilTest.java +++ b/arrow/src/test/java/org/apache/iceberg/arrow/ArrowSchemaUtilTest.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.arrow; +import static org.assertj.core.api.Assertions.assertThat; + import org.apache.arrow.vector.types.pojo.ArrowType; import org.apache.arrow.vector.types.pojo.Field; import org.apache.iceberg.Schema; @@ -34,8 +36,7 @@ import org.apache.iceberg.types.Types.StringType; import org.apache.iceberg.types.Types.TimeType; import org.apache.iceberg.types.Types.TimestampType; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class ArrowSchemaUtilTest { @@ -97,86 +98,86 @@ public void convertComplex() { MapType.ofOptional( 4, 5, StringType.get(), ListType.ofOptional(6, TimestampType.withoutZone())))); org.apache.arrow.vector.types.pojo.Schema arrow = ArrowSchemaUtil.convert(iceberg); - Assert.assertEquals(iceberg.columns().size(), arrow.getFields().size()); + assertThat(arrow.getFields()).hasSameSizeAs(iceberg.columns()); } private void validate(Schema iceberg, org.apache.arrow.vector.types.pojo.Schema arrow) { - Assert.assertEquals(iceberg.columns().size(), arrow.getFields().size()); + assertThat(arrow.getFields()).hasSameSizeAs(iceberg.columns()); for (Types.NestedField nf : iceberg.columns()) { Field field = arrow.findField(nf.name()); - Assert.assertNotNull("Missing filed: " + nf, field); + assertThat(field).as("Missing field: " + nf).isNotNull(); validate(nf.type(), field, nf.isOptional()); } } private void validate(Type iceberg, Field field, boolean optional) { ArrowType arrowType = field.getType(); - Assert.assertEquals(optional, field.isNullable()); + assertThat(field.isNullable()).isEqualTo(optional); switch (iceberg.typeId()) { case BOOLEAN: - Assert.assertEquals(BOOLEAN_FIELD, field.getName()); - Assert.assertEquals(ArrowType.ArrowTypeID.Bool, arrowType.getTypeID()); + assertThat(field.getName()).isEqualTo(BOOLEAN_FIELD); + assertThat(arrowType.getTypeID()).isEqualTo(ArrowType.ArrowTypeID.Bool); break; case INTEGER: - Assert.assertEquals(INTEGER_FIELD, field.getName()); - Assert.assertEquals(ArrowType.ArrowTypeID.Int, arrowType.getTypeID()); + assertThat(field.getName()).isEqualTo(INTEGER_FIELD); + assertThat(arrowType.getTypeID()).isEqualTo(ArrowType.ArrowTypeID.Int); break; case LONG: - Assert.assertEquals(LONG_FIELD, field.getName()); - Assert.assertEquals(ArrowType.ArrowTypeID.Int, arrowType.getTypeID()); + assertThat(field.getName()).isEqualTo(LONG_FIELD); + assertThat(arrowType.getTypeID()).isEqualTo(ArrowType.ArrowTypeID.Int); break; case FLOAT: - Assert.assertEquals(FLOAT_FIELD, field.getName()); - Assert.assertEquals(ArrowType.ArrowTypeID.FloatingPoint, arrowType.getTypeID()); + assertThat(field.getName()).isEqualTo(FLOAT_FIELD); + assertThat(arrowType.getTypeID()).isEqualTo(ArrowType.ArrowTypeID.FloatingPoint); break; case DOUBLE: - Assert.assertEquals(DOUBLE_FIELD, field.getName()); - Assert.assertEquals(ArrowType.ArrowTypeID.FloatingPoint, arrowType.getTypeID()); + assertThat(field.getName()).isEqualTo(DOUBLE_FIELD); + assertThat(arrowType.getTypeID()).isEqualTo(ArrowType.ArrowTypeID.FloatingPoint); break; case DATE: - Assert.assertEquals(DATE_FIELD, field.getName()); - Assert.assertEquals(ArrowType.ArrowTypeID.Date, arrowType.getTypeID()); + assertThat(field.getName()).isEqualTo(DATE_FIELD); + assertThat(arrowType.getTypeID()).isEqualTo(ArrowType.ArrowTypeID.Date); break; case TIME: - Assert.assertEquals(TIME_FIELD, field.getName()); - Assert.assertEquals(ArrowType.ArrowTypeID.Time, arrowType.getTypeID()); + assertThat(field.getName()).isEqualTo(TIME_FIELD); + assertThat(arrowType.getTypeID()).isEqualTo(ArrowType.ArrowTypeID.Time); break; case TIMESTAMP: - Assert.assertEquals(TIMESTAMP_FIELD, field.getName()); - Assert.assertEquals(ArrowType.ArrowTypeID.Timestamp, arrowType.getTypeID()); + assertThat(field.getName()).isEqualTo(TIMESTAMP_FIELD); + assertThat(arrowType.getTypeID()).isEqualTo(ArrowType.ArrowTypeID.Timestamp); break; case STRING: - Assert.assertEquals(STRING_FIELD, field.getName()); - Assert.assertEquals(ArrowType.ArrowTypeID.Utf8, arrowType.getTypeID()); + assertThat(field.getName()).isEqualTo(STRING_FIELD); + assertThat(arrowType.getTypeID()).isEqualTo(ArrowType.ArrowTypeID.Utf8); break; case FIXED: - Assert.assertEquals(FIXED_WIDTH_BINARY_FIELD, field.getName()); - Assert.assertEquals(ArrowType.FixedSizeBinary.TYPE_TYPE, arrowType.getTypeID()); + assertThat(field.getName()).isEqualTo(FIXED_WIDTH_BINARY_FIELD); + assertThat(arrowType.getTypeID()).isEqualTo(ArrowType.FixedSizeBinary.TYPE_TYPE); break; case BINARY: - Assert.assertEquals(BINARY_FIELD, field.getName()); - Assert.assertEquals(ArrowType.Binary.TYPE_TYPE, arrowType.getTypeID()); + assertThat(field.getName()).isEqualTo(BINARY_FIELD); + assertThat(arrowType.getTypeID()).isEqualTo(ArrowType.Binary.TYPE_TYPE); break; case DECIMAL: - Assert.assertEquals(DECIMAL_FIELD, field.getName()); - Assert.assertEquals(ArrowType.Decimal.TYPE_TYPE, arrowType.getTypeID()); + assertThat(field.getName()).isEqualTo(DECIMAL_FIELD); + assertThat(arrowType.getTypeID()).isEqualTo(ArrowType.Decimal.TYPE_TYPE); break; case STRUCT: - Assert.assertEquals(STRUCT_FIELD, field.getName()); - Assert.assertEquals(ArrowType.Struct.TYPE_TYPE, arrowType.getTypeID()); + assertThat(field.getName()).isEqualTo(STRUCT_FIELD); + assertThat(arrowType.getTypeID()).isEqualTo(ArrowType.Struct.TYPE_TYPE); break; case LIST: - Assert.assertEquals(LIST_FIELD, field.getName()); - Assert.assertEquals(ArrowType.List.TYPE_TYPE, arrowType.getTypeID()); + assertThat(field.getName()).isEqualTo(LIST_FIELD); + assertThat(arrowType.getTypeID()).isEqualTo(ArrowType.List.TYPE_TYPE); break; case MAP: - Assert.assertEquals(MAP_FIELD, field.getName()); - Assert.assertEquals(ArrowType.ArrowTypeID.Map, arrowType.getTypeID()); + assertThat(field.getName()).isEqualTo(MAP_FIELD); + assertThat(arrowType.getTypeID()).isEqualTo(ArrowType.ArrowTypeID.Map); break; case UUID: - Assert.assertEquals(UUID_FIELD, field.getName()); - Assert.assertEquals(ArrowType.FixedSizeBinary.TYPE_TYPE, arrowType.getTypeID()); + assertThat(field.getName()).isEqualTo(UUID_FIELD); + assertThat(arrowType.getTypeID()).isEqualTo(ArrowType.FixedSizeBinary.TYPE_TYPE); break; default: throw new UnsupportedOperationException("Check not implemented for type: " + iceberg); diff --git a/arrow/src/test/java/org/apache/iceberg/arrow/vectorized/ArrowReaderTest.java b/arrow/src/test/java/org/apache/iceberg/arrow/vectorized/ArrowReaderTest.java index 983c5ba1b56d..9cd9c8cc5abf 100644 --- a/arrow/src/test/java/org/apache/iceberg/arrow/vectorized/ArrowReaderTest.java +++ b/arrow/src/test/java/org/apache/iceberg/arrow/vectorized/ArrowReaderTest.java @@ -19,8 +19,7 @@ package org.apache.iceberg.arrow.vectorized; import static org.apache.iceberg.Files.localInput; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; @@ -86,9 +85,9 @@ import org.apache.iceberg.types.Types; import org.apache.iceberg.util.UUIDUtil; import org.assertj.core.api.Assertions; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; /** * Test cases for {@link ArrowReader}. @@ -127,14 +126,17 @@ public class ArrowReaderTest { "uuid_nullable", "decimal", "decimal_nullable"); - - @Rule public final TemporaryFolder temp = new TemporaryFolder(); + @TempDir private File tempDir; private HadoopTables tables; - private String tableLocation; private List rowsWritten; + @BeforeEach + public void before() { + tableLocation = tempDir.toURI().toString(); + } + /** * Read all rows and columns from the table without any filter. The test asserts that the Arrow * {@link VectorSchemaRoot} contains the expected schema and expected vector types. Then the test @@ -225,7 +227,7 @@ public void testReadRangeFilterEmptyResult() throws Exception { numRoots++; } } - assertEquals(0, numRoots); + assertThat(numRoots).isZero(); } /** @@ -322,14 +324,14 @@ private void readAndCheckArrowResult( for (ColumnarBatch batch : itr) { List expectedRows = rowsWritten.subList(rowIndex, rowIndex + numRowsPerRoot); VectorSchemaRoot root = batch.createVectorSchemaRootFromVectors(); - assertEquals(createExpectedArrowSchema(columnSet), root.getSchema()); + assertThat(root.getSchema()).isEqualTo(createExpectedArrowSchema(columnSet)); checkAllVectorTypes(root, columnSet); checkAllVectorValues(numRowsPerRoot, expectedRows, root, columnSet); rowIndex += numRowsPerRoot; totalRows += root.getRowCount(); } } - assertEquals(expectedTotalRows, totalRows); + assertThat(totalRows).isEqualTo(expectedTotalRows); } private void readAndCheckHasNextIsIdempotent( @@ -349,12 +351,12 @@ private void readAndCheckHasNextIsIdempotent( // Call hasNext() a few extra times. // This should not affect the total number of rows read. for (int i = 0; i < numExtraCallsToHasNext; i++) { - assertTrue(iterator.hasNext()); + assertThat(iterator).hasNext(); } ColumnarBatch batch = iterator.next(); VectorSchemaRoot root = batch.createVectorSchemaRootFromVectors(); - assertEquals(createExpectedArrowSchema(columnSet), root.getSchema()); + assertThat(root.getSchema()).isEqualTo(createExpectedArrowSchema(columnSet)); checkAllVectorTypes(root, columnSet); List expectedRows = rowsWritten.subList(rowIndex, rowIndex + numRowsPerRoot); checkAllVectorValues(numRowsPerRoot, expectedRows, root, columnSet); @@ -362,7 +364,7 @@ private void readAndCheckHasNextIsIdempotent( totalRows += root.getRowCount(); } } - assertEquals(expectedTotalRows, totalRows); + assertThat(totalRows).isEqualTo(expectedTotalRows); } @SuppressWarnings("MethodLength") @@ -378,8 +380,8 @@ private void checkColumnarBatch( } Set columnSet = columnNameToIndex.keySet(); - assertEquals(expectedNumRows, batch.numRows()); - assertEquals(columns.size(), batch.numCols()); + assertThat(batch.numRows()).isEqualTo(expectedNumRows); + assertThat(batch.numCols()).isEqualTo(columns.size()); checkColumnarArrayValues( expectedNumRows, @@ -667,7 +669,7 @@ private void writeTableWithIncrementalRecords() throws Exception { private void writeTable(boolean constantRecords) throws Exception { rowsWritten = Lists.newArrayList(); tables = new HadoopTables(); - tableLocation = temp.newFolder("test").toString(); + tableLocation = tempDir.toURI().toString(); Schema schema = new Schema( @@ -871,8 +873,8 @@ private List createConstantRecordsForDate(Schema schema, LocalDat private DataFile writeParquetFile(Table table, List records) throws IOException { rowsWritten.addAll(records); - File parquetFile = temp.newFile(); - assertTrue(parquetFile.delete()); + File parquetFile = File.createTempFile("junit", null, tempDir); + assertThat(parquetFile.delete()).isTrue(); FileAppender appender = Parquet.write(Files.localOutput(parquetFile)) .schema(table.schema()) @@ -949,7 +951,7 @@ private void checkAllVectorTypes(VectorSchemaRoot root, Set columnSet) { private void assertEqualsForField( VectorSchemaRoot root, Set columnSet, String columnName, Class expected) { if (columnSet.contains(columnName)) { - assertEquals(expected, root.getVector(columnName).getClass()); + assertThat(root.getVector(columnName).getClass()).isEqualTo(expected); } } @@ -959,7 +961,7 @@ private void checkAllVectorValues( List expectedRows, VectorSchemaRoot root, Set columnSet) { - assertEquals(expectedNumRows, root.getRowCount()); + assertThat(root.getRowCount()).isEqualTo(expectedNumRows); checkVectorValues( expectedNumRows, @@ -1196,7 +1198,7 @@ private static void checkVectorValues( BiFunction vectorValueExtractor) { if (columnSet.contains(columnName)) { FieldVector vector = root.getVector(columnName); - assertEquals(expectedNumRows, vector.getValueCount()); + assertThat(vector.getValueCount()).isEqualTo(expectedNumRows); for (int i = 0; i < expectedNumRows; i++) { Object expectedValue = expectedValueExtractor.apply(expectedRows, i); Object actualValue = vectorValueExtractor.apply(vector, i); diff --git a/arrow/src/test/java/org/apache/iceberg/arrow/vectorized/parquet/DecimalVectorUtilTest.java b/arrow/src/test/java/org/apache/iceberg/arrow/vectorized/parquet/DecimalVectorUtilTest.java index 4e78bafd0a1a..88e16c18cbb9 100644 --- a/arrow/src/test/java/org/apache/iceberg/arrow/vectorized/parquet/DecimalVectorUtilTest.java +++ b/arrow/src/test/java/org/apache/iceberg/arrow/vectorized/parquet/DecimalVectorUtilTest.java @@ -18,11 +18,11 @@ */ package org.apache.iceberg.arrow.vectorized.parquet; -import static org.junit.Assert.assertEquals; +import static org.assertj.core.api.Assertions.assertThat; import java.math.BigInteger; import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class DecimalVectorUtilTest { @@ -32,9 +32,9 @@ public void testPadBigEndianBytes() { byte[] bytes = bigInt.toByteArray(); byte[] paddedBytes = DecimalVectorUtil.padBigEndianBytes(bytes, 16); - assertEquals(16, paddedBytes.length); + assertThat(paddedBytes).hasSize(16); BigInteger result = new BigInteger(paddedBytes); - assertEquals(bigInt, result); + assertThat(result).isEqualTo(bigInt); } @Test @@ -43,9 +43,9 @@ public void testPadBigEndianBytesNegative() { byte[] bytes = bigInt.toByteArray(); byte[] paddedBytes = DecimalVectorUtil.padBigEndianBytes(bytes, 16); - assertEquals(16, paddedBytes.length); + assertThat(paddedBytes).hasSize(16); BigInteger result = new BigInteger(paddedBytes); - assertEquals(bigInt, result); + assertThat(result).isEqualTo(bigInt); } @Test @@ -53,16 +53,16 @@ public void testPadBigEndianBytesZero() { byte[] bytes = BigInteger.ZERO.toByteArray(); byte[] paddedBytes = DecimalVectorUtil.padBigEndianBytes(bytes, 16); - assertEquals(16, paddedBytes.length); + assertThat(paddedBytes).hasSize(16); BigInteger result = new BigInteger(paddedBytes); - assertEquals(BigInteger.ZERO, result); + assertThat(result).isEqualTo(BigInteger.ZERO); bytes = new byte[0]; paddedBytes = DecimalVectorUtil.padBigEndianBytes(bytes, 16); - assertEquals(16, paddedBytes.length); + assertThat(paddedBytes).hasSize(16); result = new BigInteger(paddedBytes); - assertEquals(BigInteger.ZERO, result); + assertThat(result).isEqualTo(BigInteger.ZERO); } @Test 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 7dc4f2223932..c12c3f58e0de 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 @@ -33,7 +33,7 @@ import java.util.UUID; import javax.crypto.KeyGenerator; import javax.crypto.SecretKey; -import org.apache.commons.lang3.SerializationUtils; +import org.apache.iceberg.TestHelpers; import org.apache.iceberg.aws.AwsClientFactories; import org.apache.iceberg.aws.AwsClientFactory; import org.apache.iceberg.aws.AwsIntegTestUtil; @@ -354,8 +354,8 @@ public void testACL() throws Exception { public void testClientFactorySerialization() throws Exception { S3FileIO fileIO = new S3FileIO(clientFactory::s3); write(fileIO); - byte[] data = SerializationUtils.serialize(fileIO); - S3FileIO fileIO2 = SerializationUtils.deserialize(data); + byte[] data = TestHelpers.serialize(fileIO); + S3FileIO fileIO2 = TestHelpers.deserialize(data); validateRead(fileIO2); } diff --git a/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientFactories.java b/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientFactories.java index 8b4f46a23a8b..8b19d339a75b 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientFactories.java +++ b/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientFactories.java @@ -29,8 +29,7 @@ import org.apache.iceberg.util.SerializationUtil; import org.assertj.core.api.Assertions; import org.assertj.core.api.ThrowableAssert; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; import software.amazon.awssdk.auth.credentials.AwsCredentials; import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; @@ -45,23 +44,22 @@ public class TestAwsClientFactories { @Test public void testLoadDefault() { - Assert.assertEquals( - "default client should be singleton", - AwsClientFactories.defaultFactory(), - AwsClientFactories.defaultFactory()); - - Assert.assertTrue( - "should load default when not configured", - AwsClientFactories.from(Maps.newHashMap()) - instanceof AwsClientFactories.DefaultAwsClientFactory); + Assertions.assertThat(AwsClientFactories.defaultFactory()) + .as("default client should be singleton") + .isSameAs(AwsClientFactories.defaultFactory()); + + Assertions.assertThat(AwsClientFactories.from(Maps.newHashMap())) + .as("should load default when not configured") + .isInstanceOf(AwsClientFactories.DefaultAwsClientFactory.class); } @Test public void testLoadCustom() { Map properties = Maps.newHashMap(); properties.put(AwsProperties.CLIENT_FACTORY, CustomFactory.class.getName()); - Assert.assertTrue( - "should load custom class", AwsClientFactories.from(properties) instanceof CustomFactory); + Assertions.assertThat(AwsClientFactories.from(properties)) + .as("should load custom class") + .isInstanceOf(CustomFactory.class); } @Test diff --git a/aws/src/test/java/org/apache/iceberg/aws/TestAwsProperties.java b/aws/src/test/java/org/apache/iceberg/aws/TestAwsProperties.java index 0ea57a466325..556968ec22eb 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/TestAwsProperties.java +++ b/aws/src/test/java/org/apache/iceberg/aws/TestAwsProperties.java @@ -28,8 +28,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.mockito.ArgumentCaptor; import org.mockito.Mockito; import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; @@ -72,7 +71,7 @@ public void testS3FileIoAcl() { Map map = Maps.newHashMap(); map.put(AwsProperties.S3FILEIO_ACL, ObjectCannedACL.AUTHENTICATED_READ.toString()); AwsProperties properties = new AwsProperties(map); - Assert.assertEquals(ObjectCannedACL.AUTHENTICATED_READ, properties.s3FileIoAcl()); + Assertions.assertThat(properties.s3FileIoAcl()).isEqualTo(ObjectCannedACL.AUTHENTICATED_READ); } @Test @@ -148,9 +147,9 @@ public void testS3FileIoDefaultCredentialsConfiguration() { Mockito.verify(mockS3ClientBuilder).credentialsProvider(awsCredentialsProviderCaptor.capture()); AwsCredentialsProvider capturedAwsCredentialsProvider = awsCredentialsProviderCaptor.getValue(); - Assert.assertTrue( - "Should use default credentials if nothing is set", - capturedAwsCredentialsProvider instanceof DefaultCredentialsProvider); + Assertions.assertThat(capturedAwsCredentialsProvider) + .as("Should use default credentials if nothing is set") + .isInstanceOf(DefaultCredentialsProvider.class); } @Test @@ -168,17 +167,16 @@ public void testS3FileIoBasicCredentialsConfiguration() { Mockito.verify(mockS3ClientBuilder).credentialsProvider(awsCredentialsProviderCaptor.capture()); AwsCredentialsProvider capturedAwsCredentialsProvider = awsCredentialsProviderCaptor.getValue(); - Assert.assertTrue( - "Should use basic credentials if access key ID and secret access key are set", - capturedAwsCredentialsProvider.resolveCredentials() instanceof AwsBasicCredentials); - Assert.assertEquals( - "The access key id should be the same as the one set by tag S3FILEIO_ACCESS_KEY_ID", - "key", - capturedAwsCredentialsProvider.resolveCredentials().accessKeyId()); - Assert.assertEquals( - "The secret access key should be the same as the one set by tag S3FILEIO_SECRET_ACCESS_KEY", - "secret", - capturedAwsCredentialsProvider.resolveCredentials().secretAccessKey()); + Assertions.assertThat(capturedAwsCredentialsProvider.resolveCredentials()) + .as("Should use basic credentials if access key ID and secret access key are set") + .isInstanceOf(AwsBasicCredentials.class); + Assertions.assertThat(capturedAwsCredentialsProvider.resolveCredentials().accessKeyId()) + .as("The access key id should be the same as the one set by tag S3FILEIO_ACCESS_KEY_ID") + .isEqualTo("key"); + Assertions.assertThat(capturedAwsCredentialsProvider.resolveCredentials().secretAccessKey()) + .as( + "The secret access key should be the same as the one set by tag S3FILEIO_SECRET_ACCESS_KEY") + .isEqualTo("secret"); } @Test @@ -197,17 +195,16 @@ public void testS3FileIoSessionCredentialsConfiguration() { Mockito.verify(mockS3ClientBuilder).credentialsProvider(awsCredentialsProviderCaptor.capture()); AwsCredentialsProvider capturedAwsCredentialsProvider = awsCredentialsProviderCaptor.getValue(); - Assert.assertTrue( - "Should use session credentials if session token is set", - capturedAwsCredentialsProvider.resolveCredentials() instanceof AwsSessionCredentials); - Assert.assertEquals( - "The access key id should be the same as the one set by tag S3FILEIO_ACCESS_KEY_ID", - "key", - capturedAwsCredentialsProvider.resolveCredentials().accessKeyId()); - Assert.assertEquals( - "The secret access key should be the same as the one set by tag S3FILEIO_SECRET_ACCESS_KEY", - "secret", - capturedAwsCredentialsProvider.resolveCredentials().secretAccessKey()); + Assertions.assertThat(capturedAwsCredentialsProvider.resolveCredentials()) + .as("Should use session credentials if session token is set") + .isInstanceOf(AwsSessionCredentials.class); + Assertions.assertThat(capturedAwsCredentialsProvider.resolveCredentials().accessKeyId()) + .as("The access key id should be the same as the one set by tag S3FILEIO_ACCESS_KEY_ID") + .isEqualTo("key"); + Assertions.assertThat(capturedAwsCredentialsProvider.resolveCredentials().secretAccessKey()) + .as( + "The secret access key should be the same as the one set by tag S3FILEIO_SECRET_ACCESS_KEY") + .isEqualTo("secret"); } @Test @@ -223,9 +220,9 @@ public void testUrlHttpClientConfiguration() { Mockito.verify(mockS3ClientBuilder).httpClientBuilder(httpClientBuilderCaptor.capture()); SdkHttpClient.Builder capturedHttpClientBuilder = httpClientBuilderCaptor.getValue(); - Assert.assertTrue( - "Should use url connection http client", - capturedHttpClientBuilder instanceof UrlConnectionHttpClient.Builder); + Assertions.assertThat(capturedHttpClientBuilder) + .as("Should use url connection http client") + .isInstanceOf(UrlConnectionHttpClient.Builder.class); } @Test @@ -240,9 +237,9 @@ public void testApacheHttpClientConfiguration() { awsProperties.applyHttpClientConfigurations(mockS3ClientBuilder); Mockito.verify(mockS3ClientBuilder).httpClientBuilder(httpClientBuilderCaptor.capture()); SdkHttpClient.Builder capturedHttpClientBuilder = httpClientBuilderCaptor.getValue(); - Assert.assertTrue( - "Should use apache http client", - capturedHttpClientBuilder instanceof ApacheHttpClient.Builder); + Assertions.assertThat(capturedHttpClientBuilder) + .as("Should use apache http client") + .isInstanceOf(ApacheHttpClient.Builder.class); } @Test @@ -263,29 +260,26 @@ public void testKryoSerialization() throws IOException { AwsProperties awsProperties = new AwsProperties(); AwsProperties deSerializedAwsProperties = TestHelpers.KryoHelpers.roundTripSerialize(awsProperties); - Assert.assertEquals( - awsProperties.s3BucketToAccessPointMapping(), - deSerializedAwsProperties.s3BucketToAccessPointMapping()); - Assert.assertEquals( - awsProperties.httpClientProperties(), deSerializedAwsProperties.httpClientProperties()); + Assertions.assertThat(deSerializedAwsProperties.s3BucketToAccessPointMapping()) + .isEqualTo(awsProperties.s3BucketToAccessPointMapping()); + Assertions.assertThat(deSerializedAwsProperties.httpClientProperties()) + .isEqualTo(awsProperties.httpClientProperties()); AwsProperties awsPropertiesWithProps = new AwsProperties(ImmutableMap.of("a", "b")); AwsProperties deSerializedAwsPropertiesWithProps = TestHelpers.KryoHelpers.roundTripSerialize(awsPropertiesWithProps); - Assert.assertEquals( - awsPropertiesWithProps.s3BucketToAccessPointMapping(), - deSerializedAwsPropertiesWithProps.s3BucketToAccessPointMapping()); - Assert.assertEquals( - awsProperties.httpClientProperties(), deSerializedAwsProperties.httpClientProperties()); + Assertions.assertThat(deSerializedAwsPropertiesWithProps.s3BucketToAccessPointMapping()) + .isEqualTo(awsPropertiesWithProps.s3BucketToAccessPointMapping()); + Assertions.assertThat(deSerializedAwsPropertiesWithProps.httpClientProperties()) + .isEqualTo(awsProperties.httpClientProperties()); AwsProperties awsPropertiesWithEmptyProps = new AwsProperties(Collections.emptyMap()); AwsProperties deSerializedAwsPropertiesWithEmptyProps = TestHelpers.KryoHelpers.roundTripSerialize(awsPropertiesWithProps); - Assert.assertEquals( - awsPropertiesWithEmptyProps.s3BucketToAccessPointMapping(), - deSerializedAwsPropertiesWithEmptyProps.s3BucketToAccessPointMapping()); - Assert.assertEquals( - awsProperties.httpClientProperties(), deSerializedAwsProperties.httpClientProperties()); + Assertions.assertThat(deSerializedAwsPropertiesWithEmptyProps.s3BucketToAccessPointMapping()) + .isEqualTo(awsPropertiesWithEmptyProps.s3BucketToAccessPointMapping()); + Assertions.assertThat(deSerializedAwsPropertiesWithEmptyProps.httpClientProperties()) + .isEqualTo(awsProperties.httpClientProperties()); } @Test diff --git a/aws/src/test/java/org/apache/iceberg/aws/TestHttpClientConfigurations.java b/aws/src/test/java/org/apache/iceberg/aws/TestHttpClientConfigurations.java index 7e380861cebe..2c6f35c3a048 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/TestHttpClientConfigurations.java +++ b/aws/src/test/java/org/apache/iceberg/aws/TestHttpClientConfigurations.java @@ -21,7 +21,7 @@ import java.time.Duration; import java.util.Map; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; import software.amazon.awssdk.http.apache.ApacheHttpClient; import software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient; diff --git a/aws/src/test/java/org/apache/iceberg/aws/TestRESTSigV4Signer.java b/aws/src/test/java/org/apache/iceberg/aws/TestRESTSigV4Signer.java index 188d488db2eb..b35363502939 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/TestRESTSigV4Signer.java +++ b/aws/src/test/java/org/apache/iceberg/aws/TestRESTSigV4Signer.java @@ -29,10 +29,10 @@ import org.apache.iceberg.rest.responses.ConfigResponse; import org.apache.iceberg.rest.responses.OAuthTokenResponse; import org.assertj.core.api.Assertions; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; +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.Header; import org.mockserver.model.HttpRequest; @@ -44,7 +44,7 @@ public class TestRESTSigV4Signer { private static ClientAndServer mockServer; private static HTTPClient client; - @BeforeClass + @BeforeAll public static void beforeClass() { mockServer = ClientAndServer.startClientAndServer(); @@ -66,13 +66,13 @@ public static void beforeClass() { .build(); } - @AfterClass + @AfterAll public static void afterClass() throws IOException { mockServer.stop(); client.close(); } - @Before + @BeforeEach public void before() { mockServer.reset(); } diff --git a/aws/src/test/java/org/apache/iceberg/aws/dynamodb/TestDynamoDbCatalog.java b/aws/src/test/java/org/apache/iceberg/aws/dynamodb/TestDynamoDbCatalog.java index 9e6e9484088a..5b5941c9a6e6 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/dynamodb/TestDynamoDbCatalog.java +++ b/aws/src/test/java/org/apache/iceberg/aws/dynamodb/TestDynamoDbCatalog.java @@ -27,8 +27,8 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.assertj.core.api.Assertions; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; import software.amazon.awssdk.services.dynamodb.DynamoDbClient; import software.amazon.awssdk.services.dynamodb.model.AttributeValue; @@ -44,7 +44,7 @@ public class TestDynamoDbCatalog { private DynamoDbClient dynamo; private DynamoDbCatalog dynamoCatalog; - @Before + @BeforeEach public void before() { dynamo = Mockito.mock(DynamoDbClient.class); dynamoCatalog = new DynamoDbCatalog(); diff --git a/aws/src/test/java/org/apache/iceberg/aws/glue/TestGlueCatalog.java b/aws/src/test/java/org/apache/iceberg/aws/glue/TestGlueCatalog.java index a915a97d0ece..1dded0ccaddd 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/glue/TestGlueCatalog.java +++ b/aws/src/test/java/org/apache/iceberg/aws/glue/TestGlueCatalog.java @@ -35,9 +35,8 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.util.LockManagers; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -72,7 +71,7 @@ public class TestGlueCatalog { private GlueClient glue; private GlueCatalog glueCatalog; - @Before + @BeforeEach public void before() { glue = Mockito.mock(GlueClient.class); glueCatalog = new GlueCatalog(); @@ -127,7 +126,7 @@ public void testConstructorWarehousePathWithEndSlash() { .when(glue) .getDatabase(Mockito.any(GetDatabaseRequest.class)); String location = catalogWithSlash.defaultWarehouseLocation(TableIdentifier.of("db", "table")); - Assert.assertEquals(WAREHOUSE_PATH + "/db.db/table", location); + Assertions.assertThat(location).isEqualTo(WAREHOUSE_PATH + "/db.db/table"); } @Test @@ -137,7 +136,7 @@ public void testDefaultWarehouseLocationNoDbUri() { .when(glue) .getDatabase(Mockito.any(GetDatabaseRequest.class)); String location = glueCatalog.defaultWarehouseLocation(TableIdentifier.of("db", "table")); - Assert.assertEquals(WAREHOUSE_PATH + "/db.db/table", location); + Assertions.assertThat(location).isEqualTo(WAREHOUSE_PATH + "/db.db/table"); } @Test @@ -149,7 +148,7 @@ public void testDefaultWarehouseLocationDbUri() { .when(glue) .getDatabase(Mockito.any(GetDatabaseRequest.class)); String location = glueCatalog.defaultWarehouseLocation(TableIdentifier.of("db", "table")); - Assert.assertEquals("s3://bucket2/db/table", location); + Assertions.assertThat(location).isEqualTo("s3://bucket2/db/table"); } @Test @@ -225,9 +224,9 @@ public void testListTables() { .build()) .when(glue) .getTables(Mockito.any(GetTablesRequest.class)); - Assert.assertEquals( - Lists.newArrayList(TableIdentifier.of("db1", "t1"), TableIdentifier.of("db1", "t2")), - glueCatalog.listTables(Namespace.of("db1"))); + Assertions.assertThat(glueCatalog.listTables(Namespace.of("db1"))) + .isEqualTo( + Lists.newArrayList(TableIdentifier.of("db1", "t1"), TableIdentifier.of("db1", "t2"))); } @Test @@ -271,7 +270,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { }) .when(glue) .getTables(Mockito.any(GetTablesRequest.class)); - Assert.assertEquals(10, glueCatalog.listTables(Namespace.of("db1")).size()); + Assertions.assertThat(glueCatalog.listTables(Namespace.of("db1"))).hasSize(10); } @Test @@ -329,7 +328,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { .when(glue) .deleteTable(Mockito.any(DeleteTableRequest.class)); glueCatalog.dropTable(TableIdentifier.of("db1", "t1")); - Assert.assertEquals(0, counter.get()); + Assertions.assertThat(counter.get()).isEqualTo(0); } @Test @@ -383,7 +382,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { .createTable(Mockito.any(CreateTableRequest.class)); glueCatalog.renameTable(TableIdentifier.of("db", "t"), TableIdentifier.of("db", "x_renamed")); - Assert.assertEquals(0, counter.get()); + Assertions.assertThat(counter.get()).isEqualTo(0); } @Test @@ -421,8 +420,8 @@ public void testListAllNamespaces() { .build()) .when(glue) .getDatabases(Mockito.any(GetDatabasesRequest.class)); - Assert.assertEquals( - Lists.newArrayList(Namespace.of("db1"), Namespace.of("db2")), glueCatalog.listNamespaces()); + Assertions.assertThat(glueCatalog.listNamespaces()) + .isEqualTo(Lists.newArrayList(Namespace.of("db1"), Namespace.of("db2"))); } @Test @@ -449,7 +448,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { }) .when(glue) .getDatabases(Mockito.any(GetDatabasesRequest.class)); - Assert.assertEquals(10, glueCatalog.listNamespaces().size()); + Assertions.assertThat(glueCatalog.listNamespaces()).hasSize(10); } @Test @@ -458,10 +457,9 @@ public void testListNamespacesWithNameShouldReturnItself() { GetDatabaseResponse.builder().database(Database.builder().name("db1").build()).build()) .when(glue) .getDatabase(Mockito.any(GetDatabaseRequest.class)); - Assert.assertEquals( - "list self should return empty list", - Lists.newArrayList(), - glueCatalog.listNamespaces(Namespace.of("db1"))); + Assertions.assertThat(glueCatalog.listNamespaces(Namespace.of("db1"))) + .as("list self should return empty list") + .isEmpty(); } @Test @@ -484,7 +482,8 @@ public void testLoadNamespaceMetadata() { .build()) .when(glue) .getDatabase(Mockito.any(GetDatabaseRequest.class)); - Assert.assertEquals(parameters, glueCatalog.loadNamespaceMetadata(Namespace.of("db1"))); + Assertions.assertThat(glueCatalog.loadNamespaceMetadata(Namespace.of("db1"))) + .isEqualTo(parameters); } @Test @@ -602,17 +601,13 @@ public void testTablePropsDefinedAtCatalogLevel() { null, catalogProps); Map properties = glueCatalog.properties(); - Assert.assertFalse(properties.isEmpty()); - Assert.assertTrue(properties.containsKey("table-default.key1")); - Assert.assertEquals("catalog-default-key1", properties.get("table-default.key1")); - Assert.assertTrue(properties.containsKey("table-default.key2")); - Assert.assertEquals("catalog-default-key2", properties.get("table-default.key2")); - Assert.assertTrue(properties.containsKey("table-default.key3")); - Assert.assertEquals("catalog-default-key3", properties.get("table-default.key3")); - Assert.assertTrue(properties.containsKey("table-override.key3")); - Assert.assertEquals("catalog-override-key3", properties.get("table-override.key3")); - Assert.assertTrue(properties.containsKey("table-override.key4")); - Assert.assertEquals("catalog-override-key4", properties.get("table-override.key4")); + Assertions.assertThat(properties) + .isNotEmpty() + .containsEntry("table-default.key1", "catalog-default-key1") + .containsEntry("table-default.key2", "catalog-default-key2") + .containsEntry("table-default.key3", "catalog-default-key3") + .containsEntry("table-override.key3", "catalog-override-key3") + .containsEntry("table-override.key4", "catalog-override-key4"); } @Test @@ -627,7 +622,8 @@ public void testValidateIdentifierSkipNameValidation() { LockManagers.defaultLockManager(), null, ImmutableMap.of()); - Assert.assertEquals(glueCatalog.isValidIdentifier(TableIdentifier.parse("db-1.a-1")), true); + Assertions.assertThat(glueCatalog.isValidIdentifier(TableIdentifier.parse("db-1.a-1"))) + .isEqualTo(true); } @Test @@ -652,19 +648,11 @@ public void testTableLevelS3TagProperties() { glueCatalog.newTableOps(TableIdentifier.of(Namespace.of("db"), "table")); Map tableCatalogProperties = glueTableOperations.tableCatalogProperties(); - Assert.assertTrue( - tableCatalogProperties.containsKey( - AwsProperties.S3_WRITE_TAGS_PREFIX.concat(AwsProperties.S3_TAG_ICEBERG_TABLE))); - Assert.assertEquals( - "table", - tableCatalogProperties.get( - AwsProperties.S3_WRITE_TAGS_PREFIX.concat(AwsProperties.S3_TAG_ICEBERG_TABLE))); - Assert.assertTrue( - tableCatalogProperties.containsKey( - AwsProperties.S3_WRITE_TAGS_PREFIX.concat(AwsProperties.S3_TAG_ICEBERG_NAMESPACE))); - Assert.assertEquals( - "db", - tableCatalogProperties.get( - AwsProperties.S3_WRITE_TAGS_PREFIX.concat(AwsProperties.S3_TAG_ICEBERG_NAMESPACE))); + Assertions.assertThat(tableCatalogProperties) + .containsEntry( + AwsProperties.S3_WRITE_TAGS_PREFIX.concat(AwsProperties.S3_TAG_ICEBERG_TABLE), "table") + .containsEntry( + AwsProperties.S3_WRITE_TAGS_PREFIX.concat(AwsProperties.S3_TAG_ICEBERG_NAMESPACE), + "db"); } } diff --git a/aws/src/test/java/org/apache/iceberg/aws/glue/TestGlueToIcebergConverter.java b/aws/src/test/java/org/apache/iceberg/aws/glue/TestGlueToIcebergConverter.java index e76f6c5ae08d..0d2c3d825c42 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/glue/TestGlueToIcebergConverter.java +++ b/aws/src/test/java/org/apache/iceberg/aws/glue/TestGlueToIcebergConverter.java @@ -25,8 +25,7 @@ import org.apache.iceberg.exceptions.NoSuchIcebergTableException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; import software.amazon.awssdk.services.glue.model.Database; import software.amazon.awssdk.services.glue.model.Table; @@ -36,14 +35,14 @@ public class TestGlueToIcebergConverter { public void testToNamespace() { Database database = Database.builder().name("db").build(); Namespace namespace = Namespace.of("db"); - Assert.assertEquals(namespace, GlueToIcebergConverter.toNamespace(database)); + Assertions.assertThat(GlueToIcebergConverter.toNamespace(database)).isEqualTo(namespace); } @Test public void testToTableId() { Table table = Table.builder().databaseName("db").name("name").build(); TableIdentifier icebergId = TableIdentifier.of("db", "name"); - Assert.assertEquals(icebergId, GlueToIcebergConverter.toTableId(table)); + Assertions.assertThat(GlueToIcebergConverter.toTableId(table)).isEqualTo(icebergId); } @Test 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 b29247af65f7..04e58e853743 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 @@ -33,8 +33,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; import software.amazon.awssdk.services.glue.model.Column; import software.amazon.awssdk.services.glue.model.DatabaseInput; import software.amazon.awssdk.services.glue.model.StorageDescriptor; @@ -50,7 +49,8 @@ public class TestIcebergToGlueConverter { @Test public void testToDatabaseName() { - Assert.assertEquals("db", IcebergToGlueConverter.toDatabaseName(Namespace.of("db"), false)); + Assertions.assertThat(IcebergToGlueConverter.toDatabaseName(Namespace.of("db"), false)) + .isEqualTo("db"); } @Test @@ -78,7 +78,8 @@ public void testSkipNamespaceValidation() { List acceptableNames = Lists.newArrayList(Namespace.of("db-1"), Namespace.of("db-1-1-1")); for (Namespace name : acceptableNames) { - Assert.assertEquals(name.toString(), IcebergToGlueConverter.toDatabaseName(name, true)); + Assertions.assertThat(IcebergToGlueConverter.toDatabaseName(name, true)) + .isEqualTo(name.toString()); } } @@ -90,7 +91,8 @@ public void testSkipTableNameValidation() { TableIdentifier.parse("db.a-1-1"), TableIdentifier.parse("db.a#1")); for (TableIdentifier identifier : acceptableIdentifiers) { - Assert.assertEquals(identifier.name(), IcebergToGlueConverter.getTableName(identifier, true)); + Assertions.assertThat(IcebergToGlueConverter.getTableName(identifier, true)) + .isEqualTo(identifier.name()); } } @@ -106,19 +108,25 @@ public void testToDatabaseInput() { "val"); DatabaseInput databaseInput = IcebergToGlueConverter.toDatabaseInput(Namespace.of("ns"), properties, false); - Assert.assertEquals("Location should be set", "s3://location", databaseInput.locationUri()); - Assert.assertEquals("Description should be set", "description", databaseInput.description()); - Assert.assertEquals( - "Parameters should be set", ImmutableMap.of("key", "val"), databaseInput.parameters()); - Assert.assertEquals("Database name should be set", "ns", databaseInput.name()); + Assertions.assertThat(databaseInput.locationUri()) + .as("Location should be set") + .isEqualTo("s3://location"); + Assertions.assertThat(databaseInput.description()) + .as("Description should be set") + .isEqualTo("description"); + Assertions.assertThat(databaseInput.parameters()) + .as("Parameters should be set") + .isEqualTo(ImmutableMap.of("key", "val")); + Assertions.assertThat(databaseInput.name()).as("Database name should be set").isEqualTo("ns"); } @Test public void testToDatabaseInputNoParameter() { DatabaseInput input = DatabaseInput.builder().name("db").parameters(ImmutableMap.of()).build(); Namespace namespace = Namespace.of("db"); - Assert.assertEquals( - input, IcebergToGlueConverter.toDatabaseInput(namespace, ImmutableMap.of(), false)); + Assertions.assertThat( + IcebergToGlueConverter.toDatabaseInput(namespace, ImmutableMap.of(), false)) + .isEqualTo(input); } @Test @@ -128,11 +136,14 @@ public void testToDatabaseInputEmptyLocation() { IcebergToGlueConverter.GLUE_DB_DESCRIPTION_KEY, "description", "key", "val"); DatabaseInput databaseInput = IcebergToGlueConverter.toDatabaseInput(Namespace.of("ns"), properties, false); - Assert.assertNull("Location should not be set", databaseInput.locationUri()); - Assert.assertEquals("Description should be set", "description", databaseInput.description()); - Assert.assertEquals( - "Parameters should be set", ImmutableMap.of("key", "val"), databaseInput.parameters()); - Assert.assertEquals("Database name should be set", "ns", databaseInput.name()); + Assertions.assertThat(databaseInput.locationUri()).as("Location should not be set").isNull(); + Assertions.assertThat(databaseInput.description()) + .as("Description should be set") + .isEqualTo("description"); + Assertions.assertThat(databaseInput.parameters()) + .as("Parameters should be set") + .isEqualTo(ImmutableMap.of("key", "val")); + Assertions.assertThat(databaseInput.name()).as("Database name should be set").isEqualTo("ns"); } @Test @@ -141,11 +152,14 @@ public void testToDatabaseInputEmptyDescription() { ImmutableMap.of(IcebergToGlueConverter.GLUE_DB_LOCATION_KEY, "s3://location", "key", "val"); DatabaseInput databaseInput = IcebergToGlueConverter.toDatabaseInput(Namespace.of("ns"), properties, false); - Assert.assertEquals("Location should be set", "s3://location", databaseInput.locationUri()); - Assert.assertNull("Description should not be set", databaseInput.description()); - Assert.assertEquals( - "Parameters should be set", ImmutableMap.of("key", "val"), databaseInput.parameters()); - Assert.assertEquals("Database name should be set", "ns", databaseInput.name()); + Assertions.assertThat(databaseInput.locationUri()) + .as("Location should be set") + .isEqualTo("s3://location"); + Assertions.assertThat(databaseInput.description()).as("Description should not be set").isNull(); + Assertions.assertThat(databaseInput.parameters()) + .as("Parameters should be set") + .isEqualTo(ImmutableMap.of("key", "val")); + Assertions.assertThat(databaseInput.name()).as("Database name should be set").isEqualTo("ns"); } @Test @@ -199,18 +213,15 @@ public void testSetTableInputInformation() { .build()) .build(); - Assert.assertEquals( - "additionalLocations should match", - expectedTableInput.storageDescriptor().additionalLocations(), - actualTableInput.storageDescriptor().additionalLocations()); - Assert.assertEquals( - "Location should match", - expectedTableInput.storageDescriptor().location(), - actualTableInput.storageDescriptor().location()); - Assert.assertEquals( - "Columns should match", - expectedTableInput.storageDescriptor().columns(), - actualTableInput.storageDescriptor().columns()); + Assertions.assertThat(actualTableInput.storageDescriptor().additionalLocations()) + .as("additionalLocations should match") + .isEqualTo(expectedTableInput.storageDescriptor().additionalLocations()); + Assertions.assertThat(actualTableInput.storageDescriptor().location()) + .as("Location should match") + .isEqualTo(expectedTableInput.storageDescriptor().location()); + Assertions.assertThat(actualTableInput.storageDescriptor().columns()) + .as("Columns should match") + .isEqualTo(expectedTableInput.storageDescriptor().columns()); } @Test @@ -268,17 +279,14 @@ public void testSetTableInputInformationWithRemovedColumns() { .build()) .build(); - Assert.assertEquals( - "additionalLocations should match", - expectedTableInput.storageDescriptor().additionalLocations(), - actualTableInput.storageDescriptor().additionalLocations()); - Assert.assertEquals( - "Location should match", - expectedTableInput.storageDescriptor().location(), - actualTableInput.storageDescriptor().location()); - Assert.assertEquals( - "Columns should match", - expectedTableInput.storageDescriptor().columns(), - actualTableInput.storageDescriptor().columns()); + Assertions.assertThat(actualTableInput.storageDescriptor().additionalLocations()) + .as("additionalLocations should match") + .isEqualTo(expectedTableInput.storageDescriptor().additionalLocations()); + Assertions.assertThat(actualTableInput.storageDescriptor().location()) + .as("Location should match") + .isEqualTo(expectedTableInput.storageDescriptor().location()); + Assertions.assertThat(actualTableInput.storageDescriptor().columns()) + .as("Columns should match") + .isEqualTo(expectedTableInput.storageDescriptor().columns()); } } 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 2bcd83d244f7..761cd514316c 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 @@ -18,10 +18,6 @@ */ package org.apache.iceberg.aws.s3; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -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,16 +25,15 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; -import com.adobe.testing.s3mock.junit4.S3MockRule; +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.util.List; import java.util.Map; import java.util.Random; import java.util.UUID; -import org.apache.commons.io.IOUtils; -import org.apache.commons.lang3.SerializationUtils; import org.apache.hadoop.conf.Configurable; import org.apache.iceberg.BaseTable; import org.apache.iceberg.CatalogProperties; @@ -50,6 +45,7 @@ import org.apache.iceberg.io.BulkDeletionFailureException; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.FileIOParser; +import org.apache.iceberg.io.IOUtil; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.jdbc.JdbcCatalog; @@ -61,14 +57,12 @@ import org.apache.iceberg.types.Types; import org.apache.iceberg.util.SerializableSupplier; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Ignore; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.junit.MockitoJUnitRunner; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; import software.amazon.awssdk.core.sync.RequestBody; import software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient; import software.amazon.awssdk.regions.Region; @@ -79,10 +73,12 @@ import software.amazon.awssdk.services.s3.model.DeleteObjectsResponse; import software.amazon.awssdk.services.s3.model.S3Error; -@RunWith(MockitoJUnitRunner.class) +@ExtendWith(S3MockExtension.class) public class TestS3FileIO { - @ClassRule public static final S3MockRule S3_MOCK_RULE = S3MockRule.builder().silent().build(); - public SerializableSupplier s3 = S3_MOCK_RULE::createS3ClientV2; + @RegisterExtension + public static final S3MockExtension S3_MOCK = S3MockExtension.builder().silent().build(); + + public SerializableSupplier s3 = S3_MOCK::createS3ClientV2; private final S3Client s3mock = mock(S3Client.class, delegatesTo(s3.get())); private final Random random = new Random(1); private final int numBucketsForBatchDeletion = 3; @@ -97,7 +93,7 @@ public class TestS3FileIO { "s3.delete.batch-size", Integer.toString(batchDeletionSize)); - @Before + @BeforeEach public void before() { s3FileIO = new S3FileIO(() -> s3mock); s3FileIO.initialize(properties); @@ -108,7 +104,7 @@ public void before() { StaticClientFactory.client = s3mock; } - @After + @AfterEach public void after() { if (null != s3FileIO) { s3FileIO.close(); @@ -122,25 +118,25 @@ public void testNewInputFile() throws IOException { random.nextBytes(expected); InputFile in = s3FileIO.newInputFile(location); - assertFalse(in.exists()); + Assertions.assertThat(in.exists()).isFalse(); OutputFile out = s3FileIO.newOutputFile(location); try (OutputStream os = out.createOrOverwrite()) { - IOUtils.write(expected, os); + IOUtil.writeFully(os, ByteBuffer.wrap(expected)); } - assertTrue(in.exists()); - byte[] actual; + Assertions.assertThat(in.exists()).isTrue(); + byte[] actual = new byte[1024 * 1024]; try (InputStream is = in.newStream()) { - actual = IOUtils.readFully(is, expected.length); + IOUtil.readFully(is, actual, 0, expected.length); } - assertArrayEquals(expected, actual); + Assertions.assertThat(actual).isEqualTo(expected); s3FileIO.deleteFile(in); - assertFalse(s3FileIO.newInputFile(location).exists()); + Assertions.assertThat(s3FileIO.newInputFile(location).exists()).isFalse(); } @Test @@ -162,17 +158,17 @@ public void testDeleteFilesSingleBatchWithRemainder() { public void testDeleteEmptyList() throws IOException { String location = "s3://bucket/path/to/file.txt"; InputFile in = s3FileIO.newInputFile(location); - assertFalse(in.exists()); + Assertions.assertThat(in.exists()).isFalse(); OutputFile out = s3FileIO.newOutputFile(location); try (OutputStream os = out.createOrOverwrite()) { - IOUtils.write(new byte[1024 * 1024], os); + IOUtil.writeFully(os, ByteBuffer.wrap(new byte[1024 * 1024])); } s3FileIO.deleteFiles(Lists.newArrayList()); - Assert.assertTrue(s3FileIO.newInputFile(location).exists()); + Assertions.assertThat(s3FileIO.newInputFile(location).exists()).isTrue(); s3FileIO.deleteFile(in); - assertFalse(s3FileIO.newInputFile(location).exists()); + Assertions.assertThat(s3FileIO.newInputFile(location).exists()).isFalse(); } @Test @@ -207,12 +203,12 @@ private void testBatchDelete(int numObjects) { int expectedDeleteRequests = expectedNumberOfBatchesPerBucket * numBucketsForBatchDeletion; verify(s3mock, times(expectedDeleteRequests)).deleteObjects((DeleteObjectsRequest) any()); for (String path : paths) { - Assert.assertFalse(s3FileIO.newInputFile(path).exists()); + Assertions.assertThat(s3FileIO.newInputFile(path).exists()).isFalse(); } } @Test - public void testSerializeClient() { + public void testSerializeClient() throws IOException, ClassNotFoundException { SerializableSupplier pre = () -> S3Client.builder() @@ -220,10 +216,10 @@ public void testSerializeClient() { .region(Region.US_EAST_1) .build(); - byte[] data = SerializationUtils.serialize(pre); - SerializableSupplier post = SerializationUtils.deserialize(data); + byte[] data = TestHelpers.serialize(pre); + SerializableSupplier post = TestHelpers.deserialize(data); - assertEquals("s3", post.get().serviceName()); + Assertions.assertThat(post.get().serviceName()).isEqualTo("s3"); } @Test @@ -239,11 +235,13 @@ public void testPrefixList() { String scalePrefix = String.format("%s/%s/", prefix, scale); createRandomObjects(scalePrefix, scale); - assertEquals((long) scale, Streams.stream(s3FileIO.listPrefix(scalePrefix)).count()); + Assertions.assertThat(Streams.stream(s3FileIO.listPrefix(scalePrefix)).count()) + .isEqualTo((long) scale); }); long totalFiles = scaleSizes.stream().mapToLong(Integer::longValue).sum(); - assertEquals(totalFiles, Streams.stream(s3FileIO.listPrefix(prefix)).count()); + Assertions.assertThat(Streams.stream(s3FileIO.listPrefix(prefix)).count()) + .isEqualTo(totalFiles); } /** @@ -251,7 +249,7 @@ public void testPrefixList() { * exists through integration tests. */ @Test - @Ignore + @Disabled public void testPrefixDelete() { String prefix = "s3://bucket/path/to/delete"; List scaleSizes = Lists.newArrayList(0, 5, 1001); @@ -262,7 +260,8 @@ public void testPrefixDelete() { createRandomObjects(scalePrefix, scale); s3FileIO.deletePrefix(scalePrefix); - assertEquals(0L, Streams.stream(s3FileIO.listPrefix(scalePrefix)).count()); + Assertions.assertThat(Streams.stream(s3FileIO.listPrefix(scalePrefix)).count()) + .isEqualTo(0); }); } @@ -306,7 +305,7 @@ public void testMissingTableMetadata() { .hasMessageStartingWith("Location does not exist"); long duration = System.currentTimeMillis() - start; - Assert.assertTrue("Should take less than 10 seconds", duration < 10_000); + Assertions.assertThat(duration < 10_000).as("Should take less than 10 seconds").isTrue(); } } @@ -321,8 +320,8 @@ public void testFileIOJsonSerialization() { String json = FileIOParser.toJson(s3FileIO); try (FileIO deserialized = FileIOParser.fromJson(json, conf)) { - Assert.assertTrue(deserialized instanceof S3FileIO); - Assert.assertEquals(s3FileIO.properties(), deserialized.properties()); + Assertions.assertThat(deserialized).isInstanceOf(S3FileIO.class); + Assertions.assertThat(deserialized.properties()).isEqualTo(s3FileIO.properties()); } } @@ -334,7 +333,8 @@ public void testS3FileIOKryoSerialization() throws IOException { testS3FileIO.initialize(ImmutableMap.of("k1", "v1")); FileIO roundTripSerializedFileIO = TestHelpers.KryoHelpers.roundTripSerialize(testS3FileIO); - Assert.assertEquals(testS3FileIO.properties(), roundTripSerializedFileIO.properties()); + Assertions.assertThat(roundTripSerializedFileIO.properties()) + .isEqualTo(testS3FileIO.properties()); } @Test @@ -345,7 +345,8 @@ public void testS3FileIOWithEmptyPropsKryoSerialization() throws IOException { testS3FileIO.initialize(ImmutableMap.of()); FileIO roundTripSerializedFileIO = TestHelpers.KryoHelpers.roundTripSerialize(testS3FileIO); - Assert.assertEquals(testS3FileIO.properties(), roundTripSerializedFileIO.properties()); + Assertions.assertThat(roundTripSerializedFileIO.properties()) + .isEqualTo(testS3FileIO.properties()); } @Test @@ -356,7 +357,8 @@ public void testS3FileIOJavaSerialization() throws IOException, ClassNotFoundExc testS3FileIO.initialize(ImmutableMap.of("k1", "v1")); FileIO roundTripSerializedFileIO = TestHelpers.roundTripSerialize(testS3FileIO); - Assert.assertEquals(testS3FileIO.properties(), roundTripSerializedFileIO.properties()); + Assertions.assertThat(roundTripSerializedFileIO.properties()) + .isEqualTo(testS3FileIO.properties()); } private void createRandomObjects(String prefix, int count) { 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 406b62c5f1dc..feaac4eadad5 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 @@ -18,33 +18,33 @@ */ package org.apache.iceberg.aws.s3; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertThrows; - -import com.adobe.testing.s3mock.junit4.S3MockRule; +import com.adobe.testing.s3mock.junit5.S3MockExtension; import java.io.IOException; import java.util.Arrays; import java.util.Random; -import org.apache.commons.io.IOUtils; +import org.apache.iceberg.io.IOUtil; import org.apache.iceberg.io.RangeReadable; import org.apache.iceberg.io.SeekableInputStream; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Test; +import org.assertj.core.api.Assertions; +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 software.amazon.awssdk.core.sync.RequestBody; import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.s3.model.BucketAlreadyExistsException; import software.amazon.awssdk.services.s3.model.CreateBucketRequest; import software.amazon.awssdk.services.s3.model.PutObjectRequest; +@ExtendWith(S3MockExtension.class) public class TestS3InputStream { - @ClassRule public static final S3MockRule S3_MOCK_RULE = S3MockRule.builder().silent().build(); + @RegisterExtension + public static final S3MockExtension S3_MOCK = S3MockExtension.builder().silent().build(); - private final S3Client s3 = S3_MOCK_RULE.createS3ClientV2(); + private final S3Client s3 = S3_MOCK.createS3ClientV2(); private final Random random = new Random(1); - @Before + @BeforeEach public void before() { createBucket("bucket"); } @@ -88,13 +88,13 @@ private void readAndCheck( SeekableInputStream in, long rangeStart, int size, byte[] original, boolean buffered) throws IOException { in.seek(rangeStart); - assertEquals(rangeStart, in.getPos()); + Assertions.assertThat(in.getPos()).isEqualTo(rangeStart); long rangeEnd = rangeStart + size; byte[] actual = new byte[size]; if (buffered) { - IOUtils.readFully(in, actual); + IOUtil.readFully(in, actual, 0, size); } else { int read = 0; while (read < size) { @@ -102,8 +102,9 @@ private void readAndCheck( } } - assertEquals(rangeEnd, in.getPos()); - assertArrayEquals(Arrays.copyOfRange(original, (int) rangeStart, (int) rangeEnd), actual); + Assertions.assertThat(in.getPos()).isEqualTo(rangeEnd); + Assertions.assertThat(actual) + .isEqualTo(Arrays.copyOfRange(original, (int) rangeStart, (int) rangeEnd)); } @Test @@ -144,9 +145,8 @@ private void readAndCheckRanges( throws IOException { in.readFully(position, buffer, offset, length); - assertArrayEquals( - Arrays.copyOfRange(original, offset, offset + length), - Arrays.copyOfRange(buffer, offset, offset + length)); + Assertions.assertThat(Arrays.copyOfRange(buffer, offset, offset + length)) + .isEqualTo(Arrays.copyOfRange(original, offset, offset + length)); } @Test @@ -154,7 +154,9 @@ public void testClose() throws Exception { S3URI uri = new S3URI("s3://bucket/path/to/closed.dat"); SeekableInputStream closed = new S3InputStream(s3, uri); closed.close(); - assertThrows(IllegalStateException.class, () -> closed.seek(0)); + Assertions.assertThatThrownBy(() -> closed.seek(0)) + .isInstanceOf(IllegalStateException.class) + .hasMessage("already closed"); } @Test @@ -166,8 +168,10 @@ public void testSeek() throws Exception { try (SeekableInputStream in = new S3InputStream(s3, uri)) { in.seek(expected.length / 2); - byte[] actual = IOUtils.readFully(in, expected.length / 2); - assertArrayEquals(Arrays.copyOfRange(expected, expected.length / 2, expected.length), actual); + byte[] actual = new byte[expected.length / 2]; + IOUtil.readFully(in, actual, 0, expected.length / 2); + Assertions.assertThat(actual) + .isEqualTo(Arrays.copyOfRange(expected, expected.length / 2, expected.length)); } } 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 b927760f6612..6f8d1d6cb988 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 @@ -19,9 +19,6 @@ package org.apache.iceberg.aws.s3; import static org.apache.iceberg.metrics.MetricsContext.nullMetrics; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; import static org.mockito.AdditionalAnswers.delegatesTo; import static org.mockito.Mockito.any; import static org.mockito.Mockito.doThrow; @@ -30,7 +27,7 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; -import com.adobe.testing.s3mock.junit4.S3MockRule; +import com.adobe.testing.s3mock.junit5.S3MockExtension; import java.io.File; import java.io.IOException; import java.io.UncheckedIOException; @@ -47,14 +44,13 @@ import java.util.stream.Stream; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.runner.RunWith; +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.mockito.junit.MockitoJUnitRunner; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.core.ResponseBytes; @@ -72,15 +68,16 @@ import software.amazon.awssdk.services.s3.model.UploadPartRequest; import software.amazon.awssdk.utils.BinaryUtils; -@RunWith(MockitoJUnitRunner.class) +@ExtendWith(S3MockExtension.class) 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; - @ClassRule public static final S3MockRule S3_MOCK_RULE = S3MockRule.builder().silent().build(); + @RegisterExtension + public static final S3MockExtension S3_MOCK = S3MockExtension.builder().silent().build(); - private final S3Client s3 = S3_MOCK_RULE.createS3ClientV2(); + private final S3Client s3 = S3_MOCK.createS3ClientV2(); private final S3Client s3mock = mock(S3Client.class, delegatesTo(s3)); private final Random random = new Random(1); private final Path tmpDir = Files.createTempDirectory("s3fileio-test-"); @@ -102,13 +99,13 @@ public class TestS3OutputStream { public TestS3OutputStream() throws IOException {} - @Before + @BeforeEach public void before() { properties.setChecksumEnabled(false); createBucket(BUCKET); } - @After + @AfterEach public void after() { File newStagingDirectory = new File(newTmpDirectory); if (newStagingDirectory.exists()) { @@ -255,7 +252,8 @@ private void checkUploadPartRequestContent( for (int i = 0; i < uploadPartRequests.size(); ++i) { int offset = i * FIVE_MBS; int len = (i + 1) * FIVE_MBS - 1 > data.length ? data.length - offset : FIVE_MBS; - assertEquals(getDigest(data, offset, len), uploadPartRequests.get(i).contentMD5()); + Assertions.assertThat(uploadPartRequests.get(i).contentMD5()) + .isEqualTo(getDigest(data, offset, len)); } } } @@ -264,7 +262,8 @@ private void checkPutObjectRequestContent( byte[] data, ArgumentCaptor putObjectRequestArgumentCaptor) { if (properties.isChecksumEnabled()) { List putObjectRequests = putObjectRequestArgumentCaptor.getAllValues(); - assertEquals(getDigest(data, 0, data.length), putObjectRequests.get(0).contentMD5()); + Assertions.assertThat(putObjectRequests.get(0).contentMD5()) + .isEqualTo(getDigest(data, 0, data.length)); } } @@ -272,7 +271,7 @@ private void checkTags(ArgumentCaptor putObjectRequestArgument if (properties.isChecksumEnabled()) { List putObjectRequests = putObjectRequestArgumentCaptor.getAllValues(); String tagging = putObjectRequests.get(0).tagging(); - assertEquals(getTags(properties.writeTags()), tagging); + Assertions.assertThat(getTags(properties.writeTags())).isEqualTo(tagging); } } @@ -286,7 +285,7 @@ private String getDigest(byte[] data, int offset, int length) { md5.update(data, offset, length); return BinaryUtils.toBase64(md5.digest()); } catch (NoSuchAlgorithmException e) { - fail(String.format("Failed to get MD5 MessageDigest. %s", e)); + Assertions.fail("Failed to get MD5 MessageDigest. %s", e); } return null; } @@ -295,11 +294,11 @@ private void writeAndVerify(S3Client client, S3URI uri, byte[] data, boolean arr try (S3OutputStream stream = new S3OutputStream(client, uri, properties, nullMetrics())) { if (arrayWrite) { stream.write(data); - assertEquals(data.length, stream.getPos()); + Assertions.assertThat(stream.getPos()).isEqualTo(data.length); } else { for (int i = 0; i < data.length; i++) { stream.write(data[i]); - assertEquals(i + 1, stream.getPos()); + Assertions.assertThat(stream.getPos()).isEqualTo(i + 1); } } } catch (IOException e) { @@ -307,11 +306,11 @@ private void writeAndVerify(S3Client client, S3URI uri, byte[] data, boolean arr } byte[] actual = readS3Data(uri); - assertArrayEquals(data, actual); + Assertions.assertThat(actual).isEqualTo(data); // Verify all staging files are cleaned up try { - assertEquals(0, Files.list(tmpDir).count()); + Assertions.assertThat(Files.list(tmpDir)).isEmpty(); } catch (IOException e) { throw new UncheckedIOException(e); } diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3RequestUtil.java b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3RequestUtil.java index 816ca4799abf..17e3e238e0fd 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3RequestUtil.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3RequestUtil.java @@ -18,8 +18,8 @@ */ package org.apache.iceberg.aws.s3; -import org.junit.Assert; -import org.junit.Test; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; import software.amazon.awssdk.services.s3.model.S3Request; import software.amazon.awssdk.services.s3.model.ServerSideEncryption; @@ -44,11 +44,11 @@ public void testConfigureServerSideCustomEncryption() { this::setCustomAlgorithm, this::setCustomKey, this::setCustomMd5); - Assert.assertNull(serverSideEncryption); - Assert.assertNull(kmsKeyId); - Assert.assertEquals(ServerSideEncryption.AES256.name(), customAlgorithm); - Assert.assertEquals("key", customKey); - Assert.assertEquals("md5", customMd5); + Assertions.assertThat(serverSideEncryption).isNull(); + Assertions.assertThat(kmsKeyId).isNull(); + Assertions.assertThat(customAlgorithm).isEqualTo(ServerSideEncryption.AES256.name()); + Assertions.assertThat(customKey).isEqualTo("key"); + Assertions.assertThat(customMd5).isEqualTo("md5"); } @Test @@ -62,11 +62,11 @@ public void testConfigureServerSideS3Encryption() { this::setCustomAlgorithm, this::setCustomKey, this::setCustomMd5); - Assert.assertEquals(ServerSideEncryption.AES256, serverSideEncryption); - Assert.assertNull(kmsKeyId); - Assert.assertNull(customAlgorithm); - Assert.assertNull(customKey); - Assert.assertNull(customMd5); + Assertions.assertThat(serverSideEncryption).isEqualTo(ServerSideEncryption.AES256); + Assertions.assertThat(kmsKeyId).isNull(); + Assertions.assertThat(customAlgorithm).isNull(); + Assertions.assertThat(customKey).isNull(); + Assertions.assertThat(customMd5).isNull(); } @Test @@ -81,11 +81,11 @@ public void testConfigureServerSideKmsEncryption() { this::setCustomAlgorithm, this::setCustomKey, this::setCustomMd5); - Assert.assertEquals(ServerSideEncryption.AWS_KMS, serverSideEncryption); - Assert.assertEquals("key", kmsKeyId); - Assert.assertNull(customAlgorithm); - Assert.assertNull(customKey); - Assert.assertNull(customMd5); + Assertions.assertThat(serverSideEncryption).isEqualTo(ServerSideEncryption.AWS_KMS); + Assertions.assertThat(kmsKeyId).isEqualTo("key"); + Assertions.assertThat(customAlgorithm).isNull(); + Assertions.assertThat(customKey).isNull(); + Assertions.assertThat(customMd5).isNull(); } @Test @@ -100,11 +100,11 @@ public void testConfigureEncryptionSkipNullSetters() { this::setCustomAlgorithm, this::setCustomKey, this::setCustomMd5); - Assert.assertNull(serverSideEncryption); - Assert.assertNull(kmsKeyId); - Assert.assertNull(customAlgorithm); - Assert.assertNull(customKey); - Assert.assertNull(customMd5); + Assertions.assertThat(serverSideEncryption).isNull(); + Assertions.assertThat(kmsKeyId).isNull(); + Assertions.assertThat(customAlgorithm).isNull(); + Assertions.assertThat(customKey).isNull(); + Assertions.assertThat(customMd5).isNull(); } public S3Request.Builder setCustomAlgorithm(String algorithm) { 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 e1916dbd06e6..803cbe098416 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 @@ -18,14 +18,12 @@ */ package org.apache.iceberg.aws.s3; -import static org.junit.Assert.assertEquals; - import java.util.Map; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestS3URI { @@ -34,9 +32,9 @@ public void testLocationParsing() { String p1 = "s3://bucket/path/to/file"; S3URI uri1 = new S3URI(p1); - assertEquals("bucket", uri1.bucket()); - assertEquals("path/to/file", uri1.key()); - assertEquals(p1, uri1.toString()); + Assertions.assertThat(uri1.bucket()).isEqualTo("bucket"); + Assertions.assertThat(uri1.key()).isEqualTo("path/to/file"); + Assertions.assertThat(uri1.toString()).isEqualTo(p1); } @Test @@ -44,9 +42,9 @@ public void testEncodedString() { String p1 = "s3://bucket/path%20to%20file"; S3URI uri1 = new S3URI(p1); - assertEquals("bucket", uri1.bucket()); - assertEquals("path%20to%20file", uri1.key()); - assertEquals(p1, uri1.toString()); + Assertions.assertThat(uri1.bucket()).isEqualTo("bucket"); + Assertions.assertThat(uri1.key()).isEqualTo("path%20to%20file"); + Assertions.assertThat(uri1.toString()).isEqualTo(p1); } @Test @@ -62,9 +60,9 @@ public void testOnlyBucketNameLocation() { String p1 = "s3://bucket"; S3URI url1 = new S3URI(p1); - assertEquals("bucket", url1.bucket()); - assertEquals("", url1.key()); - assertEquals(p1, url1.toString()); + Assertions.assertThat(url1.bucket()).isEqualTo("bucket"); + Assertions.assertThat(url1.key()).isEqualTo(""); + Assertions.assertThat(url1.toString()).isEqualTo(p1); } @Test @@ -72,17 +70,17 @@ public void testQueryAndFragment() { String p1 = "s3://bucket/path/to/file?query=foo#bar"; S3URI uri1 = new S3URI(p1); - assertEquals("bucket", uri1.bucket()); - assertEquals("path/to/file", uri1.key()); - assertEquals(p1, uri1.toString()); + Assertions.assertThat(uri1.bucket()).isEqualTo("bucket"); + Assertions.assertThat(uri1.key()).isEqualTo("path/to/file"); + Assertions.assertThat(uri1.toString()).isEqualTo(p1); } @Test public void testValidSchemes() { for (String scheme : Lists.newArrayList("https", "s3", "s3a", "s3n", "gs")) { S3URI uri = new S3URI(scheme + "://bucket/path/to/file"); - assertEquals("bucket", uri.bucket()); - assertEquals("path/to/file", uri.key()); + Assertions.assertThat(uri.bucket()).isEqualTo("bucket"); + Assertions.assertThat(uri.key()).isEqualTo("path/to/file"); } } @@ -92,8 +90,8 @@ public void testS3URIWithBucketToAccessPointMapping() { Map bucketToAccessPointMapping = ImmutableMap.of("bucket", "access-point"); S3URI uri1 = new S3URI(p1, bucketToAccessPointMapping); - assertEquals("access-point", uri1.bucket()); - assertEquals("path/to/file", uri1.key()); - assertEquals(p1, uri1.toString()); + Assertions.assertThat(uri1.bucket()).isEqualTo("access-point"); + Assertions.assertThat(uri1.key()).isEqualTo("path/to/file"); + Assertions.assertThat(uri1.toString()).isEqualTo(p1); } } 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 304116c939ba..1e44e533188b 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 @@ -33,12 +33,11 @@ import org.eclipse.jetty.servlet.ServletContextHandler; import org.eclipse.jetty.servlet.ServletHolder; import org.jetbrains.annotations.NotNull; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider; @@ -69,18 +68,14 @@ public class TestS3RestSigner { static final AwsCredentialsProvider CREDENTIALS_PROVIDER = StaticCredentialsProvider.create( AwsBasicCredentials.create("accessKeyId", "secretAccessKey")); + private static final MinioContainer MINIO_CONTAINER = + new MinioContainer(CREDENTIALS_PROVIDER.resolveCredentials()); private static Server httpServer; private static ValidatingSigner validatingSigner; private S3Client s3; - @Rule public TemporaryFolder temp = new TemporaryFolder(); - - @Rule - public MinioContainer minioContainer = - new MinioContainer(CREDENTIALS_PROVIDER.resolveCredentials()); - - @BeforeClass + @BeforeAll public static void beforeClass() throws Exception { if (null == httpServer) { httpServer = initHttpServer(); @@ -99,7 +94,7 @@ public static void beforeClass() throws Exception { new CustomAwsS3V4Signer()); } - @AfterClass + @AfterAll public static void afterClass() throws Exception { assertThat(validatingSigner.icebergSigner.tokenRefreshExecutor()) .isInstanceOf(ScheduledThreadPoolExecutor.class); @@ -123,8 +118,9 @@ public static void afterClass() throws Exception { } } - @Before + @BeforeEach public void before() throws Exception { + MINIO_CONTAINER.start(); s3 = S3Client.builder() .region(REGION) @@ -133,7 +129,7 @@ public void before() throws Exception { s3ClientBuilder -> s3ClientBuilder.httpClientBuilder( software.amazon.awssdk.http.apache.ApacheHttpClient.builder())) - .endpointOverride(minioContainer.getURI()) + .endpointOverride(MINIO_CONTAINER.getURI()) .forcePathStyle(true) // OSX won't resolve subdomains .overrideConfiguration( c -> c.putAdvancedOption(SdkAdvancedClientOption.SIGNER, validatingSigner)) @@ -199,6 +195,11 @@ public void validatedCreateMultiPartUpload() { CreateMultipartUploadRequest.builder().bucket(BUCKET).key("some/multipart-key").build()); } + @AfterEach + public void after() { + MINIO_CONTAINER.stop(); + } + @Test public void validatedUploadPart() { String multipartUploadId = diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignRequestParser.java b/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignRequestParser.java index 7348d8f20eff..753d2078c418 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignRequestParser.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignRequestParser.java @@ -23,7 +23,7 @@ import java.util.Arrays; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestS3SignRequestParser { diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignResponseParser.java b/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignResponseParser.java index 7b43135395ee..d7337b1b1777 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignResponseParser.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignResponseParser.java @@ -23,7 +23,7 @@ import java.util.Arrays; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestS3SignResponseParser { diff --git a/aws/src/test/java/org/apache/iceberg/aws/util/TestRetryDetector.java b/aws/src/test/java/org/apache/iceberg/aws/util/TestRetryDetector.java index 7403d2557c95..72cc2f91ecdc 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/util/TestRetryDetector.java +++ b/aws/src/test/java/org/apache/iceberg/aws/util/TestRetryDetector.java @@ -18,8 +18,8 @@ */ package org.apache.iceberg.aws.util; -import org.junit.Assert; -import org.junit.Test; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; import software.amazon.awssdk.core.metrics.CoreMetric; import software.amazon.awssdk.metrics.MetricCollection; @@ -31,7 +31,7 @@ public class TestRetryDetector { @Test public void testNoMetrics() { RetryDetector detector = new RetryDetector(); - Assert.assertFalse("Should default to false", detector.retried()); + Assertions.assertThat(detector.retried()).as("Should default to false").isFalse(); } @Test @@ -40,8 +40,9 @@ public void testRetryCountMissing() { RetryDetector detector = new RetryDetector(); detector.publish(metrics.collect()); - Assert.assertFalse( - "Should not detect retries if RETRY_COUNT metric is not reported", detector.retried()); + Assertions.assertThat(detector.retried()) + .as("Should not detect retries if RETRY_COUNT metric is not reported") + .isFalse(); } @Test @@ -51,7 +52,9 @@ public void testRetryCountZero() { RetryDetector detector = new RetryDetector(); detector.publish(metrics.collect()); - Assert.assertFalse("Should not detect retries if RETRY_COUNT is zero", detector.retried()); + Assertions.assertThat(detector.retried()) + .as("Should not detect retries if RETRY_COUNT is zero") + .isFalse(); } @Test @@ -61,7 +64,9 @@ public void testRetryCountNonZero() { RetryDetector detector = new RetryDetector(); detector.publish(metrics.collect()); - Assert.assertTrue("Should detect retries if RETRY_COUNT is non-zero", detector.retried()); + Assertions.assertThat(detector.retried()) + .as("Should detect retries if RETRY_COUNT is non-zero") + .isTrue(); } @Test @@ -72,8 +77,9 @@ public void testMultipleRetryCounts() { RetryDetector detector = new RetryDetector(); detector.publish(metrics.collect()); - Assert.assertTrue( - "Should detect retries if even one RETRY_COUNT is non-zero", detector.retried()); + Assertions.assertThat(detector.retried()) + .as("Should detect retries if even one RETRY_COUNT is non-zero") + .isTrue(); } @Test @@ -85,8 +91,9 @@ public void testNestedRetryCountZero() { RetryDetector detector = new RetryDetector(); detector.publish(metrics.collect()); - Assert.assertFalse( - "Should not detect retries if nested RETRY_COUNT is zero", detector.retried()); + Assertions.assertThat(detector.retried()) + .as("Should not detect retries if nested RETRY_COUNT is zero") + .isFalse(); } @Test @@ -98,8 +105,9 @@ public void testNestedRetryCountNonZero() { RetryDetector detector = new RetryDetector(); detector.publish(metrics.collect()); - Assert.assertTrue( - "Should detect retries if nested RETRY_COUNT is non-zero", detector.retried()); + Assertions.assertThat(detector.retried()) + .as("Should detect retries if nested RETRY_COUNT is non-zero") + .isTrue(); } @Test @@ -116,8 +124,9 @@ public void testNestedRetryCountMultipleChildren() { RetryDetector detector = new RetryDetector(); detector.publish(metrics.collect()); - Assert.assertTrue( - "Should detect retries if even one nested RETRY_COUNT is non-zero", detector.retried()); + Assertions.assertThat(detector.retried()) + .as("Should detect retries if even one nested RETRY_COUNT is non-zero") + .isTrue(); } @Test @@ -129,10 +138,13 @@ public void testMultipleCollectionsReported() { RetryDetector detector = new RetryDetector(); detector.publish(metrics1.collect()); - Assert.assertFalse("Should not detect retries if RETRY_COUNT is zero", detector.retried()); + Assertions.assertThat(detector.retried()) + .as("Should not detect retries if RETRY_COUNT is zero") + .isFalse(); detector.publish(metrics2.collect()); - Assert.assertTrue( - "Should continue detecting retries in additional metrics", detector.retried()); + Assertions.assertThat(detector.retried()) + .as("Should continue detecting retries in additional metrics") + .isTrue(); } @Test @@ -146,9 +158,13 @@ public void testNoOpAfterDetection() { RetryDetector detector = new RetryDetector(); detector.publish(metrics1Spy); - Assert.assertTrue("Should detect retries if RETRY_COUNT is zero", detector.retried()); + Assertions.assertThat(detector.retried()) + .as("Should detect retries if RETRY_COUNT is zero") + .isTrue(); detector.publish(metrics2Spy); - Assert.assertTrue("Should remain true once a retry is detected", detector.retried()); + Assertions.assertThat(detector.retried()) + .as("Should remain true once a retry is detected") + .isTrue(); Mockito.verify(metrics1Spy).metricValues(Mockito.eq(CoreMetric.RETRY_COUNT)); Mockito.verifyNoMoreInteractions(metrics1Spy, metrics2Spy); diff --git a/build.gradle b/build.gradle index a8951c1dfe9f..c3e4eda3f410 100644 --- a/build.gradle +++ b/build.gradle @@ -31,23 +31,16 @@ buildscript { // com.palantir.baseline:gradle-baseline-java:4.42.0 (the last version supporting Java 8) pulls // in an old version of the errorprone, which doesn't work w/ Gradle 8, so bump errorpone as // well. - classpath "net.ltgt.gradle:gradle-errorprone-plugin:3.0.1" + classpath "net.ltgt.gradle:gradle-errorprone-plugin:3.1.0" classpath 'com.diffplug.spotless:spotless-plugin-gradle:6.13.0' classpath 'gradle.plugin.org.inferred:gradle-processors:3.7.0' classpath 'me.champeau.jmh:jmh-gradle-plugin:0.7.1' classpath 'gradle.plugin.io.morethan.jmhreport:gradle-jmh-report:0.9.0' - classpath "com.github.alisiikh:gradle-scalastyle-plugin:3.4.1" + classpath "com.github.alisiikh:gradle-scalastyle-plugin:3.5.0" classpath 'com.palantir.gradle.revapi:gradle-revapi:1.7.0' classpath 'com.gorylenko.gradle-git-properties:gradle-git-properties:2.4.1' - classpath 'com.palantir.gradle.gitversion:gradle-git-version:1.0.0' - classpath ('org.eclipse.jgit:org.eclipse.jgit') { - // gradle-git-version automatically uses a jgit version that requires JDK11 - // so we need to enforce the latest jgit version that works with JDK8 - version { - strictly('5.13.1.202206130422-r') - } - } + classpath 'com.palantir.gradle.gitversion:gradle-git-version:3.0.0' } } @@ -149,7 +142,7 @@ subprojects { revapi { oldGroup = project.group oldName = project.name - oldVersion = "1.2.0" + oldVersion = "1.3.0" } tasks.register('showDeprecationRulesOnRevApiFailure') { @@ -493,7 +486,7 @@ project(':iceberg-aws') { testImplementation 'software.amazon.awssdk:iam' testImplementation 'software.amazon.awssdk:s3control' testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') - testImplementation("com.adobe.testing:s3mock-junit4") { + testImplementation("com.adobe.testing:s3mock-junit5") { exclude module: "spring-boot-starter-logging" exclude module: "logback-classic" exclude group: 'junit' @@ -532,6 +525,9 @@ project(':iceberg-aws') { project(':iceberg-delta-lake') { // use integration test since we can take advantages of spark 3.3 to read datafiles of delta lake table // and create some tests involving sql query. + test { + useJUnitPlatform() + } configurations { integrationImplementation.extendsFrom testImplementation integrationRuntime.extendsFrom testRuntimeOnly @@ -631,6 +627,10 @@ project(':iceberg-gcp') { } project(':iceberg-hive-metastore') { + test { + useJUnitPlatform() + } + dependencies { implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow') implementation project(':iceberg-core') @@ -701,6 +701,9 @@ project(':iceberg-hive-metastore') { } project(':iceberg-orc') { + test { + useJUnitPlatform() + } dependencies { implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow') api project(':iceberg-api') @@ -735,6 +738,9 @@ project(':iceberg-orc') { } project(':iceberg-parquet') { + test { + useJUnitPlatform() + } dependencies { implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow') api project(':iceberg-api') @@ -758,6 +764,9 @@ project(':iceberg-parquet') { } project(':iceberg-arrow') { + test { + useJUnitPlatform() + } dependencies { implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow') api project(':iceberg-api') @@ -828,8 +837,23 @@ project(':iceberg-pig') { } project(':iceberg-nessie') { - test { - useJUnitPlatform() + if (JavaVersion.current().isJava11Compatible()) { + test { + useJUnitPlatform() + } + compileTestJava { + sourceCompatibility = "11" + targetCompatibility = "11" + } + } else { + // Do not test Nessie against Java 8, because in-JVM testing requires Nessie server components, + // which require Java 11+. + test { + enabled = false + } + compileTestJava { + enabled = false + } } dependencies { @@ -841,25 +865,32 @@ project(':iceberg-nessie') { implementation "com.fasterxml.jackson.core:jackson-databind" implementation "com.fasterxml.jackson.core:jackson-core" - testImplementation "org.projectnessie.nessie:nessie-jaxrs-testextension" - testImplementation "org.projectnessie.nessie:nessie-versioned-persist-in-memory" - testImplementation "org.projectnessie.nessie:nessie-versioned-persist-in-memory-test" - // Need to "pull in" el-api explicitly :( - testImplementation "jakarta.el:jakarta.el-api" - compileOnly "org.apache.hadoop:hadoop-common" - testImplementation "org.apache.avro:avro" + // Only there to prevent "warning: unknown enum constant SchemaType.OBJECT" compile messages + compileOnly "org.eclipse.microprofile.openapi:microprofile-openapi-api:3.1.1" - testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') - testImplementation project(path: ':iceberg-core', configuration: 'testArtifacts') + if (JavaVersion.current().isJava11Compatible()) { + testImplementation "org.projectnessie.nessie:nessie-jaxrs-testextension" + testImplementation "org.projectnessie.nessie:nessie-versioned-storage-inmemory" + testImplementation "org.projectnessie.nessie:nessie-versioned-storage-testextension" + // Need to "pull in" el-api explicitly :( + testImplementation "jakarta.el:jakarta.el-api" - // Only there to prevent "warning: unknown enum constant SchemaType.OBJECT" compile messages - compileOnly "org.eclipse.microprofile.openapi:microprofile-openapi-api:3.1" - testCompileOnly "org.eclipse.microprofile.openapi:microprofile-openapi-api:3.0" + testImplementation "org.apache.avro:avro" + + testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') + testImplementation project(path: ':iceberg-core', configuration: 'testArtifacts') + + // Only there to prevent "warning: unknown enum constant SchemaType.OBJECT" compile messages + testCompileOnly "org.eclipse.microprofile.openapi:microprofile-openapi-api:3.1.1" + } } } project(':iceberg-dell') { + test { + useJUnitPlatform() + } dependencies { implementation project(':iceberg-core') implementation project(':iceberg-common') @@ -887,6 +918,7 @@ project(':iceberg-snowflake') { runtimeOnly("net.snowflake:snowflake-jdbc") + testImplementation 'org.mockito:mockito-junit-jupiter' testImplementation project(path: ':iceberg-core', configuration: 'testArtifacts') } } diff --git a/core/src/jmh/java/org/apache/iceberg/ManifestReadBenchmark.java b/core/src/jmh/java/org/apache/iceberg/ManifestReadBenchmark.java new file mode 100644 index 000000000000..6677e5d8b651 --- /dev/null +++ b/core/src/jmh/java/org/apache/iceberg/ManifestReadBenchmark.java @@ -0,0 +1,169 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import java.io.File; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.ByteBuffer; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import java.util.stream.Stream; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Timeout; + +@Fork(1) +@State(Scope.Benchmark) +@Measurement(iterations = 5) +@BenchmarkMode(Mode.SingleShotTime) +@Timeout(time = 1000, timeUnit = TimeUnit.HOURS) +public class ManifestReadBenchmark { + + private static final int NUM_FILES = 10; + private static final int NUM_ROWS = 100000; + private static final int NUM_COLS = 10; + + private String baseDir; + private String manifestListFile; + + @Setup + public void before() { + baseDir = + Paths.get(new File(System.getProperty("java.io.tmpdir")).getAbsolutePath()).toString(); + manifestListFile = String.format("%s/%s.avro", baseDir, UUID.randomUUID()); + + Random random = new Random(System.currentTimeMillis()); + ManifestListWriter listWriter = + ManifestLists.write(1, org.apache.iceberg.Files.localOutput(manifestListFile), 0, 1L, 0); + + try { + for (int i = 0; i < NUM_FILES; i++) { + OutputFile manifestFile = + org.apache.iceberg.Files.localOutput( + String.format("%s/%s.avro", baseDir, UUID.randomUUID())); + + ManifestWriter writer = + ManifestFiles.write(1, PartitionSpec.unpartitioned(), manifestFile, 1L); + try (ManifestWriter finalWriter = writer) { + for (int j = 0; j < NUM_ROWS; j++) { + DataFile dataFile = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withFormat(FileFormat.PARQUET) + .withPath(String.format("/path/to/data-%s-%s.parquet", i, j)) + .withFileSizeInBytes(j) + .withRecordCount(j) + .withMetrics(randomMetrics(random)) + .build(); + finalWriter.add(dataFile); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + + listWriter.add(writer.toManifestFile()); + } + + listWriter.close(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + @TearDown + public void after() throws IOException { + if (baseDir != null) { + try (Stream walk = Files.walk(Paths.get(baseDir))) { + walk.sorted(Comparator.reverseOrder()).map(Path::toFile).forEach(File::delete); + } + baseDir = null; + } + + manifestListFile = null; + } + + @Benchmark + @Threads(1) + public void readManifestFile() throws IOException { + List manifests = + ManifestLists.read(org.apache.iceberg.Files.localInput(manifestListFile)); + TestTables.LocalFileIO fileIO = new TestTables.LocalFileIO(); + Map specs = + ImmutableMap.of(PartitionSpec.unpartitioned().specId(), PartitionSpec.unpartitioned()); + long recordCount = 0L; + for (ManifestFile manifestFile : manifests) { + ManifestReader reader = ManifestFiles.read(manifestFile, fileIO, specs); + try (CloseableIterator it = reader.iterator()) { + while (it.hasNext()) { + recordCount += it.next().recordCount(); + } + } + } + } + + private Metrics randomMetrics(Random random) { + long rowCount = 100000L + random.nextInt(1000); + Map columnSizes = Maps.newHashMap(); + Map valueCounts = Maps.newHashMap(); + Map nullValueCounts = Maps.newHashMap(); + Map nanValueCounts = Maps.newHashMap(); + Map lowerBounds = Maps.newHashMap(); + Map upperBounds = Maps.newHashMap(); + for (int i = 0; i < NUM_COLS; i++) { + columnSizes.put(i, 1000000L + random.nextInt(100000)); + valueCounts.put(i, 100000L + random.nextInt(100)); + nullValueCounts.put(i, (long) random.nextInt(5)); + nanValueCounts.put(i, (long) random.nextInt(5)); + byte[] lower = new byte[8]; + random.nextBytes(lower); + lowerBounds.put(i, ByteBuffer.wrap(lower)); + byte[] upper = new byte[8]; + random.nextBytes(upper); + upperBounds.put(i, ByteBuffer.wrap(upper)); + } + + return new Metrics( + rowCount, + columnSizes, + valueCounts, + nullValueCounts, + nanValueCounts, + lowerBounds, + upperBounds); + } +} diff --git a/core/src/main/java/org/apache/iceberg/BaseContentScanTask.java b/core/src/main/java/org/apache/iceberg/BaseContentScanTask.java index e15b2b3f85d6..1521133c6466 100644 --- a/core/src/main/java/org/apache/iceberg/BaseContentScanTask.java +++ b/core/src/main/java/org/apache/iceberg/BaseContentScanTask.java @@ -34,6 +34,7 @@ abstract class BaseContentScanTask, F extends C private final String specString; private final ResidualEvaluator residuals; + private transient volatile Schema schema = null; private transient volatile PartitionSpec spec = null; BaseContentScanTask(F file, String schemaString, String specString, ResidualEvaluator residuals) { @@ -52,12 +53,24 @@ public F file() { return file; } + protected Schema schema() { + if (schema == null) { + synchronized (this) { + if (schema == null) { + this.schema = SchemaParser.fromJson(schemaString); + } + } + } + + return schema; + } + @Override public PartitionSpec spec() { if (spec == null) { synchronized (this) { if (spec == null) { - this.spec = PartitionSpecParser.fromJson(SchemaParser.fromJson(schemaString), specString); + this.spec = PartitionSpecParser.fromJson(schema(), specString); } } } diff --git a/core/src/main/java/org/apache/iceberg/BaseFileScanTask.java b/core/src/main/java/org/apache/iceberg/BaseFileScanTask.java index 2d7258be717a..f53b56696e3d 100644 --- a/core/src/main/java/org/apache/iceberg/BaseFileScanTask.java +++ b/core/src/main/java/org/apache/iceberg/BaseFileScanTask.java @@ -53,6 +53,11 @@ public List deletes() { return ImmutableList.copyOf(deletes); } + @Override + public Schema schema() { + return super.schema(); + } + @VisibleForTesting static final class SplitScanTask implements FileScanTask, MergeableScanTask { private final long len; @@ -75,6 +80,11 @@ public List deletes() { return fileScanTask.deletes(); } + @Override + public Schema schema() { + return fileScanTask.schema(); + } + @Override public PartitionSpec spec() { return fileScanTask.spec(); diff --git a/core/src/main/java/org/apache/iceberg/BaseScan.java b/core/src/main/java/org/apache/iceberg/BaseScan.java index 9db72227ac3f..953ad754aa1b 100644 --- a/core/src/main/java/org/apache/iceberg/BaseScan.java +++ b/core/src/main/java/org/apache/iceberg/BaseScan.java @@ -254,6 +254,6 @@ private static Schema lazyColumnProjection(TableScanContext context, Schema sche @Override public ThisT metricsReporter(MetricsReporter reporter) { - return newRefinedScan(table(), schema(), context().reportWith(reporter)); + return newRefinedScan(table, schema, context.reportWith(reporter)); } } diff --git a/core/src/main/java/org/apache/iceberg/BaseTransaction.java b/core/src/main/java/org/apache/iceberg/BaseTransaction.java index cef487931b0e..61da776f4c44 100644 --- a/core/src/main/java/org/apache/iceberg/BaseTransaction.java +++ b/core/src/main/java/org/apache/iceberg/BaseTransaction.java @@ -107,6 +107,10 @@ public String tableName() { } public TableMetadata startMetadata() { + return base; + } + + public TableMetadata currentMetadata() { return current; } diff --git a/core/src/main/java/org/apache/iceberg/ContentFileParser.java b/core/src/main/java/org/apache/iceberg/ContentFileParser.java new file mode 100644 index 000000000000..b3edf2927fbc --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/ContentFileParser.java @@ -0,0 +1,270 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.util.JsonUtil; + +class ContentFileParser { + private static final String SPEC_ID = "spec-id"; + private static final String CONTENT = "content"; + private static final String FILE_PATH = "file-path"; + private static final String FILE_FORMAT = "file-format"; + private static final String PARTITION = "partition"; + private static final String RECORD_COUNT = "record-count"; + private static final String FILE_SIZE = "file-size-in-bytes"; + private static final String COLUMN_SIZES = "column-sizes"; + private static final String VALUE_COUNTS = "value-counts"; + private static final String NULL_VALUE_COUNTS = "null-value-counts"; + private static final String NAN_VALUE_COUNTS = "nan-value-counts"; + private static final String LOWER_BOUNDS = "lower-bounds"; + private static final String UPPER_BOUNDS = "upper-bounds"; + private static final String KEY_METADATA = "key-metadata"; + 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 ContentFileParser() {} + + private static boolean hasPartitionData(StructLike partitionData) { + return partitionData != null && partitionData.size() > 0; + } + + static String toJson(ContentFile contentFile, PartitionSpec spec) { + return JsonUtil.generate( + generator -> ContentFileParser.toJson(contentFile, spec, generator), false); + } + + static void toJson(ContentFile contentFile, PartitionSpec spec, JsonGenerator generator) + throws IOException { + Preconditions.checkArgument(contentFile != null, "Invalid content file: null"); + Preconditions.checkArgument(spec != null, "Invalid partition spec: null"); + Preconditions.checkArgument(generator != null, "Invalid JSON generator: null"); + Preconditions.checkArgument( + contentFile.specId() == spec.specId(), + "Invalid partition spec id from content file: expected = %s, actual = %s", + spec.specId(), + contentFile.specId()); + Preconditions.checkArgument( + spec.isPartitioned() == hasPartitionData(contentFile.partition()), + "Invalid partition data from content file: expected = %s, actual = %s", + spec.isPartitioned() ? "partitioned" : "unpartitioned", + hasPartitionData(contentFile.partition()) ? "partitioned" : "unpartitioned"); + + generator.writeStartObject(); + + // ignore the ordinal position (ContentFile#pos) of the file in a manifest, + // as it isn't used and BaseFile constructor doesn't support it. + + generator.writeNumberField(SPEC_ID, contentFile.specId()); + generator.writeStringField(CONTENT, contentFile.content().name()); + generator.writeStringField(FILE_PATH, contentFile.path().toString()); + generator.writeStringField(FILE_FORMAT, contentFile.format().name()); + + if (contentFile.partition() != null) { + generator.writeFieldName(PARTITION); + SingleValueParser.toJson(spec.partitionType(), contentFile.partition(), generator); + } + + generator.writeNumberField(FILE_SIZE, contentFile.fileSizeInBytes()); + + metricsToJson(contentFile, generator); + + if (contentFile.keyMetadata() != null) { + generator.writeFieldName(KEY_METADATA); + SingleValueParser.toJson(DataFile.KEY_METADATA.type(), contentFile.keyMetadata(), generator); + } + + if (contentFile.splitOffsets() != null) { + JsonUtil.writeLongArray(SPLIT_OFFSETS, contentFile.splitOffsets(), generator); + } + + if (contentFile.equalityFieldIds() != null) { + JsonUtil.writeIntegerArray(EQUALITY_IDS, contentFile.equalityFieldIds(), generator); + } + + if (contentFile.sortOrderId() != null) { + generator.writeNumberField(SORT_ORDER_ID, contentFile.sortOrderId()); + } + + generator.writeEndObject(); + } + + static ContentFile fromJson(JsonNode jsonNode, PartitionSpec spec) { + Preconditions.checkArgument(jsonNode != null, "Invalid JSON node for content file: null"); + Preconditions.checkArgument( + jsonNode.isObject(), "Invalid JSON node for content file: non-object (%s)", jsonNode); + Preconditions.checkArgument(spec != null, "Invalid partition spec: null"); + + int specId = JsonUtil.getInt(SPEC_ID, jsonNode); + FileContent fileContent = FileContent.valueOf(JsonUtil.getString(CONTENT, jsonNode)); + String filePath = JsonUtil.getString(FILE_PATH, jsonNode); + FileFormat fileFormat = FileFormat.fromString(JsonUtil.getString(FILE_FORMAT, jsonNode)); + + PartitionData partitionData = null; + if (jsonNode.has(PARTITION)) { + partitionData = new PartitionData(spec.partitionType()); + StructLike structLike = + (StructLike) SingleValueParser.fromJson(spec.partitionType(), jsonNode.get(PARTITION)); + Preconditions.checkState( + partitionData.size() == structLike.size(), + "Invalid partition data size: expected = %s, actual = %s", + partitionData.size(), + structLike.size()); + for (int pos = 0; pos < partitionData.size(); ++pos) { + Class javaClass = spec.partitionType().fields().get(pos).type().typeId().javaClass(); + partitionData.set(pos, structLike.get(pos, javaClass)); + } + } + + long fileSizeInBytes = JsonUtil.getLong(FILE_SIZE, jsonNode); + Metrics metrics = metricsFromJson(jsonNode); + ByteBuffer keyMetadata = JsonUtil.getByteBufferOrNull(KEY_METADATA, jsonNode); + List splitOffsets = JsonUtil.getLongListOrNull(SPLIT_OFFSETS, jsonNode); + int[] equalityFieldIds = JsonUtil.getIntArrayOrNull(EQUALITY_IDS, jsonNode); + Integer sortOrderId = JsonUtil.getIntOrNull(SORT_ORDER_ID, jsonNode); + + if (fileContent == FileContent.DATA) { + return new GenericDataFile( + specId, + filePath, + fileFormat, + partitionData, + fileSizeInBytes, + metrics, + keyMetadata, + splitOffsets, + equalityFieldIds, + sortOrderId); + } else { + return new GenericDeleteFile( + specId, + fileContent, + filePath, + fileFormat, + partitionData, + fileSizeInBytes, + metrics, + equalityFieldIds, + sortOrderId, + splitOffsets, + keyMetadata); + } + } + + private static void metricsToJson(ContentFile contentFile, JsonGenerator generator) + throws IOException { + generator.writeNumberField(RECORD_COUNT, contentFile.recordCount()); + + if (contentFile.columnSizes() != null) { + generator.writeFieldName(COLUMN_SIZES); + SingleValueParser.toJson(DataFile.COLUMN_SIZES.type(), contentFile.columnSizes(), generator); + } + + if (contentFile.valueCounts() != null) { + generator.writeFieldName(VALUE_COUNTS); + SingleValueParser.toJson(DataFile.VALUE_COUNTS.type(), contentFile.valueCounts(), generator); + } + + if (contentFile.nullValueCounts() != null) { + generator.writeFieldName(NULL_VALUE_COUNTS); + SingleValueParser.toJson( + DataFile.NULL_VALUE_COUNTS.type(), contentFile.nullValueCounts(), generator); + } + + if (contentFile.nullValueCounts() != null) { + generator.writeFieldName(NAN_VALUE_COUNTS); + SingleValueParser.toJson( + DataFile.NAN_VALUE_COUNTS.type(), contentFile.nanValueCounts(), generator); + } + + if (contentFile.lowerBounds() != null) { + generator.writeFieldName(LOWER_BOUNDS); + SingleValueParser.toJson(DataFile.LOWER_BOUNDS.type(), contentFile.lowerBounds(), generator); + } + + if (contentFile.upperBounds() != null) { + generator.writeFieldName(UPPER_BOUNDS); + SingleValueParser.toJson(DataFile.UPPER_BOUNDS.type(), contentFile.upperBounds(), generator); + } + } + + private static Metrics metricsFromJson(JsonNode jsonNode) { + long recordCount = JsonUtil.getLong(RECORD_COUNT, jsonNode); + + Map columnSizes = null; + if (jsonNode.has(COLUMN_SIZES)) { + columnSizes = + (Map) + SingleValueParser.fromJson(DataFile.COLUMN_SIZES.type(), jsonNode.get(COLUMN_SIZES)); + } + + Map valueCounts = null; + if (jsonNode.has(VALUE_COUNTS)) { + valueCounts = + (Map) + SingleValueParser.fromJson(DataFile.VALUE_COUNTS.type(), jsonNode.get(VALUE_COUNTS)); + } + + Map nullValueCounts = null; + if (jsonNode.has(NULL_VALUE_COUNTS)) { + nullValueCounts = + (Map) + SingleValueParser.fromJson( + DataFile.NULL_VALUE_COUNTS.type(), jsonNode.get(NULL_VALUE_COUNTS)); + } + + Map nanValueCounts = null; + if (jsonNode.has(NAN_VALUE_COUNTS)) { + nanValueCounts = + (Map) + SingleValueParser.fromJson( + DataFile.NAN_VALUE_COUNTS.type(), jsonNode.get(NAN_VALUE_COUNTS)); + } + + Map lowerBounds = null; + if (jsonNode.has(LOWER_BOUNDS)) { + lowerBounds = + (Map) + SingleValueParser.fromJson(DataFile.LOWER_BOUNDS.type(), jsonNode.get(LOWER_BOUNDS)); + } + + Map upperBounds = null; + if (jsonNode.has(UPPER_BOUNDS)) { + upperBounds = + (Map) + SingleValueParser.fromJson(DataFile.UPPER_BOUNDS.type(), jsonNode.get(UPPER_BOUNDS)); + } + + return new Metrics( + recordCount, + columnSizes, + valueCounts, + nullValueCounts, + nanValueCounts, + lowerBounds, + upperBounds); + } +} diff --git a/core/src/main/java/org/apache/iceberg/DataFiles.java b/core/src/main/java/org/apache/iceberg/DataFiles.java index ef95c0bdf632..a6ea80d3662a 100644 --- a/core/src/main/java/org/apache/iceberg/DataFiles.java +++ b/core/src/main/java/org/apache/iceberg/DataFiles.java @@ -29,6 +29,7 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.util.ArrayUtil; import org.apache.iceberg.util.ByteBuffers; public class DataFiles { @@ -89,6 +90,26 @@ static PartitionData fillFromPath(PartitionSpec spec, String partitionPath, Part return data; } + static PartitionData fillFromValues( + PartitionSpec spec, List partitionValues, PartitionData reuse) { + PartitionData data = reuse; + if (data == null) { + data = newPartitionData(spec); + } + + Preconditions.checkArgument( + partitionValues.size() == spec.fields().size(), + "Invalid partition data, expecting %s fields, found %s", + spec.fields().size(), + partitionValues.size()); + + for (int i = 0; i < partitionValues.size(); i += 1) { + data.set(i, Conversions.fromPartitionString(data.getType(i), partitionValues.get(i))); + } + + return data; + } + public static PartitionData data(PartitionSpec spec, String partitionPath) { return fillFromPath(spec, partitionPath, null); } @@ -123,7 +144,6 @@ public static class Builder { private FileFormat format = null; private long recordCount = -1L; private long fileSizeInBytes = -1L; - private Integer sortOrderId = SortOrder.unsorted().orderId(); // optional fields private Map columnSizes = null; @@ -134,6 +154,8 @@ public static class Builder { private Map upperBounds = null; private ByteBuffer keyMetadata = null; private List splitOffsets = null; + private List equalityFieldIds = null; + private Integer sortOrderId = SortOrder.unsorted().orderId(); public Builder(PartitionSpec spec) { this.spec = spec; @@ -248,6 +270,16 @@ public Builder withPartitionPath(String newPartitionPath) { return this; } + public Builder withPartitionValues(List partitionValues) { + Preconditions.checkArgument( + isPartitioned ^ partitionValues.isEmpty(), + "Table must be partitioned or partition values must be empty"); + if (!partitionValues.isEmpty()) { + this.partitionData = fillFromValues(spec, partitionValues, partitionData); + } + return this; + } + public Builder withMetrics(Metrics metrics) { // check for null to avoid NPE when unboxing this.recordCount = metrics.recordCount() == null ? -1 : metrics.recordCount(); @@ -269,6 +301,14 @@ public Builder withSplitOffsets(List offsets) { return this; } + public Builder withEqualityFieldIds(List equalityIds) { + if (equalityIds != null) { + this.equalityFieldIds = ImmutableList.copyOf(equalityIds); + } + + return this; + } + public Builder withEncryptionKeyMetadata(ByteBuffer newKeyMetadata) { this.keyMetadata = newKeyMetadata; return this; @@ -310,6 +350,7 @@ public DataFile build() { upperBounds), keyMetadata, splitOffsets, + ArrayUtil.toIntArray(equalityFieldIds), sortOrderId); } } diff --git a/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java b/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java index eedde21397eb..977b2a387d63 100644 --- a/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java +++ b/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.Comparator; import java.util.List; @@ -49,6 +50,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Multimaps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.relocated.com.google.common.collect.Streams; import org.apache.iceberg.types.Comparators; import org.apache.iceberg.types.Conversions; import org.apache.iceberg.types.Type; @@ -66,43 +68,50 @@ * file. */ class DeleteFileIndex { - private final Map specsById; + private static final DeleteFile[] NO_DELETES = new DeleteFile[0]; + private final Map partitionTypeById; private final Map> wrapperById; - private final long[] globalSeqs; - private final DeleteFile[] globalDeletes; - private final Map, Pair> - sortedDeletesByPartition; + private final DeleteFileGroup globalDeletes; + private final Map, DeleteFileGroup> deletesByPartition; + private final boolean isEmpty; + /** @deprecated since 1.4.0, will be removed in 1.5.0. */ + @Deprecated DeleteFileIndex( - Map specsById, + Map specs, long[] globalSeqs, DeleteFile[] globalDeletes, - Map, Pair> sortedDeletesByPartition) { - this.specsById = specsById; + Map, Pair> deletesByPartition) { + this(specs, index(specs, globalSeqs, globalDeletes), index(specs, deletesByPartition)); + } + + private DeleteFileIndex( + Map specs, + DeleteFileGroup globalDeletes, + Map, DeleteFileGroup> deletesByPartition) { ImmutableMap.Builder builder = ImmutableMap.builder(); - specsById.forEach((specId, spec) -> builder.put(specId, spec.partitionType())); + specs.forEach((specId, spec) -> builder.put(specId, spec.partitionType())); this.partitionTypeById = builder.build(); this.wrapperById = Maps.newConcurrentMap(); - this.globalSeqs = globalSeqs; this.globalDeletes = globalDeletes; - this.sortedDeletesByPartition = sortedDeletesByPartition; + this.deletesByPartition = deletesByPartition; + this.isEmpty = globalDeletes == null && deletesByPartition.isEmpty(); } public boolean isEmpty() { - return (globalDeletes == null || globalDeletes.length == 0) - && sortedDeletesByPartition.isEmpty(); + return isEmpty; } public Iterable referencedDeleteFiles() { Iterable deleteFiles = Collections.emptyList(); if (globalDeletes != null) { - deleteFiles = Iterables.concat(deleteFiles, Arrays.asList(globalDeletes)); + deleteFiles = Iterables.concat(deleteFiles, globalDeletes.referencedDeleteFiles()); } - for (Pair partitionDeletes : sortedDeletesByPartition.values()) { - deleteFiles = Iterables.concat(deleteFiles, Arrays.asList(partitionDeletes.second())); + for (DeleteFileGroup partitionDeletes : deletesByPartition.values()) { + deleteFiles = Iterables.concat(deleteFiles, partitionDeletes.referencedDeleteFiles()); } return deleteFiles; @@ -123,64 +132,63 @@ DeleteFile[] forEntry(ManifestEntry entry) { } DeleteFile[] forDataFile(long sequenceNumber, DataFile file) { + if (isEmpty) { + return NO_DELETES; + } + Pair partition = partition(file.specId(), file.partition()); - Pair partitionDeletes = sortedDeletesByPartition.get(partition); + DeleteFileGroup partitionDeletes = deletesByPartition.get(partition); + + if (globalDeletes == null && partitionDeletes == null) { + return NO_DELETES; + } - Stream matchingDeletes; + Stream matchingDeletes; if (partitionDeletes == null) { - matchingDeletes = limitBySequenceNumber(sequenceNumber, globalSeqs, globalDeletes); + matchingDeletes = globalDeletes.limit(sequenceNumber); } else if (globalDeletes == null) { - matchingDeletes = - limitBySequenceNumber( - sequenceNumber, partitionDeletes.first(), partitionDeletes.second()); + matchingDeletes = partitionDeletes.limit(sequenceNumber); } else { - matchingDeletes = - Stream.concat( - limitBySequenceNumber(sequenceNumber, globalSeqs, globalDeletes), - limitBySequenceNumber( - sequenceNumber, partitionDeletes.first(), partitionDeletes.second())); + Stream matchingGlobalDeletes = globalDeletes.limit(sequenceNumber); + Stream matchingPartitionDeletes = partitionDeletes.limit(sequenceNumber); + matchingDeletes = Stream.concat(matchingGlobalDeletes, matchingPartitionDeletes); } return matchingDeletes - .filter( - deleteFile -> - canContainDeletesForFile(file, deleteFile, specsById.get(file.specId()).schema())) + .filter(deleteFile -> canContainDeletesForFile(file, deleteFile)) + .map(IndexedDeleteFile::wrapped) .toArray(DeleteFile[]::new); } - private static boolean canContainDeletesForFile( - DataFile dataFile, DeleteFile deleteFile, Schema schema) { + private static boolean canContainDeletesForFile(DataFile dataFile, IndexedDeleteFile deleteFile) { switch (deleteFile.content()) { case POSITION_DELETES: return canContainPosDeletesForFile(dataFile, deleteFile); case EQUALITY_DELETES: - return canContainEqDeletesForFile(dataFile, deleteFile, schema); + return canContainEqDeletesForFile(dataFile, deleteFile, deleteFile.spec().schema()); } return true; } - private static boolean canContainPosDeletesForFile(DataFile dataFile, DeleteFile deleteFile) { + private static boolean canContainPosDeletesForFile( + DataFile dataFile, IndexedDeleteFile deleteFile) { // check that the delete file can contain the data file's file_path - Map lowers = deleteFile.lowerBounds(); - Map uppers = deleteFile.upperBounds(); - if (lowers == null || uppers == null) { + if (deleteFile.hasNoLowerOrUpperBounds()) { return true; } - Type pathType = MetadataColumns.DELETE_FILE_PATH.type(); int pathId = MetadataColumns.DELETE_FILE_PATH.fieldId(); Comparator comparator = Comparators.charSequences(); - ByteBuffer lower = lowers.get(pathId); - if (lower != null - && comparator.compare(dataFile.path(), Conversions.fromByteBuffer(pathType, lower)) < 0) { + + CharSequence lower = deleteFile.lowerBound(pathId); + if (lower != null && comparator.compare(dataFile.path(), lower) < 0) { return false; } - ByteBuffer upper = uppers.get(pathId); - if (upper != null - && comparator.compare(dataFile.path(), Conversions.fromByteBuffer(pathType, upper)) > 0) { + CharSequence upper = deleteFile.upperBound(pathId); + if (upper != null && comparator.compare(dataFile.path(), upper) > 0) { return false; } @@ -189,20 +197,15 @@ private static boolean canContainPosDeletesForFile(DataFile dataFile, DeleteFile @SuppressWarnings("checkstyle:CyclomaticComplexity") private static boolean canContainEqDeletesForFile( - DataFile dataFile, DeleteFile deleteFile, Schema schema) { + DataFile dataFile, IndexedDeleteFile deleteFile, Schema schema) { + Map dataLowers = dataFile.lowerBounds(); + Map dataUppers = dataFile.upperBounds(); + // whether to check data ranges or to assume that the ranges match // if upper/lower bounds are missing, null counts may still be used to determine delete files // can be skipped boolean checkRanges = - dataFile.lowerBounds() != null - && dataFile.upperBounds() != null - && deleteFile.lowerBounds() != null - && deleteFile.upperBounds() != null; - - Map dataLowers = dataFile.lowerBounds(); - Map dataUppers = dataFile.upperBounds(); - Map deleteLowers = deleteFile.lowerBounds(); - Map deleteUppers = deleteFile.upperBounds(); + dataLowers != null && dataUppers != null && deleteFile.hasLowerAndUpperBounds(); Map dataNullCounts = dataFile.nullValueCounts(); Map dataValueCounts = dataFile.valueCounts(); @@ -241,15 +244,14 @@ && allNonNull(dataNullCounts, field)) { ByteBuffer dataLower = dataLowers.get(id); ByteBuffer dataUpper = dataUppers.get(id); - ByteBuffer deleteLower = deleteLowers.get(id); - ByteBuffer deleteUpper = deleteUppers.get(id); + Object deleteLower = deleteFile.lowerBound(id); + Object deleteUpper = deleteFile.upperBound(id); if (dataLower == null || dataUpper == null || deleteLower == null || deleteUpper == null) { // at least one bound is not known, assume the delete file may match continue; } - if (!rangesOverlap( - field.type().asPrimitiveType(), dataLower, dataUpper, deleteLower, deleteUpper)) { + if (!rangesOverlap(field, dataLower, dataUpper, deleteLower, deleteUpper)) { // no values overlap between the data file and the deletes return false; } @@ -259,19 +261,25 @@ && allNonNull(dataNullCounts, field)) { } private static boolean rangesOverlap( - Type.PrimitiveType type, + Types.NestedField field, ByteBuffer dataLowerBuf, ByteBuffer dataUpperBuf, - ByteBuffer deleteLowerBuf, - ByteBuffer deleteUpperBuf) { + T deleteLower, + T deleteUpper) { + Type.PrimitiveType type = field.type().asPrimitiveType(); Comparator comparator = Comparators.forType(type); + T dataLower = Conversions.fromByteBuffer(type, dataLowerBuf); + if (comparator.compare(dataLower, deleteUpper) > 0) { + return false; + } + T dataUpper = Conversions.fromByteBuffer(type, dataUpperBuf); - T deleteLower = Conversions.fromByteBuffer(type, deleteLowerBuf); - T deleteUpper = Conversions.fromByteBuffer(type, deleteUpperBuf); + if (comparator.compare(deleteLower, dataUpper) > 0) { + return false; + } - return comparator.compare(deleteLower, dataUpper) <= 0 - && comparator.compare(dataLower, deleteUpper) <= 0; + return true; } private static boolean allNonNull(Map nullValueCounts, Types.NestedField field) { @@ -327,36 +335,49 @@ private static boolean containsNull(Map nullValueCounts, Types.Ne return nullValueCount > 0; } - private static Stream limitBySequenceNumber( - long sequenceNumber, long[] seqs, DeleteFile[] files) { - if (files == null) { - return Stream.empty(); + private static DeleteFileGroup index( + Map specs, Pair pairs) { + return index(specs, pairs.first(), pairs.second()); + } + + private static DeleteFileGroup index( + Map specs, long[] seqs, DeleteFile[] files) { + if (files == null || files.length == 0) { + return null; } - int pos = Arrays.binarySearch(seqs, sequenceNumber); - int start; - if (pos < 0) { - // the sequence number was not found, where it would be inserted is -(pos + 1) - start = -(pos + 1); - } else { - // the sequence number was found, but may not be the first - // find the first delete file with the given sequence number by decrementing the position - start = pos; - while (start > 0 && seqs[start - 1] >= sequenceNumber) { - start -= 1; - } + IndexedDeleteFile[] indexedGlobalDeleteFiles = new IndexedDeleteFile[files.length]; + + for (int pos = 0; pos < files.length; pos++) { + DeleteFile file = files[pos]; + PartitionSpec spec = specs.get(file.specId()); + long applySequenceNumber = seqs[pos]; + indexedGlobalDeleteFiles[pos] = new IndexedDeleteFile(spec, file, applySequenceNumber); } - return Arrays.stream(files, start, files.length); + return new DeleteFileGroup(seqs, indexedGlobalDeleteFiles); + } + + private static Map, DeleteFileGroup> index( + Map specs, + Map, Pair> deletesByPartition) { + Map, DeleteFileGroup> indexed = Maps.newHashMap(); + deletesByPartition.forEach((key, value) -> indexed.put(key, index(specs, value))); + return indexed; } static Builder builderFor(FileIO io, Iterable deleteManifests) { return new Builder(io, Sets.newHashSet(deleteManifests)); } + static Builder builderFor(Iterable deleteFiles) { + return new Builder(deleteFiles); + } + static class Builder { private final FileIO io; private final Set deleteManifests; + private final Iterable deleteFiles; private long minSequenceNumber = 0L; private Map specsById = null; private Expression dataFilter = Expressions.alwaysTrue(); @@ -369,6 +390,13 @@ static class Builder { Builder(FileIO io, Set deleteManifests) { this.io = io; this.deleteManifests = Sets.newHashSet(deleteManifests); + this.deleteFiles = null; + } + + Builder(Iterable deleteFiles) { + this.io = null; + this.deleteManifests = null; + this.deleteFiles = deleteFiles; } Builder afterSequenceNumber(long seq) { @@ -382,16 +410,22 @@ Builder specsById(Map newSpecsById) { } Builder filterData(Expression newDataFilter) { + Preconditions.checkArgument( + deleteFiles == null, "Index constructed from files does not support data filters"); this.dataFilter = Expressions.and(dataFilter, newDataFilter); return this; } Builder filterPartitions(Expression newPartitionFilter) { + Preconditions.checkArgument( + deleteFiles == null, "Index constructed from files does not support partition filters"); this.partitionFilter = Expressions.and(partitionFilter, newPartitionFilter); return this; } Builder filterPartitions(PartitionSet newPartitionSet) { + Preconditions.checkArgument( + deleteFiles == null, "Index constructed from files does not support partition filters"); this.partitionSet = newPartitionSet; return this; } @@ -411,10 +445,16 @@ Builder scanMetrics(ScanMetrics newScanMetrics) { return this; } - DeleteFileIndex build() { + private Collection filterDeleteFiles() { + return Streams.stream(deleteFiles) + .filter(file -> file.dataSequenceNumber() > minSequenceNumber) + .collect(Collectors.toList()); + } + + private Collection loadDeleteFiles() { // read all of the matching delete manifests in parallel and accumulate the matching files in // a queue - Queue> deleteEntries = new ConcurrentLinkedQueue<>(); + Queue files = new ConcurrentLinkedQueue<>(); Tasks.foreach(deleteManifestReaders()) .stopOnFailure() .throwFailureWhenFinished() @@ -425,96 +465,76 @@ DeleteFileIndex build() { for (ManifestEntry entry : reader) { if (entry.dataSequenceNumber() > minSequenceNumber) { // copy with stats for better filtering against data file stats - deleteEntries.add(entry.copy()); + files.add(entry.file().copy()); } } } catch (IOException e) { throw new RuntimeIOException(e, "Failed to close"); } }); + return files; + } + + DeleteFileIndex build() { + Collection files = deleteFiles != null ? filterDeleteFiles() : loadDeleteFiles(); // build a map from (specId, partition) to delete file entries Map wrappersBySpecId = Maps.newHashMap(); - ListMultimap, ManifestEntry> - deleteFilesByPartition = - Multimaps.newListMultimap(Maps.newHashMap(), Lists::newArrayList); - for (ManifestEntry entry : deleteEntries) { - int specId = entry.file().specId(); + ListMultimap, IndexedDeleteFile> deleteFilesByPartition = + Multimaps.newListMultimap(Maps.newHashMap(), Lists::newArrayList); + for (DeleteFile file : files) { + int specId = file.specId(); + PartitionSpec spec = specsById.get(specId); StructLikeWrapper wrapper = wrappersBySpecId - .computeIfAbsent( - specId, id -> StructLikeWrapper.forType(specsById.get(id).partitionType())) - .copyFor(entry.file().partition()); - deleteFilesByPartition.put(Pair.of(specId, wrapper), entry); + .computeIfAbsent(specId, id -> StructLikeWrapper.forType(spec.partitionType())) + .copyFor(file.partition()); + deleteFilesByPartition.put(Pair.of(specId, wrapper), new IndexedDeleteFile(spec, file)); } // sort the entries in each map value by sequence number and split into sequence numbers and // delete files lists - Map, Pair> sortedDeletesByPartition = + Map, DeleteFileGroup> sortedDeletesByPartition = Maps.newHashMap(); // also, separate out equality deletes in an unpartitioned spec that should be applied // globally - long[] globalApplySeqs = null; - DeleteFile[] globalDeletes = null; + DeleteFileGroup globalDeletes = null; for (Pair partition : deleteFilesByPartition.keySet()) { if (specsById.get(partition.first()).isUnpartitioned()) { Preconditions.checkState( globalDeletes == null, "Detected multiple partition specs with no partitions"); - List> eqFilesSortedBySeq = + IndexedDeleteFile[] eqFilesSortedBySeq = deleteFilesByPartition.get(partition).stream() - .filter(entry -> entry.file().content() == FileContent.EQUALITY_DELETES) - .map( - entry -> - // a delete file is indexed by the sequence number it should be applied to - Pair.of(entry.dataSequenceNumber() - 1, entry.file())) - .sorted(Comparator.comparingLong(Pair::first)) - .collect(Collectors.toList()); - - globalApplySeqs = eqFilesSortedBySeq.stream().mapToLong(Pair::first).toArray(); - globalDeletes = eqFilesSortedBySeq.stream().map(Pair::second).toArray(DeleteFile[]::new); - - List> posFilesSortedBySeq = + .filter(file -> file.content() == FileContent.EQUALITY_DELETES) + .sorted(Comparator.comparingLong(IndexedDeleteFile::applySequenceNumber)) + .toArray(IndexedDeleteFile[]::new); + if (eqFilesSortedBySeq.length > 0) { + globalDeletes = new DeleteFileGroup(eqFilesSortedBySeq); + } + + IndexedDeleteFile[] posFilesSortedBySeq = deleteFilesByPartition.get(partition).stream() - .filter(entry -> entry.file().content() == FileContent.POSITION_DELETES) - .map(entry -> Pair.of(entry.dataSequenceNumber(), entry.file())) - .sorted(Comparator.comparingLong(Pair::first)) - .collect(Collectors.toList()); - - long[] seqs = posFilesSortedBySeq.stream().mapToLong(Pair::first).toArray(); - DeleteFile[] files = - posFilesSortedBySeq.stream().map(Pair::second).toArray(DeleteFile[]::new); - - sortedDeletesByPartition.put(partition, Pair.of(seqs, files)); + .filter(file -> file.content() == FileContent.POSITION_DELETES) + .sorted(Comparator.comparingLong(IndexedDeleteFile::applySequenceNumber)) + .toArray(IndexedDeleteFile[]::new); + sortedDeletesByPartition.put(partition, new DeleteFileGroup(posFilesSortedBySeq)); } else { - List> filesSortedBySeq = + IndexedDeleteFile[] filesSortedBySeq = deleteFilesByPartition.get(partition).stream() - .map( - entry -> { - // a delete file is indexed by the sequence number it should be applied to - long applySeq = - entry.dataSequenceNumber() - - (entry.file().content() == FileContent.EQUALITY_DELETES ? 1 : 0); - return Pair.of(applySeq, entry.file()); - }) - .sorted(Comparator.comparingLong(Pair::first)) - .collect(Collectors.toList()); - - long[] seqs = filesSortedBySeq.stream().mapToLong(Pair::first).toArray(); - DeleteFile[] files = - filesSortedBySeq.stream().map(Pair::second).toArray(DeleteFile[]::new); - - sortedDeletesByPartition.put(partition, Pair.of(seqs, files)); + .sorted(Comparator.comparingLong(IndexedDeleteFile::applySequenceNumber)) + .toArray(IndexedDeleteFile[]::new); + sortedDeletesByPartition.put(partition, new DeleteFileGroup(filesSortedBySeq)); } } - scanMetrics.indexedDeleteFiles().increment(deleteEntries.size()); + scanMetrics.indexedDeleteFiles().increment(files.size()); deleteFilesByPartition .values() .forEach( - entry -> { - FileContent content = entry.file().content(); + file -> { + FileContent content = file.content(); if (content == FileContent.EQUALITY_DELETES) { scanMetrics.equalityDeleteFiles().increment(); } else if (content == FileContent.POSITION_DELETES) { @@ -522,8 +542,7 @@ DeleteFileIndex build() { } }); - return new DeleteFileIndex( - specsById, globalApplySeqs, globalDeletes, sortedDeletesByPartition); + return new DeleteFileIndex(specsById, globalDeletes, sortedDeletesByPartition); } private Iterable>> deleteManifestReaders() { @@ -569,4 +588,171 @@ private Iterable>> deleteManifestRea .liveEntries()); } } + + // a group of indexed delete files sorted by the sequence number they apply to + private static class DeleteFileGroup { + private final long[] seqs; + private final IndexedDeleteFile[] files; + + DeleteFileGroup(IndexedDeleteFile[] files) { + this.seqs = Arrays.stream(files).mapToLong(IndexedDeleteFile::applySequenceNumber).toArray(); + this.files = files; + } + + DeleteFileGroup(long[] seqs, IndexedDeleteFile[] files) { + this.seqs = seqs; + this.files = files; + } + + public Stream limit(long seq) { + int pos = Arrays.binarySearch(seqs, seq); + int start; + if (pos < 0) { + // the sequence number was not found, where it would be inserted is -(pos + 1) + start = -(pos + 1); + } else { + // the sequence number was found, but may not be the first + // find the first delete file with the given sequence number by decrementing the position + start = pos; + while (start > 0 && seqs[start - 1] >= seq) { + start -= 1; + } + } + + return Arrays.stream(files, start, files.length); + } + + public Iterable referencedDeleteFiles() { + return Arrays.stream(files).map(IndexedDeleteFile::wrapped).collect(Collectors.toList()); + } + } + + // a delete file wrapper that caches the converted boundaries for faster boundary checks + // this class is not meant to be exposed beyond the delete file index + private static class IndexedDeleteFile { + private final PartitionSpec spec; + private final DeleteFile wrapped; + private final long applySequenceNumber; + private volatile Map convertedLowerBounds = null; + private volatile Map convertedUpperBounds = null; + + IndexedDeleteFile(PartitionSpec spec, DeleteFile file, long applySequenceNumber) { + this.spec = spec; + this.wrapped = file; + this.applySequenceNumber = applySequenceNumber; + } + + IndexedDeleteFile(PartitionSpec spec, DeleteFile file) { + this.spec = spec; + this.wrapped = file; + + if (file.content() == FileContent.EQUALITY_DELETES) { + this.applySequenceNumber = file.dataSequenceNumber() - 1; + } else { + this.applySequenceNumber = file.dataSequenceNumber(); + } + } + + public PartitionSpec spec() { + return spec; + } + + public DeleteFile wrapped() { + return wrapped; + } + + public long applySequenceNumber() { + return applySequenceNumber; + } + + public FileContent content() { + return wrapped.content(); + } + + public List equalityFieldIds() { + return wrapped.equalityFieldIds(); + } + + public Map valueCounts() { + return wrapped.valueCounts(); + } + + public Map nullValueCounts() { + return wrapped.nullValueCounts(); + } + + public Map nanValueCounts() { + return wrapped.nanValueCounts(); + } + + public boolean hasNoLowerOrUpperBounds() { + return wrapped.lowerBounds() == null || wrapped.upperBounds() == null; + } + + public boolean hasLowerAndUpperBounds() { + return wrapped.lowerBounds() != null && wrapped.upperBounds() != null; + } + + @SuppressWarnings("unchecked") + public T lowerBound(int id) { + return (T) lowerBounds().get(id); + } + + private Map lowerBounds() { + if (convertedLowerBounds == null) { + synchronized (this) { + if (convertedLowerBounds == null) { + this.convertedLowerBounds = convertBounds(wrapped.lowerBounds()); + } + } + } + + return convertedLowerBounds; + } + + @SuppressWarnings("unchecked") + public T upperBound(int id) { + return (T) upperBounds().get(id); + } + + private Map upperBounds() { + if (convertedUpperBounds == null) { + synchronized (this) { + if (convertedUpperBounds == null) { + this.convertedUpperBounds = convertBounds(wrapped.upperBounds()); + } + } + } + + return convertedUpperBounds; + } + + private Map convertBounds(Map bounds) { + Map converted = Maps.newHashMap(); + + if (bounds != null) { + if (wrapped.content() == FileContent.POSITION_DELETES) { + Type pathType = MetadataColumns.DELETE_FILE_PATH.type(); + int pathId = MetadataColumns.DELETE_FILE_PATH.fieldId(); + ByteBuffer bound = bounds.get(pathId); + if (bound != null) { + converted.put(pathId, Conversions.fromByteBuffer(pathType, bound)); + } + + } else { + for (int id : equalityFieldIds()) { + Type type = spec.schema().findField(id).type(); + if (type.isPrimitiveType()) { + ByteBuffer bound = bounds.get(id); + if (bound != null) { + converted.put(id, Conversions.fromByteBuffer(type, bound)); + } + } + } + } + } + + return converted; + } + } } diff --git a/core/src/main/java/org/apache/iceberg/FastAppend.java b/core/src/main/java/org/apache/iceberg/FastAppend.java index 5e5e5128411a..3079757392cd 100644 --- a/core/src/main/java/org/apache/iceberg/FastAppend.java +++ b/core/src/main/java/org/apache/iceberg/FastAppend.java @@ -49,7 +49,7 @@ class FastAppend extends SnapshotProducer implements AppendFiles { private final List newFiles = Lists.newArrayList(); private final List appendManifests = Lists.newArrayList(); private final List rewrittenAppendManifests = Lists.newArrayList(); - private ManifestFile newManifest = null; + private List newManifests = null; private boolean hasNewFiles = false; FastAppend(String tableName, TableOperations ops) { @@ -143,12 +143,12 @@ private ManifestFile copyManifest(ManifestFile manifest) { @Override public List apply(TableMetadata base, Snapshot snapshot) { - List newManifests = Lists.newArrayList(); + List manifests = Lists.newArrayList(); try { - ManifestFile manifest = writeManifest(); - if (manifest != null) { - newManifests.add(manifest); + List newWrittenManifests = writeNewManifests(); + if (newWrittenManifests != null) { + manifests.addAll(newWrittenManifests); } } catch (IOException e) { throw new RuntimeIOException(e, "Failed to write manifest"); @@ -158,13 +158,13 @@ public List apply(TableMetadata base, Snapshot snapshot) { Iterables.transform( Iterables.concat(appendManifests, rewrittenAppendManifests), manifest -> GenericManifestFile.copyOf(manifest).withSnapshotId(snapshotId()).build()); - Iterables.addAll(newManifests, appendManifestsWithMetadata); + Iterables.addAll(manifests, appendManifestsWithMetadata); if (snapshot != null) { - newManifests.addAll(snapshot.allManifests(ops.io())); + manifests.addAll(snapshot.allManifests(ops.io())); } - return newManifests; + return manifests; } @Override @@ -178,8 +178,17 @@ public Object updateEvent() { @Override protected void cleanUncommitted(Set committed) { - if (newManifest != null && !committed.contains(newManifest)) { - deleteFile(newManifest.path()); + if (newManifests != null) { + List committedNewManifests = Lists.newArrayList(); + for (ManifestFile manifest : newManifests) { + if (committed.contains(manifest)) { + committedNewManifests.add(manifest); + } else { + deleteFile(manifest.path()); + } + } + + this.newManifests = committedNewManifests; } // clean up only rewrittenAppendManifests as they are always owned by the table @@ -191,24 +200,24 @@ protected void cleanUncommitted(Set committed) { } } - private ManifestFile writeManifest() throws IOException { - if (hasNewFiles && newManifest != null) { - deleteFile(newManifest.path()); - newManifest = null; + private List writeNewManifests() throws IOException { + if (hasNewFiles && newManifests != null) { + newManifests.forEach(file -> deleteFile(file.path())); + newManifests = null; } - if (newManifest == null && newFiles.size() > 0) { - ManifestWriter writer = newManifestWriter(spec); + if (newManifests == null && newFiles.size() > 0) { + RollingManifestWriter writer = newRollingManifestWriter(spec); try { - writer.addAll(newFiles); + newFiles.forEach(writer::add); } finally { writer.close(); } - this.newManifest = writer.toManifestFile(); + this.newManifests = writer.toManifestFiles(); hasNewFiles = false; } - return newManifest; + return newManifests; } } diff --git a/core/src/main/java/org/apache/iceberg/FileScanTaskParser.java b/core/src/main/java/org/apache/iceberg/FileScanTaskParser.java new file mode 100644 index 000000000000..0a708f2668cc --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/FileScanTaskParser.java @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import java.io.IOException; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.ExpressionParser; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.ResidualEvaluator; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.util.JsonUtil; + +public class FileScanTaskParser { + private static final String SCHEMA = "schema"; + private static final String SPEC = "spec"; + private static final String DATA_FILE = "data-file"; + private static final String START = "start"; + private static final String LENGTH = "length"; + private static final String DELETE_FILES = "delete-files"; + private static final String RESIDUAL = "residual-filter"; + + private FileScanTaskParser() {} + + public static String toJson(FileScanTask fileScanTask) { + return JsonUtil.generate( + generator -> FileScanTaskParser.toJson(fileScanTask, generator), false); + } + + private static void toJson(FileScanTask fileScanTask, JsonGenerator generator) + throws IOException { + Preconditions.checkArgument(fileScanTask != null, "Invalid file scan task: null"); + Preconditions.checkArgument(generator != null, "Invalid JSON generator: null"); + generator.writeStartObject(); + + generator.writeFieldName(SCHEMA); + SchemaParser.toJson(fileScanTask.schema(), generator); + + generator.writeFieldName(SPEC); + PartitionSpec spec = fileScanTask.spec(); + PartitionSpecParser.toJson(spec, generator); + + if (fileScanTask.file() != null) { + generator.writeFieldName(DATA_FILE); + ContentFileParser.toJson(fileScanTask.file(), spec, generator); + } + + generator.writeNumberField(START, fileScanTask.start()); + generator.writeNumberField(LENGTH, fileScanTask.length()); + + if (fileScanTask.deletes() != null) { + generator.writeArrayFieldStart(DELETE_FILES); + for (DeleteFile deleteFile : fileScanTask.deletes()) { + ContentFileParser.toJson(deleteFile, spec, generator); + } + generator.writeEndArray(); + } + + if (fileScanTask.residual() != null) { + generator.writeFieldName(RESIDUAL); + ExpressionParser.toJson(fileScanTask.residual(), generator); + } + + generator.writeEndObject(); + } + + public static FileScanTask fromJson(String json, boolean caseSensitive) { + Preconditions.checkArgument(json != null, "Invalid JSON string for file scan task: null"); + return JsonUtil.parse(json, node -> FileScanTaskParser.fromJson(node, caseSensitive)); + } + + private static FileScanTask fromJson(JsonNode jsonNode, boolean caseSensitive) { + Preconditions.checkArgument(jsonNode != null, "Invalid JSON node for file scan task: null"); + Preconditions.checkArgument( + jsonNode.isObject(), "Invalid JSON node for file scan task: non-object (%s)", jsonNode); + + Schema schema = SchemaParser.fromJson(JsonUtil.get(SCHEMA, jsonNode)); + String schemaString = SchemaParser.toJson(schema); + + PartitionSpec spec = PartitionSpecParser.fromJson(schema, JsonUtil.get(SPEC, jsonNode)); + String specString = PartitionSpecParser.toJson(spec); + + DataFile dataFile = null; + if (jsonNode.has(DATA_FILE)) { + dataFile = (DataFile) ContentFileParser.fromJson(jsonNode.get(DATA_FILE), spec); + } + + long start = JsonUtil.getLong(START, jsonNode); + long length = JsonUtil.getLong(LENGTH, jsonNode); + + DeleteFile[] deleteFiles = null; + if (jsonNode.has(DELETE_FILES)) { + JsonNode deletesArray = jsonNode.get(DELETE_FILES); + Preconditions.checkArgument( + deletesArray.isArray(), + "Invalid JSON node for delete files: non-array (%s)", + deletesArray); + // parse the schema array + ImmutableList.Builder builder = ImmutableList.builder(); + for (JsonNode deleteFileNode : deletesArray) { + DeleteFile deleteFile = (DeleteFile) ContentFileParser.fromJson(deleteFileNode, spec); + builder.add(deleteFile); + } + + deleteFiles = builder.build().toArray(new DeleteFile[0]); + } + + Expression filter = Expressions.alwaysTrue(); + if (jsonNode.has(RESIDUAL)) { + filter = ExpressionParser.fromJson(jsonNode.get(RESIDUAL)); + } + + ResidualEvaluator residualEvaluator = ResidualEvaluator.of(spec, filter, caseSensitive); + BaseFileScanTask baseFileScanTask = + new BaseFileScanTask(dataFile, deleteFiles, schemaString, specString, residualEvaluator); + return new BaseFileScanTask.SplitScanTask(start, length, baseFileScanTask); + } +} diff --git a/core/src/main/java/org/apache/iceberg/GenericDataFile.java b/core/src/main/java/org/apache/iceberg/GenericDataFile.java index 34c65e669fb2..07c5172f1b3f 100644 --- a/core/src/main/java/org/apache/iceberg/GenericDataFile.java +++ b/core/src/main/java/org/apache/iceberg/GenericDataFile.java @@ -40,6 +40,7 @@ class GenericDataFile extends BaseFile implements DataFile { Metrics metrics, ByteBuffer keyMetadata, List splitOffsets, + int[] equalityFieldIds, Integer sortOrderId) { super( specId, @@ -56,7 +57,7 @@ class GenericDataFile extends BaseFile implements DataFile { metrics.lowerBounds(), metrics.upperBounds(), splitOffsets, - null, + equalityFieldIds, sortOrderId, keyMetadata); } diff --git a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java index 50270a05f3dd..cb9a361ab283 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -94,7 +94,7 @@ abstract class MergingSnapshotProducer extends SnapshotProducer { private PartitionSpec dataSpec; // cache new data manifests after writing - private ManifestFile cachedNewDataManifest = null; + private List cachedNewDataManifests = null; private boolean hasNewDataFiles = false; // cache new manifests for delete files @@ -907,9 +907,17 @@ public Object updateEvent() { } private void cleanUncommittedAppends(Set committed) { - if (cachedNewDataManifest != null && !committed.contains(cachedNewDataManifest)) { - deleteFile(cachedNewDataManifest.path()); - this.cachedNewDataManifest = null; + if (cachedNewDataManifests != null) { + List committedNewDataManifests = Lists.newArrayList(); + for (ManifestFile manifest : cachedNewDataManifests) { + if (committed.contains(manifest)) { + committedNewDataManifests.add(manifest); + } else { + deleteFile(manifest.path()); + } + } + + this.cachedNewDataManifests = committedNewDataManifests; } ListIterator deleteManifestsIterator = cachedNewDeleteManifests.listIterator(); @@ -952,10 +960,8 @@ protected void cleanUncommitted(Set committed) { private Iterable prepareNewDataManifests() { Iterable newManifests; if (newDataFiles.size() > 0) { - ManifestFile newManifest = newDataFilesAsManifest(); - newManifests = - Iterables.concat( - ImmutableList.of(newManifest), appendManifests, rewrittenAppendManifests); + List dataFileManifests = newDataFilesAsManifests(); + newManifests = Iterables.concat(dataFileManifests, appendManifests, rewrittenAppendManifests); } else { newManifests = Iterables.concat(appendManifests, rewrittenAppendManifests); } @@ -965,18 +971,18 @@ private Iterable prepareNewDataManifests() { manifest -> GenericManifestFile.copyOf(manifest).withSnapshotId(snapshotId()).build()); } - private ManifestFile newDataFilesAsManifest() { - if (hasNewDataFiles && cachedNewDataManifest != null) { - deleteFile(cachedNewDataManifest.path()); - cachedNewDataManifest = null; + private List newDataFilesAsManifests() { + if (hasNewDataFiles && cachedNewDataManifests != null) { + cachedNewDataManifests.forEach(file -> deleteFile(file.path())); + cachedNewDataManifests = null; } - if (cachedNewDataManifest == null) { + if (cachedNewDataManifests == null) { try { - ManifestWriter writer = newManifestWriter(dataSpec()); + RollingManifestWriter writer = newRollingManifestWriter(dataSpec()); try { if (newDataFilesDataSequenceNumber == null) { - writer.addAll(newDataFiles); + newDataFiles.forEach(writer::add); } else { newDataFiles.forEach(f -> writer.add(f, newDataFilesDataSequenceNumber)); } @@ -984,14 +990,14 @@ private ManifestFile newDataFilesAsManifest() { writer.close(); } - this.cachedNewDataManifest = writer.toManifestFile(); + this.cachedNewDataManifests = writer.toManifestFiles(); this.hasNewDataFiles = false; } catch (IOException e) { throw new RuntimeIOException(e, "Failed to close manifest writer"); } } - return cachedNewDataManifest; + return cachedNewDataManifests; } private Iterable prepareDeleteManifests() { @@ -1017,7 +1023,7 @@ private List newDeleteFilesAsManifests() { (specId, deleteFiles) -> { PartitionSpec spec = ops.current().spec(specId); try { - ManifestWriter writer = newDeleteManifestWriter(spec); + RollingManifestWriter writer = newRollingDeleteManifestWriter(spec); try { deleteFiles.forEach( df -> { @@ -1030,7 +1036,7 @@ private List newDeleteFilesAsManifests() { } finally { writer.close(); } - cachedNewDeleteManifests.add(writer.toManifestFile()); + cachedNewDeleteManifests.addAll(writer.toManifestFiles()); } catch (IOException e) { throw new RuntimeIOException(e, "Failed to close manifest writer"); } diff --git a/core/src/main/java/org/apache/iceberg/PartitionsTable.java b/core/src/main/java/org/apache/iceberg/PartitionsTable.java index b46239352d73..d93200c7cfca 100644 --- a/core/src/main/java/org/apache/iceberg/PartitionsTable.java +++ b/core/src/main/java/org/apache/iceberg/PartitionsTable.java @@ -53,6 +53,11 @@ public class PartitionsTable extends BaseMetadataTable { 2, "record_count", Types.LongType.get(), "Count of records in data files"), Types.NestedField.required( 3, "file_count", Types.IntegerType.get(), "Count of data files"), + Types.NestedField.required( + 11, + "total_data_file_size_in_bytes", + Types.LongType.get(), + "Total size in bytes of data files"), Types.NestedField.required( 5, "position_delete_record_count", @@ -72,7 +77,17 @@ public class PartitionsTable extends BaseMetadataTable { 8, "equality_delete_file_count", Types.IntegerType.get(), - "Count of equality delete files")); + "Count of equality delete files"), + Types.NestedField.optional( + 9, + "last_updated_at", + Types.TimestampType.withZone(), + "Commit time of snapshot that last updated this partition"), + Types.NestedField.optional( + 10, + "last_updated_snapshot_id", + Types.LongType.get(), + "Id of snapshot that last updated this partition")); this.unpartitionedTable = Partitioning.partitionType(table).fields().isEmpty(); } @@ -87,10 +102,13 @@ public Schema schema() { return schema.select( "record_count", "file_count", + "total_data_file_size_in_bytes", "position_delete_record_count", "position_delete_file_count", "equality_delete_record_count", - "equality_delete_file_count"); + "equality_delete_file_count", + "last_updated_at", + "last_updated_snapshot_id"); } return schema; } @@ -113,10 +131,13 @@ private DataTask task(StaticTableScan scan) { StaticDataTask.Row.of( root.dataRecordCount, root.dataFileCount, + root.dataFileSizeInBytes, root.posDeleteRecordCount, root.posDeleteFileCount, root.eqDeleteRecordCount, - root.eqDeleteFileCount)); + root.eqDeleteFileCount, + root.lastUpdatedAt, + root.lastUpdatedSnapshotId)); } else { return StaticDataTask.of( io().newInputFile(table().operations().current().metadataFileLocation()), @@ -133,22 +154,26 @@ private static StaticDataTask.Row convertPartition(Partition partition) { partition.specId, partition.dataRecordCount, partition.dataFileCount, + partition.dataFileSizeInBytes, partition.posDeleteRecordCount, partition.posDeleteFileCount, partition.eqDeleteRecordCount, - partition.eqDeleteFileCount); + partition.eqDeleteFileCount, + partition.lastUpdatedAt, + partition.lastUpdatedSnapshotId); } private static Iterable partitions(Table table, StaticTableScan scan) { Types.StructType partitionType = Partitioning.partitionType(table); PartitionMap partitions = new PartitionMap(partitionType); - - try (CloseableIterable> files = planFiles(scan)) { - for (ContentFile file : files) { + try (CloseableIterable>> entries = planEntries(scan)) { + for (ManifestEntry> entry : entries) { + Snapshot snapshot = table.snapshot(entry.snapshotId()); + ContentFile file = entry.file(); StructLike partition = PartitionUtil.coercePartition( partitionType, table.specs().get(file.specId()), file.partition()); - partitions.get(partition).update(file); + partitions.get(partition).update(file, snapshot); } } catch (IOException e) { throw new UncheckedIOException(e); @@ -158,25 +183,32 @@ private static Iterable partitions(Table table, StaticTableScan scan) } @VisibleForTesting - static CloseableIterable> planFiles(StaticTableScan scan) { + static CloseableIterable> planEntries(StaticTableScan scan) { Table table = scan.table(); CloseableIterable filteredManifests = filteredManifests(scan, table, scan.snapshot().allManifests(table.io())); - Iterable>> tasks = - CloseableIterable.transform( - filteredManifests, - manifest -> - CloseableIterable.transform( - ManifestFiles.open(manifest, table.io(), table.specs()) - .caseSensitive(scan.isCaseSensitive()) - .select(scanColumns(manifest.content())), // don't select stats columns - t -> (ContentFile) t)); + Iterable>> tasks = + CloseableIterable.transform(filteredManifests, manifest -> readEntries(manifest, scan)); return new ParallelIterable<>(tasks, scan.planExecutor()); } + private static CloseableIterable> readEntries( + ManifestFile manifest, StaticTableScan scan) { + Table table = scan.table(); + return CloseableIterable.transform( + ManifestFiles.open(manifest, table.io(), table.specs()) + .caseSensitive(scan.isCaseSensitive()) + .select(scanColumns(manifest.content())) // don't select stats columns + .entries(), + t -> + (ManifestEntry>) + // defensive copy of manifest entry without stats columns + t.copyWithoutStats()); + } + private static List scanColumns(ManifestContent content) { switch (content) { case DATA: @@ -245,28 +277,41 @@ static class Partition { private int specId; private long dataRecordCount; private int dataFileCount; + private long dataFileSizeInBytes; private long posDeleteRecordCount; private int posDeleteFileCount; private long eqDeleteRecordCount; private int eqDeleteFileCount; + private Long lastUpdatedAt; + private Long lastUpdatedSnapshotId; Partition(StructLike key, Types.StructType keyType) { this.partitionData = toPartitionData(key, keyType); this.specId = 0; - this.dataRecordCount = 0; + this.dataRecordCount = 0L; this.dataFileCount = 0; - this.posDeleteRecordCount = 0; + this.dataFileSizeInBytes = 0L; + this.posDeleteRecordCount = 0L; this.posDeleteFileCount = 0; - this.eqDeleteRecordCount = 0; + this.eqDeleteRecordCount = 0L; this.eqDeleteFileCount = 0; } - void update(ContentFile file) { + void update(ContentFile file, Snapshot snapshot) { + if (snapshot != null) { + long snapshotCommitTime = snapshot.timestampMillis() * 1000; + if (this.lastUpdatedAt == null || snapshotCommitTime > this.lastUpdatedAt) { + this.lastUpdatedAt = snapshotCommitTime; + this.lastUpdatedSnapshotId = snapshot.snapshotId(); + } + } + switch (file.content()) { case DATA: this.dataRecordCount += file.recordCount(); this.dataFileCount += 1; this.specId = file.specId(); + this.dataFileSizeInBytes += file.fileSizeInBytes(); break; case POSITION_DELETES: this.posDeleteRecordCount = file.recordCount(); diff --git a/core/src/main/java/org/apache/iceberg/RollingManifestWriter.java b/core/src/main/java/org/apache/iceberg/RollingManifestWriter.java new file mode 100644 index 000000000000..5480415eeed2 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/RollingManifestWriter.java @@ -0,0 +1,150 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import java.io.Closeable; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.List; +import java.util.function.Supplier; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +/** As opposed to {@link ManifestWriter}, a rolling writer could produce multiple manifest files. */ +public class RollingManifestWriter> implements Closeable { + private static final int ROWS_DIVISOR = 250; + + private final Supplier> manifestWriterSupplier; + private final long targetFileSizeInBytes; + private final List manifestFiles; + + private long currentFileRows = 0; + private ManifestWriter currentWriter = null; + + private boolean closed = false; + + public RollingManifestWriter( + Supplier> manifestWriterSupplier, long targetFileSizeInBytes) { + this.manifestWriterSupplier = manifestWriterSupplier; + this.targetFileSizeInBytes = targetFileSizeInBytes; + this.manifestFiles = Lists.newArrayList(); + } + + /** + * Add an added entry for a file. + * + *

The entry's snapshot ID will be this manifest's snapshot ID. The data and file sequence + * numbers will be assigned at commit. + * + * @param addedFile a data file + */ + public void add(F addedFile) { + currentWriter().add(addedFile); + currentFileRows++; + } + + /** + * Add an added entry for a file with a specific sequence number. + * + *

The entry's snapshot ID will be this manifest's snapshot ID. The entry's data sequence + * number will be the provided data sequence number. The entry's file sequence number will be + * assigned at commit. + * + * @param addedFile a data file + * @param dataSequenceNumber a data sequence number for the file + */ + public void add(F addedFile, long dataSequenceNumber) { + currentWriter().add(addedFile, dataSequenceNumber); + currentFileRows++; + } + + /** + * Add an existing entry for a file. + * + *

The original data and file sequence numbers, snapshot ID, which were assigned at commit, + * must be preserved when adding an existing entry. + * + * @param existingFile a file + * @param fileSnapshotId snapshot ID when the data file was added to the table + * @param dataSequenceNumber a data sequence number of the file (assigned when the file was added) + * @param fileSequenceNumber a file sequence number (assigned when the file was added) + */ + public void existing( + F existingFile, long fileSnapshotId, long dataSequenceNumber, Long fileSequenceNumber) { + currentWriter().existing(existingFile, fileSnapshotId, dataSequenceNumber, fileSequenceNumber); + currentFileRows++; + } + + /** + * Add a delete entry for a file. + * + *

The entry's snapshot ID will be this manifest's snapshot ID. However, the original data and + * file sequence numbers of the file must be preserved when the file is marked as deleted. + * + * @param deletedFile a file + * @param dataSequenceNumber a data sequence number of the file (assigned when the file was added) + * @param fileSequenceNumber a file sequence number (assigned when the file was added) + */ + public void delete(F deletedFile, long dataSequenceNumber, Long fileSequenceNumber) { + currentWriter().delete(deletedFile, dataSequenceNumber, fileSequenceNumber); + currentFileRows++; + } + + private ManifestWriter currentWriter() { + if (currentWriter == null) { + this.currentWriter = manifestWriterSupplier.get(); + } else if (shouldRollToNewFile()) { + closeCurrentWriter(); + this.currentWriter = manifestWriterSupplier.get(); + } + + return currentWriter; + } + + private boolean shouldRollToNewFile() { + return currentFileRows % ROWS_DIVISOR == 0 && currentWriter.length() >= targetFileSizeInBytes; + } + + private void closeCurrentWriter() { + if (currentWriter != null) { + try { + currentWriter.close(); + ManifestFile currentFile = currentWriter.toManifestFile(); + manifestFiles.add(currentFile); + this.currentWriter = null; + this.currentFileRows = 0; + } catch (IOException e) { + throw new UncheckedIOException("Failed to close current writer", e); + } + } + } + + @Override + public void close() throws IOException { + if (!closed) { + closeCurrentWriter(); + this.closed = true; + } + } + + public List toManifestFiles() { + Preconditions.checkState(closed, "Cannot get ManifestFile list from unclosed writer"); + return manifestFiles; + } +} diff --git a/core/src/main/java/org/apache/iceberg/ScanSummary.java b/core/src/main/java/org/apache/iceberg/ScanSummary.java index 7b8cf22e9a67..f975ef1636df 100644 --- a/core/src/main/java/org/apache/iceberg/ScanSummary.java +++ b/core/src/main/java/org/apache/iceberg/ScanSummary.java @@ -190,7 +190,7 @@ public Map build() { "Cannot satisfy time filters: time range may include expired snapshots"); } - // filter down to the the set of manifest files that were added after the start of the + // filter down to the set of manifest files that were added after the start of the // time range. manifests after the end of the time range must be included because // compaction may create a manifest after the time range that includes files added in the // range. diff --git a/core/src/main/java/org/apache/iceberg/SnapshotManager.java b/core/src/main/java/org/apache/iceberg/SnapshotManager.java index f9015c04b882..c9774f3b929b 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotManager.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotManager.java @@ -68,6 +68,20 @@ public ManageSnapshots rollbackTo(long snapshotId) { return this; } + @Override + public ManageSnapshots createBranch(String name) { + Snapshot currentSnapshot = transaction.currentMetadata().currentSnapshot(); + if (currentSnapshot != null) { + return createBranch(name, currentSnapshot.snapshotId()); + } + + SnapshotRef existingRef = transaction.currentMetadata().ref(name); + Preconditions.checkArgument(existingRef == null, "Ref %s already exists", name); + // Create an empty snapshot for the branch + transaction.newFastAppend().toBranch(name).commit(); + return this; + } + @Override public ManageSnapshots createBranch(String name, long snapshotId) { updateSnapshotReferencesOperation().createBranch(name, snapshotId); diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index 226388a2b028..5a6a01ea062d 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -26,6 +26,8 @@ import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES_DEFAULT; import static org.apache.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS; import static org.apache.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT; +import static org.apache.iceberg.TableProperties.MANIFEST_TARGET_SIZE_BYTES; +import static org.apache.iceberg.TableProperties.MANIFEST_TARGET_SIZE_BYTES_DEFAULT; import com.github.benmanes.caffeine.cache.Caffeine; import com.github.benmanes.caffeine.cache.LoadingCache; @@ -85,6 +87,7 @@ public void accept(String file) { private final AtomicInteger manifestCount = new AtomicInteger(0); private final AtomicInteger attempt = new AtomicInteger(0); private final List manifestLists = Lists.newArrayList(); + private final long targetManifestSizeBytes; private MetricsReporter reporter = LoggingMetricsReporter.instance(); private volatile Long snapshotId = null; private TableMetadata base; @@ -107,6 +110,9 @@ protected SnapshotProducer(TableOperations ops) { } return addMetadata(ops, file); }); + this.targetManifestSizeBytes = + ops.current() + .propertyAsLong(MANIFEST_TARGET_SIZE_BYTES, MANIFEST_TARGET_SIZE_BYTES_DEFAULT); } protected abstract ThisT self(); @@ -494,6 +500,15 @@ protected ManifestWriter newDeleteManifestWriter(PartitionSpec spec) ops.current().formatVersion(), spec, newManifestOutput(), snapshotId()); } + protected RollingManifestWriter newRollingManifestWriter(PartitionSpec spec) { + return new RollingManifestWriter<>(() -> newManifestWriter(spec), targetManifestSizeBytes); + } + + protected RollingManifestWriter newRollingDeleteManifestWriter(PartitionSpec spec) { + return new RollingManifestWriter<>( + () -> newDeleteManifestWriter(spec), targetManifestSizeBytes); + } + protected ManifestReader newManifestReader(ManifestFile manifest) { return ManifestFiles.read(manifest, ops.io(), ops.current().specsById()); } diff --git a/core/src/main/java/org/apache/iceberg/SnapshotScan.java b/core/src/main/java/org/apache/iceberg/SnapshotScan.java index b6520c2ff4d3..de53444ba99a 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotScan.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotScan.java @@ -140,7 +140,9 @@ public CloseableIterable planFiles() { .projectedFieldNames(projectedFieldNames) .tableName(table().name()) .snapshotId(snapshot.snapshotId()) - .filter(ExpressionUtil.sanitize(filter())) + .filter( + ExpressionUtil.sanitize( + schema().asStruct(), filter(), context().caseSensitive())) .scanMetrics(ScanMetricsResult.fromScanMetrics(scanMetrics())) .metadata(metadata) .build(); diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index bd409bdeed3a..25af350d5e8e 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -499,11 +499,6 @@ private synchronized void ensureSnapshotsLoaded() { List loadedSnapshots = Lists.newArrayList(snapshotsSupplier.get()); loadedSnapshots.removeIf(s -> s.sequenceNumber() > lastSequenceNumber); - // Format version 1 does not have accurate sequence numbering, so remove based on timestamp - if (this.formatVersion == 1) { - loadedSnapshots.removeIf(s -> s.timestampMillis() > currentSnapshot().timestampMillis()); - } - this.snapshots = ImmutableList.copyOf(loadedSnapshots); this.snapshotsById = indexAndValidateSnapshots(snapshots, lastSequenceNumber); validateCurrentSnapshot(); diff --git a/core/src/main/java/org/apache/iceberg/TableMetadataParser.java b/core/src/main/java/org/apache/iceberg/TableMetadataParser.java index dc234c32ae06..473f8497cbb4 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadataParser.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadataParser.java @@ -299,7 +299,7 @@ public static TableMetadata fromJson(String metadataLocation, String json) { return JsonUtil.parse(json, node -> TableMetadataParser.fromJson(metadataLocation, node)); } - static TableMetadata fromJson(InputFile file, JsonNode node) { + public static TableMetadata fromJson(InputFile file, JsonNode node) { return fromJson(file.location(), node); } @@ -308,7 +308,7 @@ public static TableMetadata fromJson(JsonNode node) { } @SuppressWarnings({"checkstyle:CyclomaticComplexity", "checkstyle:MethodLength"}) - static TableMetadata fromJson(String metadataLocation, JsonNode node) { + public static TableMetadata fromJson(String metadataLocation, JsonNode node) { Preconditions.checkArgument( node.isObject(), "Cannot parse metadata from a non-object: %s", node); @@ -431,15 +431,26 @@ static TableMetadata fromJson(String metadataLocation, JsonNode node) { defaultSortOrderId = defaultSortOrder.orderId(); } - // parse properties map - Map properties = JsonUtil.getStringMap(PROPERTIES, node); - long currentSnapshotId = JsonUtil.getLong(CURRENT_SNAPSHOT_ID, node); + Map properties; + if (node.has(PROPERTIES)) { + // parse properties map + properties = JsonUtil.getStringMap(PROPERTIES, node); + } else { + properties = ImmutableMap.of(); + } + + Long currentSnapshotId = JsonUtil.getLongOrNull(CURRENT_SNAPSHOT_ID, node); + if (currentSnapshotId == null) { + // This field is optional, but internally we set this to -1 when not set + currentSnapshotId = -1L; + } + long lastUpdatedMillis = JsonUtil.getLong(LAST_UPDATED_MILLIS, node); Map refs; if (node.has(REFS)) { refs = refsFromJson(node.get(REFS)); - } else if (currentSnapshotId != -1) { + } else if (currentSnapshotId != -1L) { // initialize the main branch if there are no refs refs = ImmutableMap.of( @@ -448,14 +459,19 @@ static TableMetadata fromJson(String metadataLocation, JsonNode node) { refs = ImmutableMap.of(); } - JsonNode snapshotArray = JsonUtil.get(SNAPSHOTS, node); - Preconditions.checkArgument( - snapshotArray.isArray(), "Cannot parse snapshots from non-array: %s", snapshotArray); + List snapshots; + if (node.has(SNAPSHOTS)) { + JsonNode snapshotArray = JsonUtil.get(SNAPSHOTS, node); + Preconditions.checkArgument( + snapshotArray.isArray(), "Cannot parse snapshots from non-array: %s", snapshotArray); - List snapshots = Lists.newArrayListWithExpectedSize(snapshotArray.size()); - Iterator iterator = snapshotArray.elements(); - while (iterator.hasNext()) { - snapshots.add(SnapshotParser.fromJson(iterator.next())); + snapshots = Lists.newArrayListWithExpectedSize(snapshotArray.size()); + Iterator iterator = snapshotArray.elements(); + while (iterator.hasNext()) { + snapshots.add(SnapshotParser.fromJson(iterator.next())); + } + } else { + snapshots = ImmutableList.of(); } List statisticsFiles; diff --git a/core/src/main/java/org/apache/iceberg/TableProperties.java b/core/src/main/java/org/apache/iceberg/TableProperties.java index b14354def6ac..a9116bc57f83 100644 --- a/core/src/main/java/org/apache/iceberg/TableProperties.java +++ b/core/src/main/java/org/apache/iceberg/TableProperties.java @@ -216,6 +216,9 @@ private TableProperties() {} public static final String SPLIT_OPEN_FILE_COST = "read.split.open-file-cost"; public static final long SPLIT_OPEN_FILE_COST_DEFAULT = 4 * 1024 * 1024; // 4MB + public static final String ADAPTIVE_SPLIT_SIZE_ENABLED = "read.split.adaptive-size.enabled"; + public static final boolean ADAPTIVE_SPLIT_SIZE_ENABLED_DEFAULT = true; + public static final String PARQUET_VECTORIZATION_ENABLED = "read.parquet.vectorization.enabled"; public static final boolean PARQUET_VECTORIZATION_ENABLED_DEFAULT = true; diff --git a/core/src/main/java/org/apache/iceberg/UpdateRequirement.java b/core/src/main/java/org/apache/iceberg/UpdateRequirement.java new file mode 100644 index 000000000000..2645e79fa4c5 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/UpdateRequirement.java @@ -0,0 +1,202 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** Represents a requirement for a {@link MetadataUpdate} */ +public interface UpdateRequirement { + void validate(TableMetadata base); + + class AssertTableDoesNotExist implements UpdateRequirement { + public AssertTableDoesNotExist() {} + + @Override + public void validate(TableMetadata base) { + if (base != null) { + throw new CommitFailedException("Requirement failed: table already exists"); + } + } + } + + class AssertTableUUID implements UpdateRequirement { + private final String uuid; + + public AssertTableUUID(String uuid) { + Preconditions.checkArgument(uuid != null, "Invalid required UUID: null"); + this.uuid = uuid; + } + + public String uuid() { + return uuid; + } + + @Override + public void validate(TableMetadata base) { + if (!uuid.equalsIgnoreCase(base.uuid())) { + throw new CommitFailedException( + "Requirement failed: UUID does not match: expected %s != %s", base.uuid(), uuid); + } + } + } + + class AssertRefSnapshotID implements UpdateRequirement { + private final String name; + private final Long snapshotId; + + public AssertRefSnapshotID(String name, Long snapshotId) { + this.name = name; + this.snapshotId = snapshotId; + } + + public String refName() { + return name; + } + + public Long snapshotId() { + return snapshotId; + } + + @Override + public void validate(TableMetadata base) { + SnapshotRef ref = base.ref(name); + if (ref != null) { + String type = ref.isBranch() ? "branch" : "tag"; + if (snapshotId == null) { + // a null snapshot ID means the ref should not exist already + throw new CommitFailedException( + "Requirement failed: %s %s was created concurrently", type, name); + } else if (snapshotId != ref.snapshotId()) { + throw new CommitFailedException( + "Requirement failed: %s %s has changed: expected id %s != %s", + type, name, snapshotId, ref.snapshotId()); + } + } else if (snapshotId != null) { + throw new CommitFailedException( + "Requirement failed: branch or tag %s is missing, expected %s", name, snapshotId); + } + } + } + + class AssertLastAssignedFieldId implements UpdateRequirement { + private final int lastAssignedFieldId; + + public AssertLastAssignedFieldId(int lastAssignedFieldId) { + this.lastAssignedFieldId = lastAssignedFieldId; + } + + public int lastAssignedFieldId() { + return lastAssignedFieldId; + } + + @Override + public void validate(TableMetadata base) { + if (base != null && base.lastColumnId() != lastAssignedFieldId) { + throw new CommitFailedException( + "Requirement failed: last assigned field id changed: expected id %s != %s", + lastAssignedFieldId, base.lastColumnId()); + } + } + } + + class AssertCurrentSchemaID implements UpdateRequirement { + private final int schemaId; + + public AssertCurrentSchemaID(int schemaId) { + this.schemaId = schemaId; + } + + public int schemaId() { + return schemaId; + } + + @Override + public void validate(TableMetadata base) { + if (schemaId != base.currentSchemaId()) { + throw new CommitFailedException( + "Requirement failed: current schema changed: expected id %s != %s", + schemaId, base.currentSchemaId()); + } + } + } + + class AssertLastAssignedPartitionId implements UpdateRequirement { + private final int lastAssignedPartitionId; + + public AssertLastAssignedPartitionId(int lastAssignedPartitionId) { + this.lastAssignedPartitionId = lastAssignedPartitionId; + } + + public int lastAssignedPartitionId() { + return lastAssignedPartitionId; + } + + @Override + public void validate(TableMetadata base) { + if (base != null && base.lastAssignedPartitionId() != lastAssignedPartitionId) { + throw new CommitFailedException( + "Requirement failed: last assigned partition id changed: expected id %s != %s", + lastAssignedPartitionId, base.lastAssignedPartitionId()); + } + } + } + + class AssertDefaultSpecID implements UpdateRequirement { + private final int specId; + + public AssertDefaultSpecID(int specId) { + this.specId = specId; + } + + public int specId() { + return specId; + } + + @Override + public void validate(TableMetadata base) { + if (specId != base.defaultSpecId()) { + throw new CommitFailedException( + "Requirement failed: default partition spec changed: expected id %s != %s", + specId, base.defaultSpecId()); + } + } + } + + class AssertDefaultSortOrderID implements UpdateRequirement { + private final int sortOrderId; + + public AssertDefaultSortOrderID(int sortOrderId) { + this.sortOrderId = sortOrderId; + } + + public int sortOrderId() { + return sortOrderId; + } + + @Override + public void validate(TableMetadata base) { + if (sortOrderId != base.defaultSortOrderId()) { + throw new CommitFailedException( + "Requirement failed: default sort order changed: expected id %s != %s", + sortOrderId, base.defaultSortOrderId()); + } + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/UpdateRequirementParser.java b/core/src/main/java/org/apache/iceberg/UpdateRequirementParser.java new file mode 100644 index 000000000000..091d4f1fc58c --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/UpdateRequirementParser.java @@ -0,0 +1,264 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import java.io.IOException; +import java.util.Locale; +import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.util.JsonUtil; + +public class UpdateRequirementParser { + + private UpdateRequirementParser() {} + + private static final String TYPE = "type"; + + // assertion types + static final String ASSERT_TABLE_UUID = "assert-table-uuid"; + static final String ASSERT_TABLE_DOES_NOT_EXIST = "assert-create"; + static final String ASSERT_REF_SNAPSHOT_ID = "assert-ref-snapshot-id"; + static final String ASSERT_LAST_ASSIGNED_FIELD_ID = "assert-last-assigned-field-id"; + static final String ASSERT_CURRENT_SCHEMA_ID = "assert-current-schema-id"; + static final String ASSERT_LAST_ASSIGNED_PARTITION_ID = "assert-last-assigned-partition-id"; + static final String ASSERT_DEFAULT_SPEC_ID = "assert-default-spec-id"; + static final String ASSERT_DEFAULT_SORT_ORDER_ID = "assert-default-sort-order-id"; + + // AssertTableUUID + private static final String UUID = "uuid"; + + // AssertRefSnapshotID + private static final String NAME = "ref"; + private static final String SNAPSHOT_ID = "snapshot-id"; + + // AssertLastAssignedFieldId + private static final String LAST_ASSIGNED_FIELD_ID = "last-assigned-field-id"; + + // AssertCurrentSchemaID + private static final String SCHEMA_ID = "current-schema-id"; + + // AssertLastAssignedPartitionId + private static final String LAST_ASSIGNED_PARTITION_ID = "last-assigned-partition-id"; + + // AssertDefaultSpecID + private static final String SPEC_ID = "default-spec-id"; + + // AssertDefaultSortOrderID + private static final String SORT_ORDER_ID = "default-sort-order-id"; + + private static final Map, String> TYPES = + ImmutableMap., String>builder() + .put(UpdateRequirement.AssertTableUUID.class, ASSERT_TABLE_UUID) + .put(UpdateRequirement.AssertTableDoesNotExist.class, ASSERT_TABLE_DOES_NOT_EXIST) + .put(UpdateRequirement.AssertRefSnapshotID.class, ASSERT_REF_SNAPSHOT_ID) + .put(UpdateRequirement.AssertLastAssignedFieldId.class, ASSERT_LAST_ASSIGNED_FIELD_ID) + .put(UpdateRequirement.AssertCurrentSchemaID.class, ASSERT_CURRENT_SCHEMA_ID) + .put( + UpdateRequirement.AssertLastAssignedPartitionId.class, + ASSERT_LAST_ASSIGNED_PARTITION_ID) + .put(UpdateRequirement.AssertDefaultSpecID.class, ASSERT_DEFAULT_SPEC_ID) + .put(UpdateRequirement.AssertDefaultSortOrderID.class, ASSERT_DEFAULT_SORT_ORDER_ID) + .buildOrThrow(); + + public static String toJson(UpdateRequirement updateRequirement) { + return toJson(updateRequirement, false); + } + + public static String toJson(UpdateRequirement updateRequirement, boolean pretty) { + return JsonUtil.generate(gen -> toJson(updateRequirement, gen), pretty); + } + + public static void toJson(UpdateRequirement updateRequirement, JsonGenerator generator) + throws IOException { + String requirementType = TYPES.get(updateRequirement.getClass()); + + generator.writeStartObject(); + generator.writeStringField(TYPE, requirementType); + + switch (requirementType) { + case ASSERT_TABLE_DOES_NOT_EXIST: + // No fields beyond the requirement itself + break; + case ASSERT_TABLE_UUID: + writeAssertTableUUID((UpdateRequirement.AssertTableUUID) updateRequirement, generator); + break; + case ASSERT_REF_SNAPSHOT_ID: + writeAssertRefSnapshotId( + (UpdateRequirement.AssertRefSnapshotID) updateRequirement, generator); + break; + case ASSERT_LAST_ASSIGNED_FIELD_ID: + writeAssertLastAssignedFieldId( + (UpdateRequirement.AssertLastAssignedFieldId) updateRequirement, generator); + break; + case ASSERT_LAST_ASSIGNED_PARTITION_ID: + writeAssertLastAssignedPartitionId( + (UpdateRequirement.AssertLastAssignedPartitionId) updateRequirement, generator); + break; + case ASSERT_CURRENT_SCHEMA_ID: + writeAssertCurrentSchemaId( + (UpdateRequirement.AssertCurrentSchemaID) updateRequirement, generator); + break; + case ASSERT_DEFAULT_SPEC_ID: + writeAssertDefaultSpecId( + (UpdateRequirement.AssertDefaultSpecID) updateRequirement, generator); + break; + case ASSERT_DEFAULT_SORT_ORDER_ID: + writeAssertDefaultSortOrderId( + (UpdateRequirement.AssertDefaultSortOrderID) updateRequirement, generator); + break; + default: + throw new IllegalArgumentException( + String.format( + "Cannot convert update requirement to json. Unrecognized type: %s", + requirementType)); + } + + generator.writeEndObject(); + } + + /** + * Read MetadataUpdate from a JSON string. + * + * @param json a JSON string of a MetadataUpdate + * @return a MetadataUpdate object + */ + public static UpdateRequirement fromJson(String json) { + return JsonUtil.parse(json, UpdateRequirementParser::fromJson); + } + + public static UpdateRequirement fromJson(JsonNode jsonNode) { + Preconditions.checkArgument( + jsonNode != null && jsonNode.isObject(), + "Cannot parse update requirement from non-object value: %s", + jsonNode); + Preconditions.checkArgument( + jsonNode.hasNonNull(TYPE), "Cannot parse update requirement. Missing field: type"); + String type = JsonUtil.getString(TYPE, jsonNode).toLowerCase(Locale.ROOT); + + switch (type) { + case ASSERT_TABLE_DOES_NOT_EXIST: + return readAssertTableDoesNotExist(jsonNode); + case ASSERT_TABLE_UUID: + return readAssertTableUUID(jsonNode); + case ASSERT_REF_SNAPSHOT_ID: + return readAssertRefSnapshotId(jsonNode); + case ASSERT_LAST_ASSIGNED_FIELD_ID: + return readAssertLastAssignedFieldId(jsonNode); + case ASSERT_LAST_ASSIGNED_PARTITION_ID: + return readAssertLastAssignedPartitionId(jsonNode); + case ASSERT_CURRENT_SCHEMA_ID: + return readAssertCurrentSchemaId(jsonNode); + case ASSERT_DEFAULT_SPEC_ID: + return readAssertDefaultSpecId(jsonNode); + case ASSERT_DEFAULT_SORT_ORDER_ID: + return readAssertDefaultSortOrderId(jsonNode); + default: + throw new UnsupportedOperationException( + String.format("Unrecognized update requirement. Cannot convert to json: %s", type)); + } + } + + private static void writeAssertTableUUID( + UpdateRequirement.AssertTableUUID requirement, JsonGenerator gen) throws IOException { + gen.writeStringField(UUID, requirement.uuid()); + } + + private static void writeAssertRefSnapshotId( + UpdateRequirement.AssertRefSnapshotID requirement, JsonGenerator gen) throws IOException { + gen.writeStringField(NAME, requirement.refName()); + if (requirement.snapshotId() != null) { + gen.writeNumberField(SNAPSHOT_ID, requirement.snapshotId()); + } else { + gen.writeNullField(SNAPSHOT_ID); + } + } + + private static void writeAssertLastAssignedFieldId( + UpdateRequirement.AssertLastAssignedFieldId requirement, JsonGenerator gen) + throws IOException { + gen.writeNumberField(LAST_ASSIGNED_FIELD_ID, requirement.lastAssignedFieldId()); + } + + private static void writeAssertLastAssignedPartitionId( + UpdateRequirement.AssertLastAssignedPartitionId requirement, JsonGenerator gen) + throws IOException { + gen.writeNumberField(LAST_ASSIGNED_PARTITION_ID, requirement.lastAssignedPartitionId()); + } + + private static void writeAssertCurrentSchemaId( + UpdateRequirement.AssertCurrentSchemaID requirement, JsonGenerator gen) throws IOException { + gen.writeNumberField(SCHEMA_ID, requirement.schemaId()); + } + + private static void writeAssertDefaultSpecId( + UpdateRequirement.AssertDefaultSpecID requirement, JsonGenerator gen) throws IOException { + gen.writeNumberField(SPEC_ID, requirement.specId()); + } + + private static void writeAssertDefaultSortOrderId( + UpdateRequirement.AssertDefaultSortOrderID requirement, JsonGenerator gen) + throws IOException { + gen.writeNumberField(SORT_ORDER_ID, requirement.sortOrderId()); + } + + @SuppressWarnings( + "unused") // Keep same signature in case this requirement class evolves and gets fields + private static UpdateRequirement readAssertTableDoesNotExist(JsonNode node) { + return new UpdateRequirement.AssertTableDoesNotExist(); + } + + private static UpdateRequirement readAssertTableUUID(JsonNode node) { + String uuid = JsonUtil.getString(UUID, node); + return new UpdateRequirement.AssertTableUUID(uuid); + } + + private static UpdateRequirement readAssertRefSnapshotId(JsonNode node) { + String name = JsonUtil.getString(NAME, node); + Long snapshotId = JsonUtil.getLongOrNull(SNAPSHOT_ID, node); + return new UpdateRequirement.AssertRefSnapshotID(name, snapshotId); + } + + private static UpdateRequirement readAssertLastAssignedFieldId(JsonNode node) { + int lastAssignedFieldId = JsonUtil.getInt(LAST_ASSIGNED_FIELD_ID, node); + return new UpdateRequirement.AssertLastAssignedFieldId(lastAssignedFieldId); + } + + private static UpdateRequirement readAssertCurrentSchemaId(JsonNode node) { + int schemaId = JsonUtil.getInt(SCHEMA_ID, node); + return new UpdateRequirement.AssertCurrentSchemaID(schemaId); + } + + private static UpdateRequirement readAssertLastAssignedPartitionId(JsonNode node) { + int lastAssignedPartitionId = JsonUtil.getInt(LAST_ASSIGNED_PARTITION_ID, node); + return new UpdateRequirement.AssertLastAssignedPartitionId(lastAssignedPartitionId); + } + + private static UpdateRequirement readAssertDefaultSpecId(JsonNode node) { + int specId = JsonUtil.getInt(SPEC_ID, node); + return new UpdateRequirement.AssertDefaultSpecID(specId); + } + + private static UpdateRequirement readAssertDefaultSortOrderId(JsonNode node) { + int sortOrderId = JsonUtil.getInt(SORT_ORDER_ID, node); + return new UpdateRequirement.AssertDefaultSortOrderID(sortOrderId); + } +} diff --git a/core/src/main/java/org/apache/iceberg/UpdateRequirements.java b/core/src/main/java/org/apache/iceberg/UpdateRequirements.java new file mode 100644 index 000000000000..8a7a761ff2c1 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/UpdateRequirements.java @@ -0,0 +1,169 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import java.util.List; +import java.util.Set; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; + +public class UpdateRequirements { + + private UpdateRequirements() {} + + public static List forCreateTable(List metadataUpdates) { + Preconditions.checkArgument(null != metadataUpdates, "Invalid metadata updates: null"); + Builder builder = new Builder(null, false); + builder.require(new UpdateRequirement.AssertTableDoesNotExist()); + metadataUpdates.forEach(builder::update); + return builder.build(); + } + + public static List forReplaceTable( + TableMetadata base, List metadataUpdates) { + Preconditions.checkArgument(null != base, "Invalid table metadata: null"); + Preconditions.checkArgument(null != metadataUpdates, "Invalid metadata updates: null"); + Builder builder = new Builder(base, true); + builder.require(new UpdateRequirement.AssertTableUUID(base.uuid())); + metadataUpdates.forEach(builder::update); + return builder.build(); + } + + public static List forUpdateTable( + TableMetadata base, List metadataUpdates) { + Preconditions.checkArgument(null != base, "Invalid table metadata: null"); + Preconditions.checkArgument(null != metadataUpdates, "Invalid metadata updates: null"); + Builder builder = new Builder(base, false); + builder.require(new UpdateRequirement.AssertTableUUID(base.uuid())); + metadataUpdates.forEach(builder::update); + return builder.build(); + } + + private static class Builder { + private final TableMetadata base; + private final ImmutableList.Builder requirements = ImmutableList.builder(); + private final Set changedRefs = Sets.newHashSet(); + private final boolean isReplace; + private boolean addedSchema = false; + private boolean setSchemaId = false; + private boolean addedSpec = false; + private boolean setSpecId = false; + private boolean setOrderId = false; + + private Builder(TableMetadata base, boolean isReplace) { + this.base = base; + this.isReplace = isReplace; + } + + private Builder require(UpdateRequirement requirement) { + Preconditions.checkArgument(requirement != null, "Invalid requirement: null"); + requirements.add(requirement); + return this; + } + + private Builder update(MetadataUpdate update) { + Preconditions.checkArgument(update != null, "Invalid update: null"); + + // add requirements based on the change + if (update instanceof MetadataUpdate.SetSnapshotRef) { + update((MetadataUpdate.SetSnapshotRef) update); + } else if (update instanceof MetadataUpdate.AddSchema) { + update((MetadataUpdate.AddSchema) update); + } else if (update instanceof MetadataUpdate.SetCurrentSchema) { + update((MetadataUpdate.SetCurrentSchema) update); + } else if (update instanceof MetadataUpdate.AddPartitionSpec) { + update((MetadataUpdate.AddPartitionSpec) update); + } else if (update instanceof MetadataUpdate.SetDefaultPartitionSpec) { + update((MetadataUpdate.SetDefaultPartitionSpec) update); + } else if (update instanceof MetadataUpdate.SetDefaultSortOrder) { + update((MetadataUpdate.SetDefaultSortOrder) update); + } + + return this; + } + + private void update(MetadataUpdate.SetSnapshotRef setRef) { + // require that the ref is unchanged from the base + String name = setRef.name(); + // add returns true the first time the ref name is added + boolean added = changedRefs.add(name); + if (added && base != null && !isReplace) { + SnapshotRef baseRef = base.ref(name); + // require that the ref does not exist (null) or is the same as the base snapshot + require( + new UpdateRequirement.AssertRefSnapshotID( + name, baseRef != null ? baseRef.snapshotId() : null)); + } + } + + private void update(MetadataUpdate.AddSchema update) { + if (!addedSchema) { + if (base != null) { + require(new UpdateRequirement.AssertLastAssignedFieldId(base.lastColumnId())); + } + this.addedSchema = true; + } + } + + private void update(MetadataUpdate.SetCurrentSchema update) { + if (!setSchemaId) { + if (base != null && !isReplace) { + // require that the current schema has not changed + require(new UpdateRequirement.AssertCurrentSchemaID(base.currentSchemaId())); + } + this.setSchemaId = true; + } + } + + private void update(MetadataUpdate.AddPartitionSpec update) { + if (!addedSpec) { + if (base != null) { + require( + new UpdateRequirement.AssertLastAssignedPartitionId(base.lastAssignedPartitionId())); + } + this.addedSpec = true; + } + } + + private void update(MetadataUpdate.SetDefaultPartitionSpec update) { + if (!setSpecId) { + if (base != null && !isReplace) { + // require that the default spec has not changed + require(new UpdateRequirement.AssertDefaultSpecID(base.defaultSpecId())); + } + this.setSpecId = true; + } + } + + private void update(MetadataUpdate.SetDefaultSortOrder update) { + if (!setOrderId) { + if (base != null && !isReplace) { + // require that the default write order has not changed + require(new UpdateRequirement.AssertDefaultSortOrderID(base.defaultSortOrderId())); + } + this.setOrderId = true; + } + } + + private List build() { + return requirements.build(); + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/actions/BaseCommitService.java b/core/src/main/java/org/apache/iceberg/actions/BaseCommitService.java index 1f36f133d035..82390a25f58b 100644 --- a/core/src/main/java/org/apache/iceberg/actions/BaseCommitService.java +++ b/core/src/main/java/org/apache/iceberg/actions/BaseCommitService.java @@ -28,6 +28,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Queues; @@ -49,13 +50,16 @@ abstract class BaseCommitService implements Closeable { private static final Logger LOG = LoggerFactory.getLogger(BaseCommitService.class); + public static final long TIMEOUT_IN_MS_DEFAULT = TimeUnit.MINUTES.toMillis(120); + private final Table table; private final ExecutorService committerService; private final ConcurrentLinkedQueue completedRewrites; private final ConcurrentLinkedQueue inProgressCommits; - private final List committedRewrites; + private final ConcurrentLinkedQueue committedRewrites; private final int rewritesPerCommit; private final AtomicBoolean running = new AtomicBoolean(false); + private final long timeoutInMS; /** * Constructs a {@link BaseCommitService} @@ -64,17 +68,30 @@ abstract class BaseCommitService implements Closeable { * @param rewritesPerCommit number of file groups to include in a commit */ BaseCommitService(Table table, int rewritesPerCommit) { + this(table, rewritesPerCommit, TIMEOUT_IN_MS_DEFAULT); + } + + /** + * Constructs a {@link BaseCommitService} + * + * @param table table to perform commit on + * @param rewritesPerCommit number of file groups to include in a commit + * @param timeoutInMS The timeout to wait for commits to complete after all rewrite jobs have been + * completed + */ + BaseCommitService(Table table, int rewritesPerCommit, long timeoutInMS) { this.table = table; LOG.info( "Creating commit service for table {} with {} groups per commit", table, rewritesPerCommit); this.rewritesPerCommit = rewritesPerCommit; + this.timeoutInMS = timeoutInMS; committerService = Executors.newSingleThreadExecutor( new ThreadFactoryBuilder().setNameFormat("Committer-Service").build()); completedRewrites = Queues.newConcurrentLinkedQueue(); - committedRewrites = Lists.newArrayList(); + committedRewrites = Queues.newConcurrentLinkedQueue(); inProgressCommits = Queues.newConcurrentLinkedQueue(); } @@ -138,7 +155,7 @@ public List results() { Preconditions.checkState( committerService.isShutdown(), "Cannot get results from a service which has not been closed"); - return committedRewrites; + return Lists.newArrayList(committedRewrites.iterator()); } @Override @@ -154,11 +171,13 @@ public void close() { // the commit pool to finish doing its commits to Iceberg State. In the case of partial // progress this should have been occurring simultaneously with rewrites, if not there should // be only a single commit operation. - if (!committerService.awaitTermination(120, TimeUnit.MINUTES)) { + if (!committerService.awaitTermination(timeoutInMS, TimeUnit.MILLISECONDS)) { LOG.warn( - "Commit operation did not complete within 120 minutes of the all files " + "Commit operation did not complete within {} minutes ({} ms) of the all files " + "being rewritten. This may mean that some changes were not successfully committed to the " - + "table."); + + "table.", + TimeUnit.MILLISECONDS.toMinutes(timeoutInMS), + timeoutInMS); timeout = true; } } catch (InterruptedException e) { @@ -169,7 +188,11 @@ public void close() { if (!completedRewrites.isEmpty() && timeout) { LOG.error("Attempting to cleanup uncommitted file groups"); - completedRewrites.forEach(this::abortFileGroup); + synchronized (completedRewrites) { + while (!completedRewrites.isEmpty()) { + abortFileGroup(completedRewrites.poll()); + } + } } Preconditions.checkArgument( @@ -211,11 +234,17 @@ private void commitReadyCommitGroups() { } } - private boolean canCreateCommitGroup() { + @VisibleForTesting + boolean canCreateCommitGroup() { // Either we have a full commit group, or we have completed writing and need to commit // what is left over boolean fullCommitGroup = completedRewrites.size() >= rewritesPerCommit; boolean writingComplete = !running.get() && completedRewrites.size() > 0; return fullCommitGroup || writingComplete; } + + @VisibleForTesting + boolean completedRewritesAllCommitted() { + return completedRewrites.isEmpty() && inProgressCommits.isEmpty(); + } } diff --git a/core/src/main/java/org/apache/iceberg/actions/BaseDeleteOrphanFiles.java b/core/src/main/java/org/apache/iceberg/actions/BaseDeleteOrphanFiles.java new file mode 100644 index 000000000000..bd06ebe15d28 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/actions/BaseDeleteOrphanFiles.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.immutables.value.Value; +import org.immutables.value.Value.Style.BuilderVisibility; +import org.immutables.value.Value.Style.ImplementationVisibility; + +@Value.Enclosing +@SuppressWarnings("ImmutablesStyle") +@Value.Style( + typeImmutableEnclosing = "ImmutableDeleteOrphanFiles", + visibility = ImplementationVisibility.PUBLIC, + builderVisibility = BuilderVisibility.PUBLIC) +interface BaseDeleteOrphanFiles extends DeleteOrphanFiles { + + @Value.Immutable + interface Result extends DeleteOrphanFiles.Result {} +} diff --git a/core/src/main/java/org/apache/iceberg/actions/BaseDeleteReachableFiles.java b/core/src/main/java/org/apache/iceberg/actions/BaseDeleteReachableFiles.java new file mode 100644 index 000000000000..e3a1c3fb2317 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/actions/BaseDeleteReachableFiles.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.immutables.value.Value; +import org.immutables.value.Value.Style.BuilderVisibility; +import org.immutables.value.Value.Style.ImplementationVisibility; + +@Value.Enclosing +@SuppressWarnings("ImmutablesStyle") +@Value.Style( + typeImmutableEnclosing = "ImmutableDeleteReachableFiles", + visibility = ImplementationVisibility.PUBLIC, + builderVisibility = BuilderVisibility.PUBLIC) +interface BaseDeleteReachableFiles extends DeleteReachableFiles { + + @Value.Immutable + interface Result extends DeleteReachableFiles.Result {} +} diff --git a/core/src/main/java/org/apache/iceberg/actions/BaseExpireSnapshots.java b/core/src/main/java/org/apache/iceberg/actions/BaseExpireSnapshots.java new file mode 100644 index 000000000000..646e31a64cb6 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/actions/BaseExpireSnapshots.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.actions; + +import org.immutables.value.Value; +import org.immutables.value.Value.Style.BuilderVisibility; +import org.immutables.value.Value.Style.ImplementationVisibility; + +@Value.Enclosing +@SuppressWarnings("ImmutablesStyle") +@Value.Style( + typeImmutableEnclosing = "ImmutableExpireSnapshots", + visibility = ImplementationVisibility.PUBLIC, + builderVisibility = BuilderVisibility.PUBLIC) +interface BaseExpireSnapshots extends ExpireSnapshots { + + @Value.Immutable + interface Result extends ExpireSnapshots.Result { + @Override + @Value.Default + default long deletedStatisticsFilesCount() { + return ExpireSnapshots.Result.super.deletedStatisticsFilesCount(); + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/actions/BaseMigrateTable.java b/core/src/main/java/org/apache/iceberg/actions/BaseMigrateTable.java new file mode 100644 index 000000000000..4fc271761968 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/actions/BaseMigrateTable.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.immutables.value.Value; +import org.immutables.value.Value.Style.BuilderVisibility; +import org.immutables.value.Value.Style.ImplementationVisibility; + +@Value.Enclosing +@SuppressWarnings("ImmutablesStyle") +@Value.Style( + typeImmutableEnclosing = "ImmutableMigrateTable", + visibility = ImplementationVisibility.PUBLIC, + builderVisibility = BuilderVisibility.PUBLIC) +interface BaseMigrateTable extends MigrateTable { + + @Value.Immutable + interface Result extends MigrateTable.Result {} +} diff --git a/core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFiles.java b/core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFiles.java new file mode 100644 index 000000000000..498f538e7c72 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFiles.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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 java.util.List; +import org.immutables.value.Value; +import org.immutables.value.Value.Style.BuilderVisibility; +import org.immutables.value.Value.Style.ImplementationVisibility; + +@Value.Enclosing +@SuppressWarnings("ImmutablesStyle") +@Value.Style( + typeImmutableEnclosing = "ImmutableRewriteDataFiles", + visibility = ImplementationVisibility.PUBLIC, + builderVisibility = BuilderVisibility.PUBLIC) +interface BaseRewriteDataFiles extends RewriteDataFiles { + + @Value.Immutable + interface Result extends RewriteDataFiles.Result { + @Override + @Value.Default + default List rewriteFailures() { + return RewriteDataFiles.Result.super.rewriteFailures(); + } + + @Override + @Value.Default + default int addedDataFilesCount() { + return RewriteDataFiles.Result.super.addedDataFilesCount(); + } + + @Override + @Value.Default + default int rewrittenDataFilesCount() { + return RewriteDataFiles.Result.super.rewrittenDataFilesCount(); + } + + @Override + @Value.Default + default long rewrittenBytesCount() { + return RewriteDataFiles.Result.super.rewrittenBytesCount(); + } + + @Override + @Value.Default + default int failedDataFilesCount() { + return RewriteDataFiles.Result.super.failedDataFilesCount(); + } + } + + @Value.Immutable + interface FileGroupRewriteResult extends RewriteDataFiles.FileGroupRewriteResult { + @Override + @Value.Default + default long rewrittenBytesCount() { + return RewriteDataFiles.FileGroupRewriteResult.super.rewrittenBytesCount(); + } + } + + @Value.Immutable + interface FileGroupFailureResult extends RewriteDataFiles.FileGroupFailureResult {} + + @Value.Immutable + interface FileGroupInfo extends RewriteDataFiles.FileGroupInfo {} +} diff --git a/core/src/main/java/org/apache/iceberg/actions/BaseRewriteManifests.java b/core/src/main/java/org/apache/iceberg/actions/BaseRewriteManifests.java new file mode 100644 index 000000000000..63e1e4d450d7 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/actions/BaseRewriteManifests.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.immutables.value.Value; +import org.immutables.value.Value.Style.BuilderVisibility; +import org.immutables.value.Value.Style.ImplementationVisibility; + +@Value.Enclosing +@SuppressWarnings("ImmutablesStyle") +@Value.Style( + typeImmutableEnclosing = "ImmutableRewriteManifests", + visibility = ImplementationVisibility.PUBLIC, + builderVisibility = BuilderVisibility.PUBLIC) +interface BaseRewriteManifests extends RewriteManifests { + + @Value.Immutable + interface Result extends RewriteManifests.Result {} +} diff --git a/core/src/main/java/org/apache/iceberg/actions/BaseRewritePositionalDeleteFiles.java b/core/src/main/java/org/apache/iceberg/actions/BaseRewritePositionalDeleteFiles.java new file mode 100644 index 000000000000..6732ef6c0f6a --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/actions/BaseRewritePositionalDeleteFiles.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.actions; + +import org.immutables.value.Value; +import org.immutables.value.Value.Style.BuilderVisibility; +import org.immutables.value.Value.Style.ImplementationVisibility; + +@Value.Enclosing +@SuppressWarnings("ImmutablesStyle") +@Value.Style( + typeImmutableEnclosing = "ImmutableRewritePositionDeleteFiles", + visibility = ImplementationVisibility.PUBLIC, + builderVisibility = BuilderVisibility.PUBLIC) +interface BaseRewritePositionalDeleteFiles extends RewritePositionDeleteFiles { + + @Value.Immutable + interface Result extends RewritePositionDeleteFiles.Result { + @Override + @Value.Default + default int rewrittenDeleteFilesCount() { + return RewritePositionDeleteFiles.Result.super.rewrittenDeleteFilesCount(); + } + + @Override + @Value.Default + default int addedDeleteFilesCount() { + return RewritePositionDeleteFiles.Result.super.addedDeleteFilesCount(); + } + + @Override + @Value.Default + default long rewrittenBytesCount() { + return RewritePositionDeleteFiles.Result.super.rewrittenBytesCount(); + } + + @Override + @Value.Default + default long addedBytesCount() { + return RewritePositionDeleteFiles.Result.super.addedBytesCount(); + } + } + + @Value.Immutable + interface FileGroupRewriteResult extends RewritePositionDeleteFiles.FileGroupRewriteResult {} + + @Value.Immutable + interface FileGroupInfo extends RewritePositionDeleteFiles.FileGroupInfo {} +} diff --git a/core/src/main/java/org/apache/iceberg/actions/BaseSnapshotTable.java b/core/src/main/java/org/apache/iceberg/actions/BaseSnapshotTable.java new file mode 100644 index 000000000000..fb8fbaca3d6e --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/actions/BaseSnapshotTable.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.immutables.value.Value; +import org.immutables.value.Value.Style.BuilderVisibility; +import org.immutables.value.Value.Style.ImplementationVisibility; + +@Value.Enclosing +@SuppressWarnings("ImmutablesStyle") +@Value.Style( + typeImmutableEnclosing = "ImmutableSnapshotTable", + visibility = ImplementationVisibility.PUBLIC, + builderVisibility = BuilderVisibility.PUBLIC) +interface BaseSnapshotTable extends SnapshotTable { + + @Value.Immutable + interface Result extends SnapshotTable.Result {} +} diff --git a/core/src/main/java/org/apache/iceberg/avro/GenericAvroReader.java b/core/src/main/java/org/apache/iceberg/avro/GenericAvroReader.java index c0a7895acbba..0fa2e795811b 100644 --- a/core/src/main/java/org/apache/iceberg/avro/GenericAvroReader.java +++ b/core/src/main/java/org/apache/iceberg/avro/GenericAvroReader.java @@ -30,13 +30,17 @@ import org.apache.iceberg.common.DynClasses; import org.apache.iceberg.data.avro.DecoderResolver; -class GenericAvroReader implements DatumReader, SupportsRowPosition { +public class GenericAvroReader implements DatumReader, SupportsRowPosition { private final Schema readSchema; private ClassLoader loader = Thread.currentThread().getContextClassLoader(); private Schema fileSchema = null; private ValueReader reader = null; + public static GenericAvroReader create(Schema schema) { + return new GenericAvroReader<>(schema); + } + GenericAvroReader(Schema readSchema) { this.readSchema = readSchema; } diff --git a/core/src/main/java/org/apache/iceberg/avro/GenericAvroWriter.java b/core/src/main/java/org/apache/iceberg/avro/GenericAvroWriter.java index d6a4574dab66..421bfc9dc462 100644 --- a/core/src/main/java/org/apache/iceberg/avro/GenericAvroWriter.java +++ b/core/src/main/java/org/apache/iceberg/avro/GenericAvroWriter.java @@ -28,9 +28,13 @@ import org.apache.iceberg.FieldMetrics; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -class GenericAvroWriter implements MetricsAwareDatumWriter { +public class GenericAvroWriter implements MetricsAwareDatumWriter { private ValueWriter writer = null; + public static GenericAvroWriter create(Schema schema) { + return new GenericAvroWriter<>(schema); + } + GenericAvroWriter(Schema schema) { setSchema(schema); } diff --git a/core/src/main/java/org/apache/iceberg/catalog/TableCommit.java b/core/src/main/java/org/apache/iceberg/catalog/TableCommit.java new file mode 100644 index 000000000000..26f67a012b8a --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/catalog/TableCommit.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.catalog; + +import java.util.List; +import org.apache.iceberg.MetadataUpdate; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.UpdateRequirement; +import org.apache.iceberg.UpdateRequirements; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.immutables.value.Value; + +/** + * This represents a commit to be applied for a single table with {@link UpdateRequirement}s to be + * validated and {@link MetadataUpdate}s that have been applied. The {@link UpdateRequirement}s and + * {@link MetadataUpdate}s can be derived from table's base and updated {@link TableMetadata} when + * using {@link TableCommit#create(TableIdentifier, TableMetadata, TableMetadata)}. + */ +@Value.Immutable +public interface TableCommit { + TableIdentifier identifier(); + + List requirements(); + + List updates(); + + /** + * This creates a {@link TableCommit} instance to be applied for a single table with {@link + * UpdateRequirement}s to be validated and {@link MetadataUpdate}s that have been applied. + * + * @param identifier The {@link TableIdentifier} to create the {@link TableCommit} for. + * @param base The base {@link TableMetadata} where {@link UpdateRequirement}s are derived from + * and used for validation. + * @param updated The updated {@link TableMetadata} where {@link MetadataUpdate}s that have been + * applied are derived from. + * @return A {@link TableCommit} instance to be applied for a single table + */ + static TableCommit create(TableIdentifier identifier, TableMetadata base, TableMetadata updated) { + Preconditions.checkArgument(null != identifier, "Invalid table identifier: null"); + Preconditions.checkArgument(null != base && null != updated, "Invalid table metadata: null"); + Preconditions.checkArgument( + base.uuid().equals(updated.uuid()), + "UUID of base (%s) and updated (%s) table metadata does not match", + base.uuid(), + updated.uuid()); + + return ImmutableTableCommit.builder() + .identifier(identifier) + .requirements(UpdateRequirements.forUpdateTable(base, updated.changes())) + .updates(updated.changes()) + .build(); + } +} diff --git a/core/src/main/java/org/apache/iceberg/data/avro/DecoderResolver.java b/core/src/main/java/org/apache/iceberg/data/avro/DecoderResolver.java index 97b90755d9f9..3cf3d1bcde2f 100644 --- a/core/src/main/java/org/apache/iceberg/data/avro/DecoderResolver.java +++ b/core/src/main/java/org/apache/iceberg/data/avro/DecoderResolver.java @@ -20,14 +20,15 @@ import java.io.IOException; import java.util.Map; +import java.util.WeakHashMap; import org.apache.avro.Schema; import org.apache.avro.io.Decoder; import org.apache.avro.io.DecoderFactory; import org.apache.avro.io.ResolvingDecoder; import org.apache.iceberg.avro.ValueReader; import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.collect.MapMaker; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; /** * Resolver to resolve {@link Decoder} to a {@link ResolvingDecoder}. This class uses a {@link @@ -35,7 +36,8 @@ */ public class DecoderResolver { - private static final ThreadLocal>> DECODER_CACHES = + @VisibleForTesting + static final ThreadLocal>> DECODER_CACHES = ThreadLocal.withInitial(() -> new MapMaker().weakKeys().makeMap()); private DecoderResolver() {} @@ -49,11 +51,12 @@ public static T resolveAndRead( return value; } - private static ResolvingDecoder resolve(Decoder decoder, Schema readSchema, Schema fileSchema) + @VisibleForTesting + static ResolvingDecoder resolve(Decoder decoder, Schema readSchema, Schema fileSchema) throws IOException { Map> cache = DECODER_CACHES.get(); Map fileSchemaToResolver = - cache.computeIfAbsent(readSchema, k -> Maps.newHashMap()); + cache.computeIfAbsent(readSchema, k -> new WeakHashMap<>()); ResolvingDecoder resolver = fileSchemaToResolver.computeIfAbsent(fileSchema, schema -> newResolver(readSchema, schema)); diff --git a/core/src/main/java/org/apache/iceberg/data/avro/IcebergDecoder.java b/core/src/main/java/org/apache/iceberg/data/avro/IcebergDecoder.java index 7eca98a1cfe7..94fedbcfce91 100644 --- a/core/src/main/java/org/apache/iceberg/data/avro/IcebergDecoder.java +++ b/core/src/main/java/org/apache/iceberg/data/avro/IcebergDecoder.java @@ -20,22 +20,18 @@ import java.io.IOException; import java.io.InputStream; +import java.io.UncheckedIOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.util.Map; import org.apache.avro.AvroRuntimeException; import org.apache.avro.Schema; import org.apache.avro.SchemaNormalization; -import org.apache.avro.io.BinaryDecoder; -import org.apache.avro.io.DatumReader; -import org.apache.avro.io.DecoderFactory; import org.apache.avro.message.BadHeaderException; import org.apache.avro.message.MessageDecoder; import org.apache.avro.message.MissingSchemaException; import org.apache.avro.message.SchemaStore; import org.apache.iceberg.avro.AvroSchemaUtil; -import org.apache.iceberg.avro.ProjectionDatumReader; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.MapMaker; public class IcebergDecoder extends MessageDecoder.BaseDecoder { @@ -106,7 +102,10 @@ public void addSchema(org.apache.iceberg.Schema writeSchema) { private void addSchema(Schema writeSchema) { long fp = SchemaNormalization.parsingFingerprint64(writeSchema); - decoders.put(fp, new RawDecoder<>(readSchema, writeSchema)); + RawDecoder decoder = + new RawDecoder<>( + readSchema, avroSchema -> DataReader.create(readSchema, avroSchema), writeSchema); + decoders.put(fp, decoder); } private RawDecoder getDecoder(long fp) { @@ -144,44 +143,10 @@ public D decode(InputStream stream, D reuse) throws IOException { RawDecoder decoder = getDecoder(FP_BUFFER.get().getLong(2)); - return decoder.decode(stream, reuse); - } - - private static class RawDecoder extends MessageDecoder.BaseDecoder { - private static final ThreadLocal DECODER = new ThreadLocal<>(); - - private final DatumReader reader; - - /** - * Creates a new {@link MessageDecoder} that constructs datum instances described by the {@link - * Schema readSchema}. - * - *

The {@code readSchema} is used for the expected schema and the {@code writeSchema} is the - * schema used to decode buffers. The {@code writeSchema} must be the schema that was used to - * encode all buffers decoded by this class. - * - * @param readSchema the schema used to construct datum instances - * @param writeSchema the schema used to decode buffers - */ - private RawDecoder(org.apache.iceberg.Schema readSchema, Schema writeSchema) { - this.reader = - new ProjectionDatumReader<>( - avroSchema -> DataReader.create(readSchema, avroSchema), - readSchema, - ImmutableMap.of(), - null); - this.reader.setSchema(writeSchema); - } - - @Override - public D decode(InputStream stream, D reuse) { - BinaryDecoder decoder = DecoderFactory.get().directBinaryDecoder(stream, DECODER.get()); - DECODER.set(decoder); - try { - return reader.read(reuse, decoder); - } catch (IOException e) { - throw new AvroRuntimeException("Decoding datum failed", e); - } + try { + return decoder.decode(stream, reuse); + } catch (UncheckedIOException e) { + throw new AvroRuntimeException(e); } } diff --git a/core/src/main/java/org/apache/iceberg/data/avro/RawDecoder.java b/core/src/main/java/org/apache/iceberg/data/avro/RawDecoder.java new file mode 100644 index 000000000000..c32ea707bfab --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/data/avro/RawDecoder.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.data.avro; + +import java.io.IOException; +import java.io.InputStream; +import java.io.UncheckedIOException; +import java.util.function.Function; +import org.apache.avro.Schema; +import org.apache.avro.io.BinaryDecoder; +import org.apache.avro.io.DatumReader; +import org.apache.avro.io.DecoderFactory; +import org.apache.avro.message.MessageDecoder; +import org.apache.iceberg.avro.ProjectionDatumReader; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; + +public class RawDecoder extends MessageDecoder.BaseDecoder { + private static final ThreadLocal DECODER = new ThreadLocal<>(); + + private final DatumReader reader; + + /** + * Creates a new {@link MessageDecoder} that constructs datum instances described by the {@link + * Schema readSchema}. + * + *

The {@code readSchema} is used for the expected schema and the {@code writeSchema} is the + * schema used to decode buffers. The {@code writeSchema} must be the schema that was used to + * encode all buffers decoded by this class. + */ + public RawDecoder( + org.apache.iceberg.Schema readSchema, + Function> readerFunction, + Schema writeSchema) { + this.reader = new ProjectionDatumReader<>(readerFunction, readSchema, ImmutableMap.of(), null); + this.reader.setSchema(writeSchema); + } + + @Override + public D decode(InputStream stream, D reuse) { + BinaryDecoder decoder = DecoderFactory.get().directBinaryDecoder(stream, DECODER.get()); + DECODER.set(decoder); + try { + return reader.read(reuse, decoder); + } catch (IOException e) { + throw new UncheckedIOException("Decoding datum failed", e); + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/encryption/KeyMetadata.java b/core/src/main/java/org/apache/iceberg/encryption/KeyMetadata.java new file mode 100644 index 000000000000..0d7ec43f6ebc --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/encryption/KeyMetadata.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.encryption; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.ByteBuffer; +import java.util.Map; +import org.apache.avro.generic.IndexedRecord; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; + +class KeyMetadata implements EncryptionKeyMetadata, IndexedRecord { + private static final byte V1 = 1; + private static final Schema SCHEMA_V1 = + new Schema( + required(0, "encryption_key", Types.BinaryType.get()), + optional(1, "aad_prefix", Types.BinaryType.get())); + private static final org.apache.avro.Schema AVRO_SCHEMA_V1 = + AvroSchemaUtil.convert(SCHEMA_V1, KeyMetadata.class.getCanonicalName()); + + private static final Map schemaVersions = ImmutableMap.of(V1, SCHEMA_V1); + private static final Map avroSchemaVersions = + ImmutableMap.of(V1, AVRO_SCHEMA_V1); + + private static final KeyMetadataEncoder KEY_METADATA_ENCODER = new KeyMetadataEncoder(V1); + private static final KeyMetadataDecoder KEY_METADATA_DECODER = new KeyMetadataDecoder(V1); + + private ByteBuffer encryptionKey; + private ByteBuffer aadPrefix; + private org.apache.avro.Schema avroSchema; + + /** Used by Avro reflection to instantiate this class * */ + KeyMetadata() {} + + KeyMetadata(ByteBuffer encryptionKey, ByteBuffer aadPrefix) { + this.encryptionKey = encryptionKey; + this.aadPrefix = aadPrefix; + this.avroSchema = AVRO_SCHEMA_V1; + } + + static Map supportedSchemaVersions() { + return schemaVersions; + } + + static Map supportedAvroSchemaVersions() { + return avroSchemaVersions; + } + + ByteBuffer encryptionKey() { + return encryptionKey; + } + + ByteBuffer aadPrefix() { + return aadPrefix; + } + + static KeyMetadata parse(ByteBuffer buffer) { + try { + return KEY_METADATA_DECODER.decode(buffer); + } catch (IOException e) { + throw new UncheckedIOException("Failed to parse envelope encryption metadata", e); + } + } + + @Override + public ByteBuffer buffer() { + try { + return KEY_METADATA_ENCODER.encode(this); + } catch (IOException e) { + throw new UncheckedIOException("Failed to serialize envelope key metadata", e); + } + } + + @Override + public EncryptionKeyMetadata copy() { + KeyMetadata metadata = new KeyMetadata(encryptionKey(), aadPrefix()); + return metadata; + } + + @Override + public void put(int i, Object v) { + switch (i) { + case 0: + this.encryptionKey = (ByteBuffer) v; + return; + case 1: + this.aadPrefix = (ByteBuffer) v; + return; + default: + // ignore the object, it must be from a newer version of the format + } + } + + @Override + public Object get(int i) { + switch (i) { + case 0: + return encryptionKey; + case 1: + return aadPrefix; + default: + throw new UnsupportedOperationException("Unknown field ordinal: " + i); + } + } + + @Override + public org.apache.avro.Schema getSchema() { + return avroSchema; + } +} diff --git a/core/src/main/java/org/apache/iceberg/encryption/KeyMetadataDecoder.java b/core/src/main/java/org/apache/iceberg/encryption/KeyMetadataDecoder.java new file mode 100644 index 000000000000..674685c30164 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/encryption/KeyMetadataDecoder.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.encryption; + +import java.io.IOException; +import java.io.InputStream; +import java.io.UncheckedIOException; +import java.util.Map; +import org.apache.avro.Schema; +import org.apache.avro.message.MessageDecoder; +import org.apache.iceberg.avro.GenericAvroReader; +import org.apache.iceberg.data.avro.RawDecoder; +import org.apache.iceberg.relocated.com.google.common.collect.MapMaker; + +class KeyMetadataDecoder extends MessageDecoder.BaseDecoder { + private final org.apache.iceberg.Schema readSchema; + private final Map> decoders = new MapMaker().makeMap(); + + /** + * Creates a new decoder that constructs key metadata instances described by schema version. + * + *

The {@code readSchemaVersion} is as used the version of the expected (read) schema. Datum + * instances created by this class will are described by the expected schema. + */ + KeyMetadataDecoder(byte readSchemaVersion) { + this.readSchema = KeyMetadata.supportedSchemaVersions().get(readSchemaVersion); + } + + @Override + public KeyMetadata decode(InputStream stream, KeyMetadata reuse) { + byte writeSchemaVersion; + + try { + writeSchemaVersion = (byte) stream.read(); + } catch (IOException e) { + throw new UncheckedIOException("Failed to read the version byte", e); + } + + if (writeSchemaVersion < 0) { + throw new RuntimeException("Version byte - end of stream reached"); + } + + Schema writeSchema = KeyMetadata.supportedAvroSchemaVersions().get(writeSchemaVersion); + + if (writeSchema == null) { + throw new UnsupportedOperationException( + "Cannot resolve schema for version: " + writeSchemaVersion); + } + + RawDecoder decoder = decoders.get(writeSchemaVersion); + + if (decoder == null) { + decoder = new RawDecoder<>(readSchema, GenericAvroReader::create, writeSchema); + + decoders.put(writeSchemaVersion, decoder); + } + + return decoder.decode(stream, reuse); + } +} diff --git a/core/src/main/java/org/apache/iceberg/encryption/KeyMetadataEncoder.java b/core/src/main/java/org/apache/iceberg/encryption/KeyMetadataEncoder.java new file mode 100644 index 000000000000..faab6a47c814 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/encryption/KeyMetadataEncoder.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.encryption; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import org.apache.avro.Schema; +import org.apache.avro.io.BinaryEncoder; +import org.apache.avro.io.DatumWriter; +import org.apache.avro.io.EncoderFactory; +import org.apache.avro.message.MessageEncoder; +import org.apache.iceberg.avro.GenericAvroWriter; + +class KeyMetadataEncoder implements MessageEncoder { + private static final ThreadLocal TEMP = + ThreadLocal.withInitial(BufferOutputStream::new); + private static final ThreadLocal ENCODER = new ThreadLocal<>(); + + private final byte schemaVersion; + private final boolean copyOutputBytes; + private final DatumWriter writer; + + /** + * Creates a new {@link MessageEncoder} that will deconstruct {@link KeyMetadata} instances + * described by the schema version. + * + *

Buffers returned by {@code encode} are copied and will not be modified by future calls to + * {@code encode}. + */ + KeyMetadataEncoder(byte schemaVersion) { + this(schemaVersion, true); + } + + /** + * Creates a new {@link MessageEncoder} that will deconstruct {@link KeyMetadata} instances + * described by the schema version. + * + *

If {@code shouldCopy} is true, then buffers returned by {@code encode} are copied and will + * not be modified by future calls to {@code encode}. + * + *

If {@code shouldCopy} is false, then buffers returned by {@code encode} wrap a thread-local + * buffer that can be reused by future calls to {@code encode}, but may not be. Callers should + * only set {@code shouldCopy} to false if the buffer will be copied before the current thread's + * next call to {@code encode}. + */ + KeyMetadataEncoder(byte schemaVersion, boolean shouldCopy) { + Schema writeSchema = KeyMetadata.supportedAvroSchemaVersions().get(schemaVersion); + + if (writeSchema == null) { + throw new UnsupportedOperationException( + "Cannot resolve schema for version: " + schemaVersion); + } + + this.writer = GenericAvroWriter.create(writeSchema); + this.schemaVersion = schemaVersion; + this.copyOutputBytes = shouldCopy; + } + + @Override + public ByteBuffer encode(KeyMetadata datum) throws IOException { + BufferOutputStream temp = TEMP.get(); + temp.reset(); + temp.write(schemaVersion); + encode(datum, temp); + + if (copyOutputBytes) { + return temp.toBufferWithCopy(); + } else { + return temp.toBufferWithoutCopy(); + } + } + + @Override + public void encode(KeyMetadata datum, OutputStream stream) throws IOException { + BinaryEncoder encoder = EncoderFactory.get().directBinaryEncoder(stream, ENCODER.get()); + ENCODER.set(encoder); + writer.write(datum, encoder); + encoder.flush(); + } + + private static class BufferOutputStream extends ByteArrayOutputStream { + BufferOutputStream() {} + + ByteBuffer toBufferWithoutCopy() { + return ByteBuffer.wrap(buf, 0, count); + } + + ByteBuffer toBufferWithCopy() { + return ByteBuffer.wrap(toByteArray()); + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/inmemory/InMemoryCatalog.java b/core/src/main/java/org/apache/iceberg/inmemory/InMemoryCatalog.java index e9f25c8c84ff..3956e9192aaf 100644 --- a/core/src/main/java/org/apache/iceberg/inmemory/InMemoryCatalog.java +++ b/core/src/main/java/org/apache/iceberg/inmemory/InMemoryCatalog.java @@ -45,7 +45,6 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.base.Joiner; 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.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @@ -137,35 +136,27 @@ public List listTables(Namespace namespace) { } @Override - public void renameTable(TableIdentifier fromTableIdentifier, TableIdentifier toTableIdentifier) { - if (fromTableIdentifier.equals(toTableIdentifier)) { + public synchronized void renameTable(TableIdentifier from, TableIdentifier to) { + if (from.equals(to)) { return; } - if (!namespaceExists(toTableIdentifier.namespace())) { + if (!namespaceExists(to.namespace())) { throw new NoSuchNamespaceException( - "Cannot rename %s to %s. Namespace does not exist: %s", - fromTableIdentifier, toTableIdentifier, toTableIdentifier.namespace()); + "Cannot rename %s to %s. Namespace does not exist: %s", from, to, to.namespace()); } - if (!tables.containsKey(fromTableIdentifier)) { - throw new NoSuchTableException( - "Cannot rename %s to %s. Table does not exist", fromTableIdentifier, toTableIdentifier); + String fromLocation = tables.get(from); + if (null == fromLocation) { + throw new NoSuchTableException("Cannot rename %s to %s. Table does not exist", from, to); } - if (tables.containsKey(toTableIdentifier)) { - throw new AlreadyExistsException( - "Cannot rename %s to %s. Table already exists", fromTableIdentifier, toTableIdentifier); + if (tables.containsKey(to)) { + throw new AlreadyExistsException("Cannot rename %s to %s. Table already exists", from, to); } - String fromLocation = tables.remove(fromTableIdentifier); - Preconditions.checkState( - null != fromLocation, - "Cannot rename from %s to %s. Source table does not exist", - fromTableIdentifier, - toTableIdentifier); - - tables.put(toTableIdentifier, fromLocation); + tables.put(to, fromLocation); + tables.remove(from); } @Override diff --git a/core/src/main/java/org/apache/iceberg/io/ClusteredWriter.java b/core/src/main/java/org/apache/iceberg/io/ClusteredWriter.java index ee9e35eb32cb..12ceda1bfd4f 100644 --- a/core/src/main/java/org/apache/iceberg/io/ClusteredWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/ClusteredWriter.java @@ -132,13 +132,17 @@ public final R result() { return aggregatedResult(); } + /** @deprecated will be removed in 1.5.0 */ + @Deprecated protected EncryptedOutputFile newOutputFile( OutputFileFactory fileFactory, PartitionSpec spec, StructLike partition) { Preconditions.checkArgument( spec.isUnpartitioned() || partition != null, "Partition must not be null when creating output file for partitioned spec"); - return partition == null - ? fileFactory.newOutputFile() - : fileFactory.newOutputFile(spec, partition); + if (spec.isUnpartitioned() || partition == null) { + return fileFactory.newOutputFile(); + } else { + return fileFactory.newOutputFile(spec, partition); + } } } diff --git a/core/src/main/java/org/apache/iceberg/io/FanoutWriter.java b/core/src/main/java/org/apache/iceberg/io/FanoutWriter.java index ca19919bb1bd..9b88e4011cf1 100644 --- a/core/src/main/java/org/apache/iceberg/io/FanoutWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/FanoutWriter.java @@ -95,13 +95,17 @@ public final R result() { return aggregatedResult(); } + /** @deprecated will be removed in 1.5.0 */ + @Deprecated protected EncryptedOutputFile newOutputFile( OutputFileFactory fileFactory, PartitionSpec spec, StructLike partition) { Preconditions.checkArgument( spec.isUnpartitioned() || partition != null, "Partition must not be null when creating output file for partitioned spec"); - return partition == null - ? fileFactory.newOutputFile() - : fileFactory.newOutputFile(spec, partition); + if (spec.isUnpartitioned() || partition == null) { + return fileFactory.newOutputFile(); + } else { + return fileFactory.newOutputFile(spec, partition); + } } } diff --git a/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java b/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java index f7369224ee29..28d07bcf249e 100644 --- a/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java +++ b/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java @@ -42,11 +42,13 @@ public class ResolvingFileIO implements FileIO, HadoopConfigurable { private static final Logger LOG = LoggerFactory.getLogger(ResolvingFileIO.class); private static final String FALLBACK_IMPL = "org.apache.iceberg.hadoop.HadoopFileIO"; private static final String S3_FILE_IO_IMPL = "org.apache.iceberg.aws.s3.S3FileIO"; + private static final String GCS_FILE_IO_IMPL = "org.apache.iceberg.gcp.gcs.GCSFileIO"; private static final Map SCHEME_TO_FILE_IO = ImmutableMap.of( "s3", S3_FILE_IO_IMPL, "s3a", S3_FILE_IO_IMPL, - "s3n", S3_FILE_IO_IMPL); + "s3n", S3_FILE_IO_IMPL, + "gs", GCS_FILE_IO_IMPL); private final Map ioInstances = Maps.newHashMap(); private final AtomicBoolean isClosed = new AtomicBoolean(false); diff --git a/core/src/main/java/org/apache/iceberg/io/RollingFileWriter.java b/core/src/main/java/org/apache/iceberg/io/RollingFileWriter.java index 791d6d1bb201..7232aded44b9 100644 --- a/core/src/main/java/org/apache/iceberg/io/RollingFileWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/RollingFileWriter.java @@ -109,7 +109,7 @@ protected void openCurrentWriter() { } private EncryptedOutputFile newFile() { - if (partition == null) { + if (spec.isUnpartitioned() || partition == null) { return fileFactory.newOutputFile(); } else { return fileFactory.newOutputFile(spec, partition); diff --git a/core/src/main/java/org/apache/iceberg/metrics/InMemoryMetricsReporter.java b/core/src/main/java/org/apache/iceberg/metrics/InMemoryMetricsReporter.java new file mode 100644 index 000000000000..79b446c0ddbf --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/metrics/InMemoryMetricsReporter.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.metrics; + +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +public class InMemoryMetricsReporter implements MetricsReporter { + + private MetricsReport metricsReport; + + @Override + public void report(MetricsReport report) { + this.metricsReport = report; + } + + public ScanReport scanReport() { + Preconditions.checkArgument( + metricsReport == null || metricsReport instanceof ScanReport, + "Metrics report is not a scan report"); + return (ScanReport) metricsReport; + } +} diff --git a/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java b/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java index d297fc738317..1e0ef660271b 100644 --- a/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java +++ b/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java @@ -40,6 +40,7 @@ import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableOperations; import org.apache.iceberg.Transaction; +import org.apache.iceberg.UpdateRequirement; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.SupportsNamespaces; @@ -53,6 +54,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.rest.requests.CreateNamespaceRequest; import org.apache.iceberg.rest.requests.CreateTableRequest; +import org.apache.iceberg.rest.requests.RegisterTableRequest; import org.apache.iceberg.rest.requests.RenameTableRequest; import org.apache.iceberg.rest.requests.UpdateNamespacePropertiesRequest; import org.apache.iceberg.rest.requests.UpdateTableRequest; @@ -222,6 +224,21 @@ public static LoadTableResponse createTable( throw new IllegalStateException("Cannot wrap catalog that does not produce BaseTable"); } + public static LoadTableResponse registerTable( + Catalog catalog, Namespace namespace, RegisterTableRequest request) { + request.validate(); + + TableIdentifier identifier = TableIdentifier.of(namespace, request.name()); + Table table = catalog.registerTable(identifier, request.metadataLocation()); + if (table instanceof BaseTable) { + return LoadTableResponse.builder() + .withTableMetadata(((BaseTable) table).operations().current()) + .build(); + } + + throw new IllegalStateException("Cannot wrap catalog that does not produce BaseTable"); + } + public static void dropTable(Catalog catalog, TableIdentifier ident) { boolean dropped = catalog.dropTable(ident, false); if (!dropped) { @@ -286,16 +303,12 @@ public static void renameTable(Catalog catalog, RenameTableRequest request) { private static boolean isCreate(UpdateTableRequest request) { boolean isCreate = request.requirements().stream() - .anyMatch( - UpdateTableRequest.UpdateRequirement.AssertTableDoesNotExist.class::isInstance); + .anyMatch(UpdateRequirement.AssertTableDoesNotExist.class::isInstance); if (isCreate) { - List invalidRequirements = + List invalidRequirements = request.requirements().stream() - .filter( - req -> - !(req - instanceof UpdateTableRequest.UpdateRequirement.AssertTableDoesNotExist)) + .filter(req -> !(req instanceof UpdateRequirement.AssertTableDoesNotExist)) .collect(Collectors.toList()); Preconditions.checkArgument( invalidRequirements.isEmpty(), "Invalid create requirements: %s", invalidRequirements); @@ -317,7 +330,7 @@ private static TableMetadata create(TableOperations ops, UpdateTableRequest requ return ops.current(); } - private static TableMetadata commit(TableOperations ops, UpdateTableRequest request) { + static TableMetadata commit(TableOperations ops, UpdateTableRequest request) { AtomicBoolean isRetry = new AtomicBoolean(false); try { Tasks.foreach(ops) diff --git a/core/src/main/java/org/apache/iceberg/rest/HTTPClient.java b/core/src/main/java/org/apache/iceberg/rest/HTTPClient.java index 9af4b56797db..b2b6fc8a7c59 100644 --- a/core/src/main/java/org/apache/iceberg/rest/HTTPClient.java +++ b/core/src/main/java/org/apache/iceberg/rest/HTTPClient.java @@ -24,6 +24,7 @@ import java.io.UncheckedIOException; import java.net.URI; import java.net.URISyntaxException; +import java.nio.charset.StandardCharsets; import java.util.Map; import java.util.function.Consumer; import java.util.stream.Collectors; @@ -105,7 +106,7 @@ private static String extractResponseBodyAsString(CloseableHttpResponse response } // EntityUtils.toString returns null when HttpEntity.getContent returns null. - return EntityUtils.toString(response.getEntity(), "UTF-8"); + return EntityUtils.toString(response.getEntity(), StandardCharsets.UTF_8); } catch (IOException | ParseException e) { throw new RESTException(e, "Failed to convert HTTP response body to string"); } @@ -471,13 +472,13 @@ public HTTPClient build() { private StringEntity toJson(Object requestBody) { try { - return new StringEntity(mapper.writeValueAsString(requestBody)); + return new StringEntity(mapper.writeValueAsString(requestBody), StandardCharsets.UTF_8); } catch (JsonProcessingException e) { throw new RESTException(e, "Failed to write request body: %s", requestBody); } } private StringEntity toFormEncoding(Map formData) { - return new StringEntity(RESTUtil.encodeFormData(formData)); + return new StringEntity(RESTUtil.encodeFormData(formData), StandardCharsets.UTF_8); } } diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTCatalog.java b/core/src/main/java/org/apache/iceberg/rest/RESTCatalog.java index 71195b9585ef..63b660c46aa3 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTCatalog.java @@ -33,16 +33,19 @@ import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.SessionCatalog; import org.apache.iceberg.catalog.SupportsNamespaces; +import org.apache.iceberg.catalog.TableCommit; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.exceptions.NamespaceNotEmptyException; import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.hadoop.Configurable; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; public class RESTCatalog implements Catalog, SupportsNamespaces, Configurable, Closeable { private final RESTSessionCatalog sessionCatalog; private final Catalog delegate; private final SupportsNamespaces nsDelegate; + private final SessionCatalog.SessionContext context; public RESTCatalog() { this( @@ -60,6 +63,7 @@ public RESTCatalog( this.sessionCatalog = new RESTSessionCatalog(clientBuilder, null); this.delegate = sessionCatalog.asCatalog(context); this.nsDelegate = (SupportsNamespaces) delegate; + this.context = context; } @Override @@ -248,4 +252,13 @@ public void setConf(Object conf) { public void close() throws IOException { sessionCatalog.close(); } + + public void commitTransaction(List commits) { + sessionCatalog.commitTransaction(context, commits); + } + + public void commitTransaction(TableCommit... commits) { + sessionCatalog.commitTransaction( + context, ImmutableList.builder().add(commits).build()); + } } diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTSerializers.java b/core/src/main/java/org/apache/iceberg/rest/RESTSerializers.java index 9e17f50c530a..06f2de32df0c 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSerializers.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSerializers.java @@ -42,11 +42,18 @@ import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.catalog.TableIdentifierParser; import org.apache.iceberg.rest.auth.OAuth2Util; +import org.apache.iceberg.rest.requests.CommitTransactionRequest; +import org.apache.iceberg.rest.requests.CommitTransactionRequestParser; +import org.apache.iceberg.rest.requests.ImmutableRegisterTableRequest; import org.apache.iceberg.rest.requests.ImmutableReportMetricsRequest; +import org.apache.iceberg.rest.requests.RegisterTableRequest; +import org.apache.iceberg.rest.requests.RegisterTableRequestParser; import org.apache.iceberg.rest.requests.ReportMetricsRequest; import org.apache.iceberg.rest.requests.ReportMetricsRequestParser; import org.apache.iceberg.rest.requests.UpdateRequirementParser; +import org.apache.iceberg.rest.requests.UpdateTableRequest; import org.apache.iceberg.rest.requests.UpdateTableRequest.UpdateRequirement; +import org.apache.iceberg.rest.requests.UpdateTableRequestParser; import org.apache.iceberg.rest.responses.ErrorResponse; import org.apache.iceberg.rest.responses.ErrorResponseParser; import org.apache.iceberg.rest.responses.OAuthTokenResponse; @@ -77,16 +84,29 @@ public static void registerAll(ObjectMapper mapper) { .addDeserializer(TableMetadata.class, new TableMetadataDeserializer()) .addSerializer(UpdateRequirement.class, new UpdateRequirementSerializer()) .addDeserializer(UpdateRequirement.class, new UpdateRequirementDeserializer()) + .addSerializer(org.apache.iceberg.UpdateRequirement.class, new UpdateReqSerializer()) + .addDeserializer(org.apache.iceberg.UpdateRequirement.class, new UpdateReqDeserializer()) .addSerializer(OAuthTokenResponse.class, new OAuthTokenResponseSerializer()) .addDeserializer(OAuthTokenResponse.class, new OAuthTokenResponseDeserializer()) .addSerializer(ReportMetricsRequest.class, new ReportMetricsRequestSerializer<>()) .addDeserializer(ReportMetricsRequest.class, new ReportMetricsRequestDeserializer<>()) .addSerializer(ImmutableReportMetricsRequest.class, new ReportMetricsRequestSerializer<>()) .addDeserializer( - ImmutableReportMetricsRequest.class, new ReportMetricsRequestDeserializer<>()); + ImmutableReportMetricsRequest.class, new ReportMetricsRequestDeserializer<>()) + .addSerializer(CommitTransactionRequest.class, new CommitTransactionRequestSerializer()) + .addDeserializer(CommitTransactionRequest.class, new CommitTransactionRequestDeserializer()) + .addSerializer(UpdateTableRequest.class, new UpdateTableRequestSerializer()) + .addDeserializer(UpdateTableRequest.class, new UpdateTableRequestDeserializer()) + .addSerializer(RegisterTableRequest.class, new RegisterTableRequestSerializer<>()) + .addDeserializer(RegisterTableRequest.class, new RegisterTableRequestDeserializer<>()) + .addSerializer(ImmutableRegisterTableRequest.class, new RegisterTableRequestSerializer<>()) + .addDeserializer( + ImmutableRegisterTableRequest.class, new RegisterTableRequestDeserializer<>()); mapper.registerModule(module); } + /** @deprecated will be removed in 1.5.0, use {@link UpdateReqDeserializer} instead. */ + @Deprecated public static class UpdateRequirementDeserializer extends JsonDeserializer { @Override public UpdateRequirement deserialize(JsonParser p, DeserializationContext ctxt) @@ -96,6 +116,8 @@ public UpdateRequirement deserialize(JsonParser p, DeserializationContext ctxt) } } + /** @deprecated will be removed in 1.5.0, use {@link UpdateReqSerializer} instead. */ + @Deprecated public static class UpdateRequirementSerializer extends JsonSerializer { @Override public void serialize( @@ -105,6 +127,27 @@ public void serialize( } } + static class UpdateReqDeserializer + extends JsonDeserializer { + @Override + public org.apache.iceberg.UpdateRequirement deserialize( + JsonParser p, DeserializationContext ctxt) throws IOException { + JsonNode node = p.getCodec().readTree(p); + return org.apache.iceberg.UpdateRequirementParser.fromJson(node); + } + } + + static class UpdateReqSerializer extends JsonSerializer { + @Override + public void serialize( + org.apache.iceberg.UpdateRequirement value, + JsonGenerator gen, + SerializerProvider serializers) + throws IOException { + org.apache.iceberg.UpdateRequirementParser.toJson(value, gen); + } + } + public static class TableMetadataDeserializer extends JsonDeserializer { @Override public TableMetadata deserialize(JsonParser p, DeserializationContext context) @@ -280,4 +323,60 @@ public T deserialize(JsonParser p, DeserializationContext context) throws IOExce return (T) ReportMetricsRequestParser.fromJson(jsonNode); } } + + public static class CommitTransactionRequestSerializer + extends JsonSerializer { + @Override + public void serialize( + CommitTransactionRequest request, JsonGenerator gen, SerializerProvider serializers) + throws IOException { + CommitTransactionRequestParser.toJson(request, gen); + } + } + + public static class CommitTransactionRequestDeserializer + extends JsonDeserializer { + @Override + public CommitTransactionRequest deserialize(JsonParser p, DeserializationContext context) + throws IOException { + JsonNode jsonNode = p.getCodec().readTree(p); + return CommitTransactionRequestParser.fromJson(jsonNode); + } + } + + public static class UpdateTableRequestSerializer extends JsonSerializer { + @Override + public void serialize( + UpdateTableRequest request, JsonGenerator gen, SerializerProvider serializers) + throws IOException { + UpdateTableRequestParser.toJson(request, gen); + } + } + + public static class UpdateTableRequestDeserializer extends JsonDeserializer { + @Override + public UpdateTableRequest deserialize(JsonParser p, DeserializationContext context) + throws IOException { + JsonNode jsonNode = p.getCodec().readTree(p); + return UpdateTableRequestParser.fromJson(jsonNode); + } + } + + public static class RegisterTableRequestSerializer + extends JsonSerializer { + @Override + public void serialize(T request, JsonGenerator gen, SerializerProvider serializers) + throws IOException { + RegisterTableRequestParser.toJson(request, gen); + } + } + + public static class RegisterTableRequestDeserializer + extends JsonDeserializer { + @Override + public T deserialize(JsonParser p, DeserializationContext context) throws IOException { + JsonNode jsonNode = p.getCodec().readTree(p); + return (T) RegisterTableRequestParser.fromJson(jsonNode); + } + } } diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java index b83794acca9c..72547eec3486 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -52,6 +52,7 @@ import org.apache.iceberg.catalog.BaseSessionCatalog; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableCommit; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.exceptions.NoSuchTableException; @@ -63,13 +64,18 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.rest.auth.OAuth2Properties; import org.apache.iceberg.rest.auth.OAuth2Util; import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession; +import org.apache.iceberg.rest.requests.CommitTransactionRequest; import org.apache.iceberg.rest.requests.CreateNamespaceRequest; import org.apache.iceberg.rest.requests.CreateTableRequest; +import org.apache.iceberg.rest.requests.ImmutableRegisterTableRequest; +import org.apache.iceberg.rest.requests.RegisterTableRequest; import org.apache.iceberg.rest.requests.RenameTableRequest; import org.apache.iceberg.rest.requests.UpdateNamespacePropertiesRequest; +import org.apache.iceberg.rest.requests.UpdateTableRequest; import org.apache.iceberg.rest.responses.ConfigResponse; import org.apache.iceberg.rest.responses.CreateNamespaceResponse; import org.apache.iceberg.rest.responses.GetNamespaceResponse; @@ -400,7 +406,40 @@ public void invalidateTable(SessionContext context, TableIdentifier ident) {} @Override public Table registerTable( SessionContext context, TableIdentifier ident, String metadataFileLocation) { - throw new UnsupportedOperationException("Register table is not supported"); + checkIdentifierIsValid(ident); + + Preconditions.checkArgument( + metadataFileLocation != null && !metadataFileLocation.isEmpty(), + "Invalid metadata file location: %s", + metadataFileLocation); + + RegisterTableRequest request = + ImmutableRegisterTableRequest.builder() + .name(ident.name()) + .metadataLocation(metadataFileLocation) + .build(); + + LoadTableResponse response = + client.post( + paths.register(ident.namespace()), + request, + LoadTableResponse.class, + headers(context), + ErrorHandlers.tableErrorHandler()); + + AuthSession session = tableSession(response.config(), session(context)); + RESTTableOperations ops = + new RESTTableOperations( + client, + paths.table(ident), + session::headers, + tableFileIO(context, response.config()), + response.tableMetadata()); + + trackFileIO(ops); + + return new BaseTable( + ops, fullTableName(ident), metricsReporter(paths.metrics(ident), session::headers)); } @Override @@ -916,4 +955,20 @@ private Cache newFileIOCloser() { }) .build(); } + + public void commitTransaction(SessionContext context, List commits) { + List tableChanges = Lists.newArrayListWithCapacity(commits.size()); + + for (TableCommit commit : commits) { + tableChanges.add( + UpdateTableRequest.create(commit.identifier(), commit.requirements(), commit.updates())); + } + + client.post( + paths.commitTransaction(), + new CommitTransactionRequest(tableChanges), + null, + headers(context), + ErrorHandlers.tableCommitHandler()); + } } diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTTableOperations.java b/core/src/main/java/org/apache/iceberg/rest/RESTTableOperations.java index 0841de763bf1..0ce1afd93a79 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTTableOperations.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTTableOperations.java @@ -28,6 +28,8 @@ import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableOperations; import org.apache.iceberg.TableProperties; +import org.apache.iceberg.UpdateRequirement; +import org.apache.iceberg.UpdateRequirements; import org.apache.iceberg.encryption.EncryptionManager; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.LocationProvider; @@ -101,30 +103,38 @@ public TableMetadata refresh() { @Override public void commit(TableMetadata base, TableMetadata metadata) { - UpdateTableRequest.Builder requestBuilder; - List baseChanges; Consumer errorHandler; + List requirements; + List updates; switch (updateType) { case CREATE: Preconditions.checkState( base == null, "Invalid base metadata for create transaction, expected null: %s", base); - requestBuilder = UpdateTableRequest.builderForCreate(); - baseChanges = createChanges; + updates = + ImmutableList.builder() + .addAll(createChanges) + .addAll(metadata.changes()) + .build(); + requirements = UpdateRequirements.forCreateTable(updates); errorHandler = ErrorHandlers.tableErrorHandler(); // throws NoSuchTableException break; case REPLACE: Preconditions.checkState(base != null, "Invalid base metadata: null"); + updates = + ImmutableList.builder() + .addAll(createChanges) + .addAll(metadata.changes()) + .build(); // use the original replace base metadata because the transaction will refresh - requestBuilder = UpdateTableRequest.builderForReplace(replaceBase); - baseChanges = createChanges; + requirements = UpdateRequirements.forReplaceTable(replaceBase, updates); errorHandler = ErrorHandlers.tableCommitHandler(); break; case SIMPLE: Preconditions.checkState(base != null, "Invalid base metadata: null"); - requestBuilder = UpdateTableRequest.builderFor(base); - baseChanges = ImmutableList.of(); + updates = metadata.changes(); + requirements = UpdateRequirements.forUpdateTable(base, updates); errorHandler = ErrorHandlers.tableCommitHandler(); break; @@ -133,9 +143,7 @@ public void commit(TableMetadata base, TableMetadata metadata) { String.format("Update type %s is not supported", updateType)); } - baseChanges.forEach(requestBuilder::update); - metadata.changes().forEach(requestBuilder::update); - UpdateTableRequest request = requestBuilder.build(); + UpdateTableRequest request = new UpdateTableRequest(requirements, updates); // the error handler will throw necessary exceptions like CommitFailedException and // UnknownCommitStateException diff --git a/core/src/main/java/org/apache/iceberg/rest/ResourcePaths.java b/core/src/main/java/org/apache/iceberg/rest/ResourcePaths.java index 6fa09f33d2ba..b5b974f14c5e 100644 --- a/core/src/main/java/org/apache/iceberg/rest/ResourcePaths.java +++ b/core/src/main/java/org/apache/iceberg/rest/ResourcePaths.java @@ -71,6 +71,10 @@ public String table(TableIdentifier ident) { RESTUtil.encodeString(ident.name())); } + public String register(Namespace ns) { + return SLASH.join("v1", prefix, "namespaces", RESTUtil.encodeNamespace(ns), "register"); + } + public String rename() { return SLASH.join("v1", prefix, "tables", "rename"); } @@ -85,4 +89,8 @@ public String metrics(TableIdentifier identifier) { RESTUtil.encodeString(identifier.name()), "metrics"); } + + public String commitTransaction() { + return SLASH.join("v1", prefix, "transactions", "commit"); + } } diff --git a/core/src/main/java/org/apache/iceberg/rest/requests/CommitTransactionRequest.java b/core/src/main/java/org/apache/iceberg/rest/requests/CommitTransactionRequest.java new file mode 100644 index 000000000000..ef41664a078b --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/rest/requests/CommitTransactionRequest.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest.requests; + +import java.util.List; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.rest.RESTRequest; + +public class CommitTransactionRequest implements RESTRequest { + private final List tableChanges; + + public CommitTransactionRequest(List tableChanges) { + this.tableChanges = tableChanges; + validate(); + } + + public List tableChanges() { + return ImmutableList.copyOf(tableChanges); + } + + @Override + public void validate() { + Preconditions.checkArgument(null != tableChanges, "Invalid table changes: null"); + Preconditions.checkArgument(!tableChanges.isEmpty(), "Invalid table changes: empty"); + for (UpdateTableRequest tableChange : tableChanges) { + Preconditions.checkArgument( + null != tableChange.identifier(), "Invalid table changes: table identifier is required"); + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/rest/requests/CommitTransactionRequestParser.java b/core/src/main/java/org/apache/iceberg/rest/requests/CommitTransactionRequestParser.java new file mode 100644 index 000000000000..a2519ed0f570 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/rest/requests/CommitTransactionRequestParser.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest.requests; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import java.io.IOException; +import java.util.List; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.JsonUtil; + +public class CommitTransactionRequestParser { + private static final String TABLE_CHANGES = "table-changes"; + + private CommitTransactionRequestParser() {} + + public static String toJson(CommitTransactionRequest request) { + return toJson(request, false); + } + + public static String toJson(CommitTransactionRequest request, boolean pretty) { + return JsonUtil.generate(gen -> toJson(request, gen), pretty); + } + + public static void toJson(CommitTransactionRequest request, JsonGenerator gen) + throws IOException { + Preconditions.checkArgument(null != request, "Invalid commit transaction request: null"); + + gen.writeStartObject(); + gen.writeFieldName(TABLE_CHANGES); + gen.writeStartArray(); + + for (UpdateTableRequest tableChange : request.tableChanges()) { + UpdateTableRequestParser.toJson(tableChange, gen); + } + + gen.writeEndArray(); + gen.writeEndObject(); + } + + public static CommitTransactionRequest fromJson(String json) { + return JsonUtil.parse(json, CommitTransactionRequestParser::fromJson); + } + + public static CommitTransactionRequest fromJson(JsonNode json) { + Preconditions.checkArgument( + null != json, "Cannot parse commit transaction request from null object"); + + List tableChanges = Lists.newArrayList(); + JsonNode changes = JsonUtil.get(TABLE_CHANGES, json); + + Preconditions.checkArgument( + changes.isArray(), "Cannot parse commit transaction request from non-array: %s", changes); + + for (JsonNode node : changes) { + tableChanges.add(UpdateTableRequestParser.fromJson(node)); + } + + return new CommitTransactionRequest(tableChanges); + } +} diff --git a/core/src/main/java/org/apache/iceberg/rest/requests/RegisterTableRequest.java b/core/src/main/java/org/apache/iceberg/rest/requests/RegisterTableRequest.java new file mode 100644 index 000000000000..33b37dae242f --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/rest/requests/RegisterTableRequest.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.rest.requests; + +import org.apache.iceberg.rest.RESTRequest; +import org.immutables.value.Value; + +@Value.Immutable +public interface RegisterTableRequest extends RESTRequest { + + String name(); + + String metadataLocation(); + + @Override + default void validate() { + // nothing to validate as it's not possible to create an invalid instance + } +} diff --git a/core/src/main/java/org/apache/iceberg/rest/requests/RegisterTableRequestParser.java b/core/src/main/java/org/apache/iceberg/rest/requests/RegisterTableRequestParser.java new file mode 100644 index 000000000000..961b6c185b87 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/rest/requests/RegisterTableRequestParser.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest.requests; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import java.io.IOException; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.util.JsonUtil; + +public class RegisterTableRequestParser { + + private static final String NAME = "name"; + private static final String METADATA_LOCATION = "metadata-location"; + + private RegisterTableRequestParser() {} + + public static String toJson(RegisterTableRequest request) { + return toJson(request, false); + } + + public static String toJson(RegisterTableRequest request, boolean pretty) { + return JsonUtil.generate(gen -> toJson(request, gen), pretty); + } + + public static void toJson(RegisterTableRequest request, JsonGenerator gen) throws IOException { + Preconditions.checkArgument(null != request, "Invalid register table request: null"); + + gen.writeStartObject(); + + gen.writeStringField(NAME, request.name()); + gen.writeStringField(METADATA_LOCATION, request.metadataLocation()); + + gen.writeEndObject(); + } + + public static RegisterTableRequest fromJson(String json) { + return JsonUtil.parse(json, RegisterTableRequestParser::fromJson); + } + + public static RegisterTableRequest fromJson(JsonNode json) { + Preconditions.checkArgument( + null != json, "Cannot parse register table request from null object"); + + String name = JsonUtil.getString(NAME, json); + String metadataLocation = JsonUtil.getString(METADATA_LOCATION, json); + + return ImmutableRegisterTableRequest.builder() + .name(name) + .metadataLocation(metadataLocation) + .build(); + } +} diff --git a/core/src/main/java/org/apache/iceberg/rest/requests/UpdateRequirementParser.java b/core/src/main/java/org/apache/iceberg/rest/requests/UpdateRequirementParser.java index 443ae87b9eee..2df522b9aac7 100644 --- a/core/src/main/java/org/apache/iceberg/rest/requests/UpdateRequirementParser.java +++ b/core/src/main/java/org/apache/iceberg/rest/requests/UpdateRequirementParser.java @@ -28,6 +28,11 @@ import org.apache.iceberg.rest.requests.UpdateTableRequest.UpdateRequirement; import org.apache.iceberg.util.JsonUtil; +/** + * @deprecated will be removed in 1.5.0, use {@link org.apache.iceberg.UpdateRequirementParser} + * instead. + */ +@Deprecated public class UpdateRequirementParser { private UpdateRequirementParser() {} diff --git a/core/src/main/java/org/apache/iceberg/rest/requests/UpdateTableRequest.java b/core/src/main/java/org/apache/iceberg/rest/requests/UpdateTableRequest.java index 694e44e841dc..088875d472f7 100644 --- a/core/src/main/java/org/apache/iceberg/rest/requests/UpdateTableRequest.java +++ b/core/src/main/java/org/apache/iceberg/rest/requests/UpdateTableRequest.java @@ -23,6 +23,7 @@ import org.apache.iceberg.MetadataUpdate; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -33,22 +34,32 @@ public class UpdateTableRequest implements RESTRequest { - private List requirements; + private TableIdentifier identifier; + private List requirements; private List updates; public UpdateTableRequest() { // needed for Jackson deserialization } - public UpdateTableRequest(List requirements, List updates) { + public UpdateTableRequest( + List requirements, List updates) { this.requirements = requirements; this.updates = updates; } + UpdateTableRequest( + TableIdentifier identifier, + List requirements, + List updates) { + this(requirements, updates); + this.identifier = identifier; + } + @Override public void validate() {} - public List requirements() { + public List requirements() { return requirements != null ? requirements : ImmutableList.of(); } @@ -56,6 +67,10 @@ public List updates() { return updates != null ? updates : ImmutableList.of(); } + public TableIdentifier identifier() { + return identifier; + } + @Override public String toString() { return MoreObjects.toStringHelper(this) @@ -64,23 +79,51 @@ public String toString() { .toString(); } + public static UpdateTableRequest create( + TableIdentifier identifier, + List requirements, + List updates) { + return new UpdateTableRequest(identifier, requirements, updates); + } + + /** + * @deprecated will be removed in 1.5.0, use {@link + * org.apache.iceberg.UpdateRequirements#forCreateTable(List)} instead. + */ + @Deprecated public static Builder builderForCreate() { return new Builder(null, false).requireCreate(); } + /** + * @deprecated will be removed in 1.5.0, use {@link + * org.apache.iceberg.UpdateRequirements#forReplaceTable(TableMetadata, List)} instead. + */ + @Deprecated public static Builder builderForReplace(TableMetadata base) { Preconditions.checkNotNull(base, "Cannot create a builder from table metadata: null"); return new Builder(base, true).requireTableUUID(base.uuid()); } + /** + * @deprecated will be removed in 1.5.0, use {@link + * org.apache.iceberg.UpdateRequirements#forUpdateTable(TableMetadata, List)} instead. + */ + @Deprecated public static Builder builderFor(TableMetadata base) { Preconditions.checkNotNull(base, "Cannot create a builder from table metadata: null"); return new Builder(base, false).requireTableUUID(base.uuid()); } + /** + * @deprecated will be removed in 1.5.0, use {@link org.apache.iceberg.UpdateRequirements} + * instead. + */ + @Deprecated public static class Builder { private final TableMetadata base; - private final ImmutableList.Builder requirements = ImmutableList.builder(); + private final ImmutableList.Builder requirements = + ImmutableList.builder(); private final List updates = Lists.newArrayList(); private final Set changedRefs = Sets.newHashSet(); private final boolean isReplace; @@ -221,8 +264,11 @@ public UpdateTableRequest build() { } } - public interface UpdateRequirement { - void validate(TableMetadata base); + /** + * @deprecated will be removed in 1.5.0, use {@link org.apache.iceberg.UpdateRequirement} instead. + */ + @Deprecated + public interface UpdateRequirement extends org.apache.iceberg.UpdateRequirement { class AssertTableDoesNotExist implements UpdateRequirement { AssertTableDoesNotExist() {} diff --git a/core/src/main/java/org/apache/iceberg/rest/requests/UpdateTableRequestParser.java b/core/src/main/java/org/apache/iceberg/rest/requests/UpdateTableRequestParser.java new file mode 100644 index 000000000000..e2a8f8b307d1 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/rest/requests/UpdateTableRequestParser.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest.requests; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import java.io.IOException; +import java.util.List; +import org.apache.iceberg.MetadataUpdate; +import org.apache.iceberg.MetadataUpdateParser; +import org.apache.iceberg.UpdateRequirement; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.catalog.TableIdentifierParser; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.JsonUtil; + +public class UpdateTableRequestParser { + + private static final String IDENTIFIER = "identifier"; + private static final String REQUIREMENTS = "requirements"; + private static final String UPDATES = "updates"; + + private UpdateTableRequestParser() {} + + public static String toJson(UpdateTableRequest request) { + return toJson(request, false); + } + + public static String toJson(UpdateTableRequest request, boolean pretty) { + return JsonUtil.generate(gen -> toJson(request, gen), pretty); + } + + public static void toJson(UpdateTableRequest request, JsonGenerator gen) throws IOException { + Preconditions.checkArgument(null != request, "Invalid update table request: null"); + + gen.writeStartObject(); + + if (null != request.identifier()) { + gen.writeFieldName(IDENTIFIER); + TableIdentifierParser.toJson(request.identifier(), gen); + } + + gen.writeArrayFieldStart(REQUIREMENTS); + for (UpdateRequirement updateRequirement : request.requirements()) { + org.apache.iceberg.UpdateRequirementParser.toJson(updateRequirement, gen); + } + gen.writeEndArray(); + + gen.writeArrayFieldStart(UPDATES); + for (MetadataUpdate metadataUpdate : request.updates()) { + MetadataUpdateParser.toJson(metadataUpdate, gen); + } + gen.writeEndArray(); + + gen.writeEndObject(); + } + + public static UpdateTableRequest fromJson(String json) { + return JsonUtil.parse(json, UpdateTableRequestParser::fromJson); + } + + public static UpdateTableRequest fromJson(JsonNode json) { + Preconditions.checkArgument(null != json, "Cannot parse update table request from null object"); + + TableIdentifier identifier = null; + List requirements = Lists.newArrayList(); + List updates = Lists.newArrayList(); + + if (json.hasNonNull(IDENTIFIER)) { + identifier = TableIdentifierParser.fromJson(JsonUtil.get(IDENTIFIER, json)); + } + + if (json.hasNonNull(REQUIREMENTS)) { + JsonNode requirementsNode = JsonUtil.get(REQUIREMENTS, json); + Preconditions.checkArgument( + requirementsNode.isArray(), + "Cannot parse requirements from non-array: %s", + requirementsNode); + requirementsNode.forEach( + req -> requirements.add(org.apache.iceberg.UpdateRequirementParser.fromJson(req))); + } + + if (json.hasNonNull(UPDATES)) { + JsonNode updatesNode = JsonUtil.get(UPDATES, json); + Preconditions.checkArgument( + updatesNode.isArray(), "Cannot parse metadata updates from non-array: %s", updatesNode); + + updatesNode.forEach(update -> updates.add(MetadataUpdateParser.fromJson(update))); + } + + return UpdateTableRequest.create(identifier, requirements, updates); + } +} diff --git a/core/src/main/java/org/apache/iceberg/util/JsonUtil.java b/core/src/main/java/org/apache/iceberg/util/JsonUtil.java index 84c0681164d6..aa90c63f80da 100644 --- a/core/src/main/java/org/apache/iceberg/util/JsonUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/JsonUtil.java @@ -26,14 +26,17 @@ import java.io.IOException; import java.io.StringWriter; import java.io.UncheckedIOException; +import java.nio.ByteBuffer; import java.util.Iterator; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Set; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.io.BaseEncoding; public class JsonUtil { @@ -173,12 +176,24 @@ public static String getStringOrNull(String property, JsonNode node) { return getString(property, node); } + public static ByteBuffer getByteBufferOrNull(String property, JsonNode node) { + if (!node.has(property) || node.get(property).isNull()) { + return null; + } + + JsonNode pNode = node.get(property); + Preconditions.checkArgument( + pNode.isTextual(), "Cannot parse byte buffer from non-text value: %s: %s", property, pNode); + return ByteBuffer.wrap( + BaseEncoding.base16().decode(pNode.textValue().toUpperCase(Locale.ROOT))); + } + public static Map getStringMap(String property, JsonNode node) { Preconditions.checkArgument(node.has(property), "Cannot parse missing map: %s", property); JsonNode pNode = node.get(property); Preconditions.checkArgument( pNode != null && !pNode.isNull() && pNode.isObject(), - "Cannot parse from non-object value: %s: %s", + "Cannot parse string map from non-object value: %s: %s", property, pNode); @@ -229,6 +244,14 @@ public static List getStringListOrNull(String property, JsonNode node) { .build(); } + public static int[] getIntArrayOrNull(String property, JsonNode node) { + if (!node.has(property) || node.get(property).isNull()) { + return null; + } + + return ArrayUtil.toIntArray(getIntegerList(property, node)); + } + public static List getIntegerList(String property, JsonNode node) { Preconditions.checkArgument(node.has(property), "Cannot parse missing list: %s", property); return ImmutableList.builder() @@ -256,6 +279,14 @@ public static List getLongList(String property, JsonNode node) { return ImmutableList.builder().addAll(new JsonLongArrayIterator(property, node)).build(); } + public static List getLongListOrNull(String property, JsonNode node) { + if (!node.has(property) || node.get(property).isNull()) { + return null; + } + + return ImmutableList.builder().addAll(new JsonLongArrayIterator(property, node)).build(); + } + public static Set getLongSetOrNull(String property, JsonNode node) { if (!node.hasNonNull(property)) { return null; @@ -291,7 +322,7 @@ abstract static class JsonArrayIterator implements Iterator { JsonNode pNode = node.get(property); Preconditions.checkArgument( pNode != null && !pNode.isNull() && pNode.isArray(), - "Cannot parse from non-array value: %s: %s", + "Cannot parse JSON array from non-array value: %s: %s", property, pNode); this.elements = pNode.elements(); diff --git a/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java b/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java index e5c4351f1e4c..2f9590dfcddb 100644 --- a/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java @@ -239,6 +239,10 @@ public boolean hasNext() { return true; } + if (next == null) { + return false; + } + Long parentId = next.parentId(); if (parentId == null) { return false; diff --git a/core/src/main/java/org/apache/iceberg/util/TableScanUtil.java b/core/src/main/java/org/apache/iceberg/util/TableScanUtil.java index af3c28c81d6c..6e25e380dd21 100644 --- a/core/src/main/java/org/apache/iceberg/util/TableScanUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/TableScanUtil.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.util; +import java.math.RoundingMode; import java.util.List; import java.util.Map; import java.util.function.Function; @@ -42,10 +43,13 @@ 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.math.LongMath; import org.apache.iceberg.types.Types; public class TableScanUtil { + private static final long MIN_SPLIT_SIZE = 16 * 1024 * 1024; // 16 MB + private TableScanUtil() {} public static boolean hasDeletes(CombinedScanTask task) { @@ -246,6 +250,15 @@ public static List mergeTasks(List tasks) { return mergedTasks; } + public static long adjustSplitSize(long scanSize, int parallelism, long splitSize) { + // use the configured split size if it produces at least one split per slot + // otherwise, adjust the split size to target parallelism with a reasonable minimum + // increasing the split size may cause expensive spills and is not done automatically + long splitCount = LongMath.divide(scanSize, splitSize, RoundingMode.CEILING); + long adjustedSplitSize = Math.max(scanSize / parallelism, Math.min(MIN_SPLIT_SIZE, splitSize)); + return splitCount < parallelism ? adjustedSplitSize : splitSize; + } + private static void validatePlanningArguments(long splitSize, int lookback, long openFileCost) { Preconditions.checkArgument(splitSize > 0, "Split size must be > 0: %s", splitSize); Preconditions.checkArgument(lookback > 0, "Split planning lookback must be > 0: %s", lookback); diff --git a/core/src/main/java/org/apache/iceberg/view/BaseViewHistoryEntry.java b/core/src/main/java/org/apache/iceberg/view/BaseViewHistoryEntry.java new file mode 100644 index 000000000000..3ce28142ea10 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/view/BaseViewHistoryEntry.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.view; + +import org.immutables.value.Value; +import org.immutables.value.Value.Style.BuilderVisibility; +import org.immutables.value.Value.Style.ImplementationVisibility; + +/** + * View history entry. + * + *

An entry contains a change to the view state. At the given timestamp, the current version was + * set to the given version ID. + */ +@Value.Immutable +@SuppressWarnings("ImmutablesStyle") +@Value.Style( + typeImmutable = "ImmutableViewHistoryEntry", + visibility = ImplementationVisibility.PUBLIC, + builderVisibility = BuilderVisibility.PUBLIC) +interface BaseViewHistoryEntry extends ViewHistoryEntry {} diff --git a/core/src/main/java/org/apache/iceberg/view/BaseViewVersion.java b/core/src/main/java/org/apache/iceberg/view/BaseViewVersion.java new file mode 100644 index 000000000000..986628a91157 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/view/BaseViewVersion.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.view; + +import javax.annotation.Nullable; +import org.apache.iceberg.catalog.Namespace; +import org.immutables.value.Value; +import org.immutables.value.Value.Style.BuilderVisibility; +import org.immutables.value.Value.Style.ImplementationVisibility; + +/** + * A version of the view at a point in time. + * + *

A version consists of a view metadata file. + * + *

Versions are created by view operations, like Create and Replace. + */ +@Value.Immutable +@SuppressWarnings("ImmutablesStyle") +@Value.Style( + typeImmutable = "ImmutableViewVersion", + visibility = ImplementationVisibility.PUBLIC, + builderVisibility = BuilderVisibility.PUBLIC) +interface BaseViewVersion extends ViewVersion { + + @Override + @Value.Lazy + default String operation() { + return summary().get("operation"); + } + + @Override + @Nullable + String defaultCatalog(); + + @Override + @Nullable + Namespace defaultNamespace(); + + @Override + @Value.Check + default void check() { + ViewVersion.super.check(); + } +} diff --git a/api/src/main/java/org/apache/iceberg/view/SQLViewRepresentation.java b/core/src/main/java/org/apache/iceberg/view/SQLViewRepresentation.java similarity index 72% rename from api/src/main/java/org/apache/iceberg/view/SQLViewRepresentation.java rename to core/src/main/java/org/apache/iceberg/view/SQLViewRepresentation.java index c308a6289b94..41124d9dde03 100644 --- a/api/src/main/java/org/apache/iceberg/view/SQLViewRepresentation.java +++ b/core/src/main/java/org/apache/iceberg/view/SQLViewRepresentation.java @@ -18,9 +18,6 @@ */ package org.apache.iceberg.view; -import java.util.List; -import javax.annotation.Nullable; -import org.apache.iceberg.catalog.Namespace; import org.immutables.value.Value; @Value.Immutable @@ -36,18 +33,4 @@ default String type() { /** The view query SQL dialect. */ String dialect(); - - /** The default catalog when the view is created. */ - @Nullable - String defaultCatalog(); - - /** The default namespace when the view is created. */ - @Nullable - Namespace defaultNamespace(); - - /** The view field comments. */ - List fieldComments(); - - /** The view field aliases. */ - List fieldAliases(); } diff --git a/core/src/main/java/org/apache/iceberg/view/SQLViewRepresentationParser.java b/core/src/main/java/org/apache/iceberg/view/SQLViewRepresentationParser.java index ae3676e8bd18..bd1d84a6031f 100644 --- a/core/src/main/java/org/apache/iceberg/view/SQLViewRepresentationParser.java +++ b/core/src/main/java/org/apache/iceberg/view/SQLViewRepresentationParser.java @@ -21,21 +21,12 @@ import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.databind.JsonNode; import java.io.IOException; -import java.util.Arrays; -import java.util.List; -import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.util.JsonUtil; class SQLViewRepresentationParser { private static final String SQL = "sql"; private static final String DIALECT = "dialect"; - private static final String SCHEMA_ID = "schema-id"; - private static final String DEFAULT_CATALOG = "default-catalog"; - private static final String DEFAULT_NAMESPACE = "default-namespace"; - private static final String FIELD_ALIASES = "field-aliases"; - private static final String FIELD_COMMENTS = "field-comments"; private SQLViewRepresentationParser() {} @@ -50,23 +41,6 @@ static void toJson(SQLViewRepresentation view, JsonGenerator generator) throws I generator.writeStringField(SQL, view.sql()); generator.writeStringField(DIALECT, view.dialect()); - if (view.defaultCatalog() != null) { - generator.writeStringField(DEFAULT_CATALOG, view.defaultCatalog()); - } - - if (view.defaultNamespace() != null) { - JsonUtil.writeStringArray( - DEFAULT_NAMESPACE, Arrays.asList(view.defaultNamespace().levels()), generator); - } - - if (!view.fieldAliases().isEmpty()) { - JsonUtil.writeStringArray(FIELD_ALIASES, view.fieldAliases(), generator); - } - - if (!view.fieldComments().isEmpty()) { - JsonUtil.writeStringArray(FIELD_COMMENTS, view.fieldComments(), generator); - } - generator.writeEndObject(); } @@ -83,27 +57,6 @@ static SQLViewRepresentation fromJson(JsonNode node) { ImmutableSQLViewRepresentation.builder() .sql(JsonUtil.getString(SQL, node)) .dialect(JsonUtil.getString(DIALECT, node)); - String defaultCatalog = JsonUtil.getStringOrNull(DEFAULT_CATALOG, node); - if (defaultCatalog != null) { - builder.defaultCatalog(defaultCatalog); - } - - Integer schemaId = JsonUtil.getIntOrNull(SCHEMA_ID, node); - - List namespace = JsonUtil.getStringListOrNull(DEFAULT_NAMESPACE, node); - if (namespace != null && !namespace.isEmpty()) { - builder.defaultNamespace(Namespace.of(Iterables.toArray(namespace, String.class))); - } - - List fieldAliases = JsonUtil.getStringListOrNull(FIELD_ALIASES, node); - if (fieldAliases != null) { - builder.fieldAliases(fieldAliases); - } - - List fieldComments = JsonUtil.getStringListOrNull(FIELD_COMMENTS, node); - if (fieldComments != null) { - builder.fieldComments(fieldComments); - } return builder.build(); } diff --git a/core/src/main/java/org/apache/iceberg/view/ViewMetadata.java b/core/src/main/java/org/apache/iceberg/view/ViewMetadata.java new file mode 100644 index 000000000000..cfd91b02f2e8 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/view/ViewMetadata.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.view; + +import java.io.Serializable; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.Schema; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.util.PropertyUtil; +import org.immutables.value.Value; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Value.Immutable +public interface ViewMetadata extends Serializable { + Logger LOG = LoggerFactory.getLogger(ViewMetadata.class); + int SUPPORTED_VIEW_FORMAT_VERSION = 1; + + int formatVersion(); + + String location(); + + Integer currentSchemaId(); + + List schemas(); + + int currentVersionId(); + + List versions(); + + List history(); + + Map properties(); + + default ViewVersion version(int versionId) { + return versionsById().get(versionId); + } + + default ViewVersion currentVersion() { + return versionsById().get(currentVersionId()); + } + + @Value.Derived + default Map versionsById() { + ImmutableMap.Builder builder = ImmutableMap.builder(); + for (ViewVersion version : versions()) { + builder.put(version.versionId(), version); + } + + return builder.build(); + } + + @Value.Derived + default Map schemasById() { + ImmutableMap.Builder builder = ImmutableMap.builder(); + for (Schema schema : schemas()) { + builder.put(schema.schemaId(), schema); + } + + return builder.build(); + } + + default Schema schema() { + return schemasById().get(currentSchemaId()); + } + + @Value.Check + default ViewMetadata checkAndNormalize() { + Preconditions.checkArgument( + formatVersion() > 0 && formatVersion() <= ViewMetadata.SUPPORTED_VIEW_FORMAT_VERSION, + "Unsupported format version: %s", + formatVersion()); + + Preconditions.checkArgument(versions().size() > 0, "Invalid view versions: empty"); + Preconditions.checkArgument(history().size() > 0, "Invalid view history: empty"); + Preconditions.checkArgument(schemas().size() > 0, "Invalid schemas: empty"); + + Preconditions.checkArgument( + versionsById().containsKey(currentVersionId()), + "Cannot find current version %s in view versions: %s", + currentVersionId(), + versionsById().keySet()); + + Preconditions.checkArgument( + schemasById().containsKey(currentSchemaId()), + "Cannot find current schema with id %s in schemas: %s", + currentSchemaId(), + schemasById().keySet()); + + int versionHistorySizeToKeep = + PropertyUtil.propertyAsInt( + properties(), + ViewProperties.VERSION_HISTORY_SIZE, + ViewProperties.VERSION_HISTORY_SIZE_DEFAULT); + + if (versionHistorySizeToKeep <= 0) { + LOG.warn( + "{} must be positive but was {}", + ViewProperties.VERSION_HISTORY_SIZE, + versionHistorySizeToKeep); + } else if (versions().size() > versionHistorySizeToKeep) { + List versions = + versions().subList(versions().size() - versionHistorySizeToKeep, versions().size()); + List history = + history().subList(history().size() - versionHistorySizeToKeep, history().size()); + return ImmutableViewMetadata.builder().from(this).versions(versions).history(history).build(); + } + + return this; + } +} diff --git a/core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java b/core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java new file mode 100644 index 000000000000..57c393c44772 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java @@ -0,0 +1,176 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.view; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.OutputStreamWriter; +import java.io.UncheckedIOException; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.JsonUtil; + +public class ViewMetadataParser { + + static final String FORMAT_VERSION = "format-version"; + static final String LOCATION = "location"; + static final String CURRENT_VERSION_ID = "current-version-id"; + static final String VERSIONS = "versions"; + static final String VERSION_LOG = "version-log"; + static final String PROPERTIES = "properties"; + static final String SCHEMAS = "schemas"; + static final String CURRENT_SCHEMA_ID = "current-schema-id"; + + private ViewMetadataParser() {} + + public static String toJson(ViewMetadata metadata) { + return toJson(metadata, false); + } + + public static String toJson(ViewMetadata metadata, boolean pretty) { + return JsonUtil.generate(gen -> toJson(metadata, gen), pretty); + } + + static void toJson(ViewMetadata metadata, JsonGenerator gen) throws IOException { + Preconditions.checkArgument(null != metadata, "Invalid view metadata: null"); + + gen.writeStartObject(); + + gen.writeNumberField(FORMAT_VERSION, metadata.formatVersion()); + gen.writeStringField(LOCATION, metadata.location()); + JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), gen); + gen.writeNumberField(CURRENT_SCHEMA_ID, metadata.currentSchemaId()); + + gen.writeArrayFieldStart(SCHEMAS); + for (Schema schema : metadata.schemas()) { + SchemaParser.toJson(schema, gen); + } + gen.writeEndArray(); + + gen.writeNumberField(CURRENT_VERSION_ID, metadata.currentVersionId()); + gen.writeArrayFieldStart(VERSIONS); + for (ViewVersion version : metadata.versions()) { + ViewVersionParser.toJson(version, gen); + } + gen.writeEndArray(); + + gen.writeArrayFieldStart(VERSION_LOG); + for (ViewHistoryEntry viewHistoryEntry : metadata.history()) { + ViewHistoryEntryParser.toJson(viewHistoryEntry, gen); + } + gen.writeEndArray(); + + gen.writeEndObject(); + } + + public static ViewMetadata fromJson(String json) { + Preconditions.checkArgument(json != null, "Cannot parse view metadata from null string"); + return JsonUtil.parse(json, ViewMetadataParser::fromJson); + } + + public static ViewMetadata fromJson(JsonNode json) { + Preconditions.checkArgument(json != null, "Cannot parse view metadata from null object"); + Preconditions.checkArgument( + json.isObject(), "Cannot parse view metadata from non-object: %s", json); + + int formatVersion = JsonUtil.getInt(FORMAT_VERSION, json); + String location = JsonUtil.getString(LOCATION, json); + Map properties = JsonUtil.getStringMap(PROPERTIES, json); + + int currentSchemaId = JsonUtil.getInt(CURRENT_SCHEMA_ID, json); + JsonNode schemasNode = JsonUtil.get(SCHEMAS, json); + + Preconditions.checkArgument( + schemasNode.isArray(), "Cannot parse schemas from non-array: %s", schemasNode); + List schemas = Lists.newArrayListWithExpectedSize(schemasNode.size()); + + for (JsonNode schemaNode : schemasNode) { + schemas.add(SchemaParser.fromJson(schemaNode)); + } + + int currentVersionId = JsonUtil.getInt(CURRENT_VERSION_ID, json); + JsonNode versionsNode = JsonUtil.get(VERSIONS, json); + Preconditions.checkArgument( + versionsNode.isArray(), "Cannot parse versions from non-array: %s", versionsNode); + List versions = Lists.newArrayListWithExpectedSize(versionsNode.size()); + for (JsonNode versionNode : versionsNode) { + versions.add(ViewVersionParser.fromJson(versionNode)); + } + + JsonNode versionLogNode = JsonUtil.get(VERSION_LOG, json); + Preconditions.checkArgument( + versionLogNode.isArray(), "Cannot parse version-log from non-array: %s", versionLogNode); + List historyEntries = + Lists.newArrayListWithExpectedSize(versionLogNode.size()); + for (JsonNode vLog : versionLogNode) { + historyEntries.add(ViewHistoryEntryParser.fromJson(vLog)); + } + + return ImmutableViewMetadata.builder() + .location(location) + .currentVersionId(currentVersionId) + .properties(properties) + .versions(versions) + .schemas(schemas) + .currentSchemaId(currentSchemaId) + .history(historyEntries) + .formatVersion(formatVersion) + .build(); + } + + public static void overwrite(ViewMetadata metadata, OutputFile outputFile) { + internalWrite(metadata, outputFile, true); + } + + public static void write(ViewMetadata metadata, OutputFile outputFile) { + internalWrite(metadata, outputFile, false); + } + + public static ViewMetadata read(InputFile file) { + try (InputStream is = file.newStream()) { + return fromJson(JsonUtil.mapper().readValue(is, JsonNode.class)); + } catch (IOException e) { + throw new UncheckedIOException(String.format("Failed to read json file: %s", file), e); + } + } + + private static void internalWrite( + ViewMetadata metadata, OutputFile outputFile, boolean overwrite) { + OutputStream stream = overwrite ? outputFile.createOrOverwrite() : outputFile.create(); + try (OutputStreamWriter writer = new OutputStreamWriter(stream, StandardCharsets.UTF_8)) { + JsonGenerator generator = JsonUtil.factory().createGenerator(writer); + generator.useDefaultPrettyPrinter(); + toJson(metadata, generator); + generator.flush(); + } catch (IOException e) { + throw new UncheckedIOException( + String.format("Failed to write json to file: %s", outputFile), e); + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/view/ViewProperties.java b/core/src/main/java/org/apache/iceberg/view/ViewProperties.java new file mode 100644 index 000000000000..07cd3cbf6567 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/view/ViewProperties.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.view; + +/** View properties that can be set during CREATE/REPLACE view or using updateProperties API. */ +public class ViewProperties { + public static final String VERSION_HISTORY_SIZE = "version.history.num-entries"; + public static final int VERSION_HISTORY_SIZE_DEFAULT = 10; + + private ViewProperties() {} +} diff --git a/core/src/main/java/org/apache/iceberg/view/ViewVersionParser.java b/core/src/main/java/org/apache/iceberg/view/ViewVersionParser.java index a0df4564985d..39783557e4b0 100644 --- a/core/src/main/java/org/apache/iceberg/view/ViewVersionParser.java +++ b/core/src/main/java/org/apache/iceberg/view/ViewVersionParser.java @@ -21,9 +21,13 @@ import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.databind.JsonNode; import java.io.IOException; +import java.util.Arrays; +import java.util.List; import java.util.Map; +import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.util.JsonUtil; class ViewVersionParser { @@ -33,6 +37,8 @@ class ViewVersionParser { private static final String SUMMARY = "summary"; private static final String REPRESENTATIONS = "representations"; private static final String SCHEMA_ID = "schema-id"; + private static final String DEFAULT_CATALOG = "default-catalog"; + private static final String DEFAULT_NAMESPACE = "default-namespace"; private ViewVersionParser() {} @@ -45,6 +51,15 @@ static void toJson(ViewVersion version, JsonGenerator generator) throws IOExcept generator.writeNumberField(SCHEMA_ID, version.schemaId()); JsonUtil.writeStringMap(SUMMARY, version.summary(), generator); + if (version.defaultCatalog() != null) { + generator.writeStringField(DEFAULT_CATALOG, version.defaultCatalog()); + } + + if (version.defaultNamespace() != null) { + JsonUtil.writeStringArray( + DEFAULT_NAMESPACE, Arrays.asList(version.defaultNamespace().levels()), generator); + } + generator.writeArrayFieldStart(REPRESENTATIONS); for (ViewRepresentation representation : version.representations()) { ViewRepresentationParser.toJson(representation, generator); @@ -81,11 +96,21 @@ static ViewVersion fromJson(JsonNode node) { representations.add(representation); } + String defaultCatalog = JsonUtil.getStringOrNull(DEFAULT_CATALOG, node); + + List namespace = JsonUtil.getStringListOrNull(DEFAULT_NAMESPACE, node); + Namespace defaultNamespace = null; + if (namespace != null && !namespace.isEmpty()) { + defaultNamespace = Namespace.of(Iterables.toArray(namespace, String.class)); + } + return ImmutableViewVersion.builder() .versionId(versionId) .timestampMillis(timestamp) .schemaId(schemaId) .summary(summary) + .defaultNamespace(defaultNamespace) + .defaultCatalog(defaultCatalog) .representations(representations.build()) .build(); } diff --git a/core/src/test/java/org/apache/iceberg/MetadataTableScanTestBase.java b/core/src/test/java/org/apache/iceberg/MetadataTableScanTestBase.java index 4c8f25c45bb2..b5ef31a50c0f 100644 --- a/core/src/test/java/org/apache/iceberg/MetadataTableScanTestBase.java +++ b/core/src/test/java/org/apache/iceberg/MetadataTableScanTestBase.java @@ -81,18 +81,20 @@ protected void validateTaskScanResiduals(TableScan scan, boolean ignoreResiduals } protected void validateSingleFieldPartition( - CloseableIterable> files, int partitionValue) { + CloseableIterable> files, int partitionValue) { validatePartition(files, 0, partitionValue); } protected void validatePartition( - CloseableIterable> files, int position, int partitionValue) { + CloseableIterable>> entries, + int position, + int partitionValue) { Assert.assertTrue( "File scan tasks do not include correct file", - StreamSupport.stream(files.spliterator(), false) + StreamSupport.stream(entries.spliterator(), false) .anyMatch( - file -> { - StructLike partition = file.partition(); + entry -> { + StructLike partition = entry.file().partition(); if (position >= partition.size()) { return false; } diff --git a/core/src/test/java/org/apache/iceberg/ScanTestBase.java b/core/src/test/java/org/apache/iceberg/ScanTestBase.java index f33893d8161b..70b4475bde60 100644 --- a/core/src/test/java/org/apache/iceberg/ScanTestBase.java +++ b/core/src/test/java/org/apache/iceberg/ScanTestBase.java @@ -31,6 +31,7 @@ import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Assume; import org.junit.Test; @@ -67,14 +68,14 @@ public void testTableScanHonorsSelect() { @Test public void testTableBothProjectAndSelect() { - AssertHelpers.assertThrows( - "Cannot set projection schema when columns are selected", - IllegalStateException.class, - () -> newScan().select(Arrays.asList("id")).project(SCHEMA.select("data"))); - AssertHelpers.assertThrows( - "Cannot select columns when projection schema is set", - IllegalStateException.class, - () -> newScan().project(SCHEMA.select("data")).select(Arrays.asList("id"))); + Assertions.assertThatThrownBy( + () -> newScan().select(Arrays.asList("id")).project(SCHEMA.select("data"))) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Cannot set projection schema when columns are selected"); + Assertions.assertThatThrownBy( + () -> newScan().project(SCHEMA.select("data")).select(Arrays.asList("id"))) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Cannot select columns when projection schema is set"); } @Test diff --git a/core/src/test/java/org/apache/iceberg/TableTestBase.java b/core/src/test/java/org/apache/iceberg/TableTestBase.java index 038dfadbff05..a800214bc9a7 100644 --- a/core/src/test/java/org/apache/iceberg/TableTestBase.java +++ b/core/src/test/java/org/apache/iceberg/TableTestBase.java @@ -57,7 +57,7 @@ public class TableTestBase { protected static final int BUCKETS_NUMBER = 16; // Partition spec used to create tables - protected static final PartitionSpec SPEC = + public static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).bucket("data", BUCKETS_NUMBER).build(); static final DataFile FILE_A = diff --git a/core/src/test/java/org/apache/iceberg/TestBaseIncrementalAppendScan.java b/core/src/test/java/org/apache/iceberg/TestBaseIncrementalAppendScan.java index 00feaf80abdf..58365555930f 100644 --- a/core/src/test/java/org/apache/iceberg/TestBaseIncrementalAppendScan.java +++ b/core/src/test/java/org/apache/iceberg/TestBaseIncrementalAppendScan.java @@ -19,6 +19,7 @@ package org.apache.iceberg; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Test; @@ -127,23 +128,21 @@ public void testMultipleRootSnapshots() throws Exception { // scan should fail because snapshot B is not an ancestor of snapshot D IncrementalAppendScan scanShouldFail = newScan().fromSnapshotExclusive(snapshotBId).toSnapshot(snapshotDId); - AssertHelpers.assertThrows( - "Should throw exception", - IllegalArgumentException.class, - String.format( - "Starting snapshot (exclusive) %d is not a parent ancestor of end snapshot %d", - snapshotBId, snapshotDId), - () -> Iterables.size(scanShouldFail.planFiles())); + Assertions.assertThatThrownBy(() -> Iterables.size(scanShouldFail.planFiles())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + String.format( + "Starting snapshot (exclusive) %d is not a parent ancestor of end snapshot %d", + snapshotBId, snapshotDId)); // scan should fail because snapshot B is not an ancestor of snapshot D IncrementalAppendScan scanShouldFailInclusive = newScan().fromSnapshotInclusive(snapshotBId).toSnapshot(snapshotDId); - AssertHelpers.assertThrows( - "Should throw exception", - IllegalArgumentException.class, - String.format( - "Starting snapshot (inclusive) %d is not an ancestor of end snapshot %d", - snapshotBId, snapshotDId), - () -> Iterables.size(scanShouldFailInclusive.planFiles())); + Assertions.assertThatThrownBy(() -> Iterables.size(scanShouldFailInclusive.planFiles())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + String.format( + "Starting snapshot (inclusive) %d is not an ancestor of end snapshot %d", + snapshotBId, snapshotDId)); } } diff --git a/core/src/test/java/org/apache/iceberg/TestBaseIncrementalChangelogScan.java b/core/src/test/java/org/apache/iceberg/TestBaseIncrementalChangelogScan.java index 1a1844345b21..dcda2f354a26 100644 --- a/core/src/test/java/org/apache/iceberg/TestBaseIncrementalChangelogScan.java +++ b/core/src/test/java/org/apache/iceberg/TestBaseIncrementalChangelogScan.java @@ -31,6 +31,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Assume; import org.junit.Test; @@ -256,11 +257,9 @@ public void testDeleteFilesAreNotSupported() { table.newRowDelta().addDeletes(FILE_A2_DELETES).commit(); - AssertHelpers.assertThrows( - "Should complain about delete files", - UnsupportedOperationException.class, - "Delete files are currently not supported", - () -> plan(newScan())); + Assertions.assertThatThrownBy(() -> plan(newScan())) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage("Delete files are currently not supported in changelog scans"); } // plans tasks and reorders them to have deterministic order diff --git a/core/src/test/java/org/apache/iceberg/TestCatalogUtil.java b/core/src/test/java/org/apache/iceberg/TestCatalogUtil.java index 75c27c7fc8cd..33172aa7f09c 100644 --- a/core/src/test/java/org/apache/iceberg/TestCatalogUtil.java +++ b/core/src/test/java/org/apache/iceberg/TestCatalogUtil.java @@ -34,8 +34,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestCatalogUtil { @@ -48,8 +47,8 @@ public void loadCustomCatalog() { Catalog catalog = CatalogUtil.loadCatalog(TestCatalog.class.getName(), name, options, hadoopConf); Assertions.assertThat(catalog).isInstanceOf(TestCatalog.class); - Assert.assertEquals(name, ((TestCatalog) catalog).catalogName); - Assert.assertEquals(options, ((TestCatalog) catalog).catalogProperties); + Assertions.assertThat(((TestCatalog) catalog).catalogName).isEqualTo(name); + Assertions.assertThat(((TestCatalog) catalog).catalogProperties).isEqualTo(options); } @Test @@ -62,9 +61,9 @@ public void loadCustomCatalog_withHadoopConfig() { Catalog catalog = CatalogUtil.loadCatalog(TestCatalogConfigurable.class.getName(), name, options, hadoopConf); Assertions.assertThat(catalog).isInstanceOf(TestCatalogConfigurable.class); - Assert.assertEquals(name, ((TestCatalogConfigurable) catalog).catalogName); - Assert.assertEquals(options, ((TestCatalogConfigurable) catalog).catalogProperties); - Assert.assertEquals(hadoopConf, ((TestCatalogConfigurable) catalog).configuration); + Assertions.assertThat(((TestCatalogConfigurable) catalog).catalogName).isEqualTo(name); + Assertions.assertThat(((TestCatalogConfigurable) catalog).catalogProperties).isEqualTo(options); + Assertions.assertThat(((TestCatalogConfigurable) catalog).configuration).isEqualTo(hadoopConf); } @Test @@ -73,13 +72,14 @@ public void loadCustomCatalog_NoArgConstructorNotFound() { options.put("key", "val"); Configuration hadoopConf = new Configuration(); String name = "custom"; - AssertHelpers.assertThrows( - "must have no-arg constructor", - IllegalArgumentException.class, - "NoSuchMethodException: org.apache.iceberg.TestCatalogUtil$TestCatalogBadConstructor.()", - () -> - CatalogUtil.loadCatalog( - TestCatalogBadConstructor.class.getName(), name, options, hadoopConf)); + Assertions.assertThatThrownBy( + () -> + CatalogUtil.loadCatalog( + TestCatalogBadConstructor.class.getName(), name, options, hadoopConf)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot initialize Catalog implementation") + .hasMessageContaining( + "NoSuchMethodException: org.apache.iceberg.TestCatalogUtil$TestCatalogBadConstructor.()"); } @Test @@ -89,13 +89,13 @@ public void loadCustomCatalog_NotImplementCatalog() { Configuration hadoopConf = new Configuration(); String name = "custom"; - AssertHelpers.assertThrows( - "must implement catalog", - IllegalArgumentException.class, - "does not implement Catalog", - () -> - CatalogUtil.loadCatalog( - TestCatalogNoInterface.class.getName(), name, options, hadoopConf)); + Assertions.assertThatThrownBy( + () -> + CatalogUtil.loadCatalog( + TestCatalogNoInterface.class.getName(), name, options, hadoopConf)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot initialize Catalog") + .hasMessageContaining("does not implement Catalog"); } @Test @@ -106,11 +106,10 @@ public void loadCustomCatalog_ConstructorErrorCatalog() { String name = "custom"; String impl = TestCatalogErrorConstructor.class.getName(); - AssertHelpers.assertThrows( - "must be able to initialize catalog", - IllegalArgumentException.class, - "NoClassDefFoundError: Error while initializing class", - () -> CatalogUtil.loadCatalog(impl, name, options, hadoopConf)); + Assertions.assertThatThrownBy(() -> CatalogUtil.loadCatalog(impl, name, options, hadoopConf)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot initialize Catalog implementation") + .hasMessageContaining("NoClassDefFoundError: Error while initializing class"); } @Test @@ -120,11 +119,10 @@ public void loadCustomCatalog_BadCatalogNameCatalog() { Configuration hadoopConf = new Configuration(); String name = "custom"; String impl = "CatalogDoesNotExist"; - AssertHelpers.assertThrows( - "catalog must exist", - IllegalArgumentException.class, - "java.lang.ClassNotFoundException: CatalogDoesNotExist", - () -> CatalogUtil.loadCatalog(impl, name, options, hadoopConf)); + Assertions.assertThatThrownBy(() -> CatalogUtil.loadCatalog(impl, name, options, hadoopConf)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot initialize Catalog implementation") + .hasMessageContaining("java.lang.ClassNotFoundException: CatalogDoesNotExist"); } @Test @@ -133,7 +131,7 @@ public void loadCustomFileIO_noArg() { properties.put("key", "val"); FileIO fileIO = CatalogUtil.loadFileIO(TestFileIONoArg.class.getName(), properties, null); Assertions.assertThat(fileIO).isInstanceOf(TestFileIONoArg.class); - Assert.assertEquals(properties, ((TestFileIONoArg) fileIO).map); + Assertions.assertThat(((TestFileIONoArg) fileIO).map).isEqualTo(properties); } @Test @@ -143,7 +141,7 @@ public void loadCustomFileIO_hadoopConfigConstructor() { FileIO fileIO = CatalogUtil.loadFileIO(HadoopFileIO.class.getName(), Maps.newHashMap(), configuration); Assertions.assertThat(fileIO).isInstanceOf(HadoopFileIO.class); - Assert.assertEquals("val", ((HadoopFileIO) fileIO).conf().get("key")); + Assertions.assertThat(((HadoopFileIO) fileIO).conf().get("key")).isEqualTo("val"); } @Test @@ -154,25 +152,25 @@ public void loadCustomFileIO_configurable() { CatalogUtil.loadFileIO( TestFileIOConfigurable.class.getName(), Maps.newHashMap(), configuration); Assertions.assertThat(fileIO).isInstanceOf(TestFileIOConfigurable.class); - Assert.assertEquals(configuration, ((TestFileIOConfigurable) fileIO).configuration); + Assertions.assertThat(((TestFileIOConfigurable) fileIO).configuration).isEqualTo(configuration); } @Test public void loadCustomFileIO_badArg() { - AssertHelpers.assertThrows( - "cannot find constructor", - IllegalArgumentException.class, - "missing no-arg constructor", - () -> CatalogUtil.loadFileIO(TestFileIOBadArg.class.getName(), Maps.newHashMap(), null)); + Assertions.assertThatThrownBy( + () -> CatalogUtil.loadFileIO(TestFileIOBadArg.class.getName(), Maps.newHashMap(), null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot initialize FileIO, missing no-arg constructor"); } @Test public void loadCustomFileIO_badClass() { - AssertHelpers.assertThrows( - "cannot cast", - IllegalArgumentException.class, - "does not implement FileIO", - () -> CatalogUtil.loadFileIO(TestFileIONotImpl.class.getName(), Maps.newHashMap(), null)); + Assertions.assertThatThrownBy( + () -> + CatalogUtil.loadFileIO(TestFileIONotImpl.class.getName(), Maps.newHashMap(), null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot initialize FileIO") + .hasMessageContaining("does not implement FileIO"); } @Test @@ -182,12 +180,10 @@ public void buildCustomCatalog_withTypeSet() { options.put(CatalogUtil.ICEBERG_CATALOG_TYPE, "hive"); Configuration hadoopConf = new Configuration(); String name = "custom"; - - AssertHelpers.assertThrows( - "Should complain about both configs being set", - IllegalArgumentException.class, - "both type and catalog-impl are set", - () -> CatalogUtil.buildIcebergCatalog(name, options, hadoopConf)); + Assertions.assertThatThrownBy(() -> CatalogUtil.buildIcebergCatalog(name, options, hadoopConf)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot create catalog custom, both type and catalog-impl are set: type=hive, catalog-impl=CustomCatalog"); } @Test diff --git a/core/src/test/java/org/apache/iceberg/TestContentFileParser.java b/core/src/test/java/org/apache/iceberg/TestContentFileParser.java new file mode 100644 index 000000000000..9360f571c5bb --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestContentFileParser.java @@ -0,0 +1,338 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import com.fasterxml.jackson.databind.JsonNode; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collections; +import java.util.stream.Stream; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Comparators; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.JsonUtil; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +public class TestContentFileParser { + @Test + public void testNullArguments() throws Exception { + Assertions.assertThatThrownBy(() -> ContentFileParser.toJson(null, TableTestBase.SPEC)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid content file: null"); + + Assertions.assertThatThrownBy(() -> ContentFileParser.toJson(TableTestBase.FILE_A, null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid partition spec: null"); + + Assertions.assertThatThrownBy( + () -> ContentFileParser.toJson(TableTestBase.FILE_A, TableTestBase.SPEC, null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid JSON generator: null"); + + Assertions.assertThatThrownBy(() -> ContentFileParser.fromJson(null, TableTestBase.SPEC)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid JSON node for content file: null"); + + String jsonStr = ContentFileParser.toJson(TableTestBase.FILE_A, TableTestBase.SPEC); + JsonNode jsonNode = JsonUtil.mapper().readTree(jsonStr); + Assertions.assertThatThrownBy(() -> ContentFileParser.fromJson(jsonNode, null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid partition spec: null"); + } + + @ParameterizedTest + @MethodSource("provideSpecAndDataFile") + public void testDataFile(PartitionSpec spec, DataFile dataFile, String expectedJson) + throws Exception { + String jsonStr = ContentFileParser.toJson(dataFile, spec); + Assertions.assertThat(jsonStr).isEqualTo(expectedJson); + JsonNode jsonNode = JsonUtil.mapper().readTree(jsonStr); + ContentFile deserializedContentFile = ContentFileParser.fromJson(jsonNode, spec); + Assertions.assertThat(deserializedContentFile).isInstanceOf(DataFile.class); + assertContentFileEquals(dataFile, deserializedContentFile, spec); + } + + @ParameterizedTest + @MethodSource("provideSpecAndDeleteFile") + public void testDeleteFile(PartitionSpec spec, DeleteFile deleteFile, String expectedJson) + throws Exception { + String jsonStr = ContentFileParser.toJson(deleteFile, spec); + Assertions.assertThat(jsonStr).isEqualTo(expectedJson); + JsonNode jsonNode = JsonUtil.mapper().readTree(jsonStr); + ContentFile deserializedContentFile = ContentFileParser.fromJson(jsonNode, spec); + Assertions.assertThat(deserializedContentFile).isInstanceOf(DeleteFile.class); + assertContentFileEquals(deleteFile, deserializedContentFile, spec); + } + + private static Stream provideSpecAndDataFile() { + return Stream.of( + Arguments.of( + PartitionSpec.unpartitioned(), + dataFileWithRequiredOnly(PartitionSpec.unpartitioned()), + dataFileJsonWithRequiredOnly(PartitionSpec.unpartitioned())), + Arguments.of( + PartitionSpec.unpartitioned(), + dataFileWithAllOptional(PartitionSpec.unpartitioned()), + dataFileJsonWithAllOptional(PartitionSpec.unpartitioned())), + Arguments.of( + TableTestBase.SPEC, + dataFileWithRequiredOnly(TableTestBase.SPEC), + dataFileJsonWithRequiredOnly(TableTestBase.SPEC)), + Arguments.of( + TableTestBase.SPEC, + dataFileWithAllOptional(TableTestBase.SPEC), + dataFileJsonWithAllOptional(TableTestBase.SPEC))); + } + + private static DataFile dataFileWithRequiredOnly(PartitionSpec spec) { + DataFiles.Builder builder = + DataFiles.builder(spec) + .withPath("/path/to/data-a.parquet") + .withFileSizeInBytes(10) + .withRecordCount(1); + + if (spec.isPartitioned()) { + // easy way to set partition data for now + builder.withPartitionPath("data_bucket=1"); + } + + return builder.build(); + } + + private static String dataFileJsonWithRequiredOnly(PartitionSpec spec) { + if (spec.isUnpartitioned()) { + return "{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/path/to/data-a.parquet\",\"file-format\":\"PARQUET\"," + + "\"partition\":{},\"file-size-in-bytes\":10,\"record-count\":1,\"sort-order-id\":0}"; + } else { + return "{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/path/to/data-a.parquet\",\"file-format\":\"PARQUET\"," + + "\"partition\":{\"1000\":1},\"file-size-in-bytes\":10,\"record-count\":1,\"sort-order-id\":0}"; + } + } + + private static String dataFileJsonWithAllOptional(PartitionSpec spec) { + if (spec.isUnpartitioned()) { + return "{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/path/to/data-with-stats.parquet\"," + + "\"file-format\":\"PARQUET\",\"partition\":{},\"file-size-in-bytes\":350,\"record-count\":10," + + "\"column-sizes\":{\"keys\":[3,4],\"values\":[100,200]}," + + "\"value-counts\":{\"keys\":[3,4],\"values\":[90,180]}," + + "\"null-value-counts\":{\"keys\":[3,4],\"values\":[10,20]}," + + "\"nan-value-counts\":{\"keys\":[3,4],\"values\":[0,0]}," + + "\"lower-bounds\":{\"keys\":[3,4],\"values\":[\"01000000\",\"02000000\"]}," + + "\"upper-bounds\":{\"keys\":[3,4],\"values\":[\"05000000\",\"0A000000\"]}," + + "\"key-metadata\":\"00000000000000000000000000000000\"," + + "\"split-offsets\":[128,256],\"equality-ids\":[1],\"sort-order-id\":1}"; + } else { + return "{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/path/to/data-with-stats.parquet\"," + + "\"file-format\":\"PARQUET\",\"partition\":{\"1000\":1},\"file-size-in-bytes\":350,\"record-count\":10," + + "\"column-sizes\":{\"keys\":[3,4],\"values\":[100,200]}," + + "\"value-counts\":{\"keys\":[3,4],\"values\":[90,180]}," + + "\"null-value-counts\":{\"keys\":[3,4],\"values\":[10,20]}," + + "\"nan-value-counts\":{\"keys\":[3,4],\"values\":[0,0]}," + + "\"lower-bounds\":{\"keys\":[3,4],\"values\":[\"01000000\",\"02000000\"]}," + + "\"upper-bounds\":{\"keys\":[3,4],\"values\":[\"05000000\",\"0A000000\"]}," + + "\"key-metadata\":\"00000000000000000000000000000000\"," + + "\"split-offsets\":[128,256],\"equality-ids\":[1],\"sort-order-id\":1}"; + } + } + + private static DataFile dataFileWithAllOptional(PartitionSpec spec) { + DataFiles.Builder builder = + DataFiles.builder(spec) + .withPath("/path/to/data-with-stats.parquet") + .withMetrics( + new Metrics( + 10L, // record count + ImmutableMap.of(3, 100L, 4, 200L), // column sizes + ImmutableMap.of(3, 90L, 4, 180L), // value counts + ImmutableMap.of(3, 10L, 4, 20L), // null value counts + ImmutableMap.of(3, 0L, 4, 0L), // nan value counts + ImmutableMap.of( + 3, + Conversions.toByteBuffer(Types.IntegerType.get(), 1), + 4, + Conversions.toByteBuffer(Types.IntegerType.get(), 2)), // lower bounds + ImmutableMap.of( + 3, + Conversions.toByteBuffer(Types.IntegerType.get(), 5), + 4, + Conversions.toByteBuffer(Types.IntegerType.get(), 10)) // upperbounds + )) + .withFileSizeInBytes(350) + .withSplitOffsets(Arrays.asList(128L, 256L)) + .withEqualityFieldIds(Collections.singletonList(1)) + .withEncryptionKeyMetadata(ByteBuffer.wrap(new byte[16])) + .withSortOrder( + SortOrder.builderFor(TableTestBase.SCHEMA) + .withOrderId(1) + .sortBy("id", SortDirection.ASC, NullOrder.NULLS_FIRST) + .build()); + + if (spec.isPartitioned()) { + // easy way to set partition data for now + builder.withPartitionPath("data_bucket=1"); + } + + return builder.build(); + } + + private static Stream provideSpecAndDeleteFile() { + return Stream.of( + Arguments.of( + PartitionSpec.unpartitioned(), + deleteFileWithRequiredOnly(PartitionSpec.unpartitioned()), + deleteFileJsonWithRequiredOnly(PartitionSpec.unpartitioned())), + Arguments.of( + PartitionSpec.unpartitioned(), + deleteFileWithAllOptional(PartitionSpec.unpartitioned()), + deleteFileJsonWithAllOptional(PartitionSpec.unpartitioned())), + Arguments.of( + TableTestBase.SPEC, + deleteFileWithRequiredOnly(TableTestBase.SPEC), + deleteFileJsonWithRequiredOnly(TableTestBase.SPEC)), + Arguments.of( + TableTestBase.SPEC, + deleteFileWithAllOptional(TableTestBase.SPEC), + deleteFileJsonWithAllOptional(TableTestBase.SPEC))); + } + + private static DeleteFile deleteFileWithRequiredOnly(PartitionSpec spec) { + PartitionData partitionData = null; + if (spec.isPartitioned()) { + partitionData = new PartitionData(spec.partitionType()); + partitionData.set(0, 9); + } + + return new GenericDeleteFile( + spec.specId(), + FileContent.POSITION_DELETES, + "/path/to/delete-a.parquet", + FileFormat.PARQUET, + partitionData, + 1234, + new Metrics(9L, null, null, null, null), + null, + null, + null, + null); + } + + private static DeleteFile deleteFileWithAllOptional(PartitionSpec spec) { + PartitionData partitionData = new PartitionData(spec.partitionType()); + if (spec.isPartitioned()) { + partitionData.set(0, 9); + } + + Metrics metrics = + new Metrics( + 10L, // record count + ImmutableMap.of(3, 100L, 4, 200L), // column sizes + ImmutableMap.of(3, 90L, 4, 180L), // value counts + ImmutableMap.of(3, 10L, 4, 20L), // null value counts + ImmutableMap.of(3, 0L, 4, 0L), // nan value counts + ImmutableMap.of( + 3, + Conversions.toByteBuffer(Types.IntegerType.get(), 1), + 4, + Conversions.toByteBuffer(Types.IntegerType.get(), 2)), // lower bounds + ImmutableMap.of( + 3, + Conversions.toByteBuffer(Types.IntegerType.get(), 5), + 4, + Conversions.toByteBuffer(Types.IntegerType.get(), 10)) // upperbounds + ); + + return new GenericDeleteFile( + spec.specId(), + FileContent.EQUALITY_DELETES, + "/path/to/delete-with-stats.parquet", + FileFormat.PARQUET, + partitionData, + 1234, + metrics, + new int[] {3}, + 1, + Arrays.asList(128L), + ByteBuffer.wrap(new byte[16])); + } + + private static String deleteFileJsonWithRequiredOnly(PartitionSpec spec) { + if (spec.isUnpartitioned()) { + return "{\"spec-id\":0,\"content\":\"POSITION_DELETES\",\"file-path\":\"/path/to/delete-a.parquet\"," + + "\"file-format\":\"PARQUET\",\"partition\":{},\"file-size-in-bytes\":1234,\"record-count\":9}"; + } else { + return "{\"spec-id\":0,\"content\":\"POSITION_DELETES\",\"file-path\":\"/path/to/delete-a.parquet\"," + + "\"file-format\":\"PARQUET\",\"partition\":{\"1000\":9},\"file-size-in-bytes\":1234,\"record-count\":9}"; + } + } + + private static String deleteFileJsonWithAllOptional(PartitionSpec spec) { + if (spec.isUnpartitioned()) { + return "{\"spec-id\":0,\"content\":\"EQUALITY_DELETES\",\"file-path\":\"/path/to/delete-with-stats.parquet\"," + + "\"file-format\":\"PARQUET\",\"partition\":{},\"file-size-in-bytes\":1234,\"record-count\":10," + + "\"column-sizes\":{\"keys\":[3,4],\"values\":[100,200]}," + + "\"value-counts\":{\"keys\":[3,4],\"values\":[90,180]}," + + "\"null-value-counts\":{\"keys\":[3,4],\"values\":[10,20]}," + + "\"nan-value-counts\":{\"keys\":[3,4],\"values\":[0,0]}," + + "\"lower-bounds\":{\"keys\":[3,4],\"values\":[\"01000000\",\"02000000\"]}," + + "\"upper-bounds\":{\"keys\":[3,4],\"values\":[\"05000000\",\"0A000000\"]}," + + "\"key-metadata\":\"00000000000000000000000000000000\"," + + "\"split-offsets\":[128],\"equality-ids\":[3],\"sort-order-id\":1}"; + } else { + return "{\"spec-id\":0,\"content\":\"EQUALITY_DELETES\",\"file-path\":\"/path/to/delete-with-stats.parquet\"," + + "\"file-format\":\"PARQUET\",\"partition\":{\"1000\":9},\"file-size-in-bytes\":1234,\"record-count\":10," + + "\"column-sizes\":{\"keys\":[3,4],\"values\":[100,200]}," + + "\"value-counts\":{\"keys\":[3,4],\"values\":[90,180]}," + + "\"null-value-counts\":{\"keys\":[3,4],\"values\":[10,20]}," + + "\"nan-value-counts\":{\"keys\":[3,4],\"values\":[0,0]}," + + "\"lower-bounds\":{\"keys\":[3,4],\"values\":[\"01000000\",\"02000000\"]}," + + "\"upper-bounds\":{\"keys\":[3,4],\"values\":[\"05000000\",\"0A000000\"]}," + + "\"key-metadata\":\"00000000000000000000000000000000\"," + + "\"split-offsets\":[128],\"equality-ids\":[3],\"sort-order-id\":1}"; + } + } + + static void assertContentFileEquals( + ContentFile expected, ContentFile actual, PartitionSpec spec) { + Assertions.assertThat(actual.getClass()).isEqualTo(expected.getClass()); + Assertions.assertThat(actual.specId()).isEqualTo(expected.specId()); + Assertions.assertThat(actual.content()).isEqualTo(expected.content()); + Assertions.assertThat(actual.path()).isEqualTo(expected.path()); + Assertions.assertThat(actual.format()).isEqualTo(expected.format()); + Assertions.assertThat(actual.partition()) + .usingComparator(Comparators.forType(spec.partitionType())) + .isEqualTo(expected.partition()); + Assertions.assertThat(actual.recordCount()).isEqualTo(expected.recordCount()); + Assertions.assertThat(actual.fileSizeInBytes()).isEqualTo(expected.fileSizeInBytes()); + Assertions.assertThat(actual.columnSizes()).isEqualTo(expected.columnSizes()); + Assertions.assertThat(actual.valueCounts()).isEqualTo(expected.valueCounts()); + Assertions.assertThat(actual.nullValueCounts()).isEqualTo(expected.nullValueCounts()); + Assertions.assertThat(actual.nanValueCounts()).isEqualTo(expected.nanValueCounts()); + Assertions.assertThat(actual.lowerBounds()).isEqualTo(expected.lowerBounds()); + Assertions.assertThat(actual.upperBounds()).isEqualTo(expected.upperBounds()); + Assertions.assertThat(actual.keyMetadata()).isEqualTo(expected.keyMetadata()); + Assertions.assertThat(actual.splitOffsets()).isEqualTo(expected.splitOffsets()); + Assertions.assertThat(actual.equalityFieldIds()).isEqualTo(expected.equalityFieldIds()); + Assertions.assertThat(actual.sortOrderId()).isEqualTo(expected.sortOrderId()); + } +} diff --git a/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java b/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java index 157e1233232b..4240184e913c 100644 --- a/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java +++ b/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java @@ -28,6 +28,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; @@ -300,11 +301,9 @@ public void testCreateDetectsUncommittedChange() throws IOException { txn.updateProperties().set("test-property", "test-value"); // not committed - AssertHelpers.assertThrows( - "Should reject commit when last operation has not committed", - IllegalStateException.class, - "Cannot create new DeleteFiles: last operation has not committed", - txn::newDelete); + Assertions.assertThatThrownBy(txn::newDelete) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Cannot create new DeleteFiles: last operation has not committed"); } @Test @@ -323,11 +322,9 @@ public void testCreateDetectsUncommittedChangeOnCommit() throws IOException { txn.updateProperties().set("test-property", "test-value"); // not committed - AssertHelpers.assertThrows( - "Should reject commit when last operation has not committed", - IllegalStateException.class, - "Cannot commit transaction: last operation has not committed", - txn::commitTransaction); + Assertions.assertThatThrownBy(txn::commitTransaction) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Cannot commit transaction: last operation has not committed"); } @Test @@ -360,11 +357,9 @@ public void testCreateTransactionConflict() throws IOException { Assert.assertFalse( "Table should not have any snapshots", conflict.snapshots().iterator().hasNext()); - AssertHelpers.assertThrows( - "Transaction commit should fail", - CommitFailedException.class, - "Commit failed: table was updated", - txn::commitTransaction); + Assertions.assertThatThrownBy(txn::commitTransaction) + .isInstanceOf(CommitFailedException.class) + .hasMessageStartingWith("Commit failed: table was updated"); Assert.assertEquals( "Should clean up metadata", diff --git a/core/src/test/java/org/apache/iceberg/TestDataTableScan.java b/core/src/test/java/org/apache/iceberg/TestDataTableScan.java index e221310a012d..8541a96c8d43 100644 --- a/core/src/test/java/org/apache/iceberg/TestDataTableScan.java +++ b/core/src/test/java/org/apache/iceberg/TestDataTableScan.java @@ -25,6 +25,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Assume; import org.junit.Test; @@ -124,11 +125,10 @@ public void testScanFromRefWhenSnapshotSetFails() { table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); table.manageSnapshots().createTag("tagB", table.currentSnapshot().snapshotId()).commit(); - AssertHelpers.assertThrows( - "Should throw when attempting to use a ref for scanning when a snapshot is set", - IllegalArgumentException.class, - "Cannot override ref, already set snapshot id=1", - () -> table.newScan().useSnapshot(table.currentSnapshot().snapshotId()).useRef("tagB")); + Assertions.assertThatThrownBy( + () -> table.newScan().useSnapshot(table.currentSnapshot().snapshotId()).useRef("tagB")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot override ref, already set snapshot id=1"); } @Test @@ -138,11 +138,10 @@ public void testSettingSnapshotWhenRefSetFails() { table.newFastAppend().appendFile(FILE_B).commit(); table.manageSnapshots().createTag("tagB", table.currentSnapshot().snapshotId()).commit(); - AssertHelpers.assertThrows( - "Should throw when attempting to use a snapshot for scanning when a ref is set", - IllegalArgumentException.class, - "Cannot override snapshot, already set snapshot id=2", - () -> table.newScan().useRef("tagB").useSnapshot(snapshotA.snapshotId())); + Assertions.assertThatThrownBy( + () -> table.newScan().useRef("tagB").useSnapshot(snapshotA.snapshotId())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot override snapshot, already set snapshot id=2"); } @Test @@ -152,11 +151,11 @@ public void testBranchTimeTravelFails() { .manageSnapshots() .createBranch("testBranch", table.currentSnapshot().snapshotId()) .commit(); - AssertHelpers.assertThrows( - "Should throw when attempting to use a snapshot for scanning when a ref is set", - IllegalArgumentException.class, - "Cannot override snapshot, already set snapshot id=1", - () -> table.newScan().useRef("testBranch").asOfTime(System.currentTimeMillis())); + + Assertions.assertThatThrownBy( + () -> table.newScan().useRef("testBranch").asOfTime(System.currentTimeMillis())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot override snapshot, already set snapshot id=1"); } @Test @@ -166,20 +165,16 @@ public void testSettingMultipleRefsFails() { table.newFastAppend().appendFile(FILE_B).commit(); table.manageSnapshots().createTag("tagB", table.currentSnapshot().snapshotId()).commit(); - AssertHelpers.assertThrows( - "Should throw when attempting to use multiple refs", - IllegalArgumentException.class, - "Cannot override ref, already set snapshot id=2", - () -> table.newScan().useRef("tagB").useRef("tagA")); + Assertions.assertThatThrownBy(() -> table.newScan().useRef("tagB").useRef("tagA")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot override ref, already set snapshot id=2"); } @Test public void testSettingInvalidRefFails() { - AssertHelpers.assertThrows( - "Should throw when attempting to use an invalid ref for scanning", - IllegalArgumentException.class, - "Cannot find ref nonexisting", - () -> table.newScan().useRef("nonexisting")); + Assertions.assertThatThrownBy(() -> table.newScan().useRef("nonexisting")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot find ref nonexisting"); } private void validateExpectedFileScanTasks( diff --git a/core/src/test/java/org/apache/iceberg/TestDeleteFileIndex.java b/core/src/test/java/org/apache/iceberg/TestDeleteFileIndex.java index 95ce34ab0443..4e574690f707 100644 --- a/core/src/test/java/org/apache/iceberg/TestDeleteFileIndex.java +++ b/core/src/test/java/org/apache/iceberg/TestDeleteFileIndex.java @@ -25,12 +25,11 @@ import java.io.IOException; import java.util.Arrays; import java.util.List; +import java.util.UUID; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.util.Pair; -import org.apache.iceberg.util.StructLikeWrapper; import org.junit.Assert; import org.junit.Test; @@ -47,7 +46,6 @@ public TestDeleteFileIndex() { .withPartition(FILE_A.partition()) .withRecordCount(1) .build(); - static final DeleteFile FILE_A_POS_2 = FILE_A_POS_1.copy(); static final DeleteFile FILE_A_EQ_1 = FileMetadata.deleteFileBuilder(SPEC) @@ -57,9 +55,6 @@ public TestDeleteFileIndex() { .withPartition(FILE_A.partition()) .withRecordCount(1) .build(); - static final DeleteFile FILE_A_EQ_2 = FILE_A_EQ_1.copy(); - static final DeleteFile[] DELETE_FILES = - new DeleteFile[] {FILE_A_POS_1, FILE_A_EQ_1, FILE_A_POS_2, FILE_A_EQ_2}; private static DataFile unpartitionedFile(PartitionSpec spec) { return DataFiles.builder(spec) @@ -72,7 +67,17 @@ private static DataFile unpartitionedFile(PartitionSpec spec) { private static DeleteFile unpartitionedPosDeletes(PartitionSpec spec) { return FileMetadata.deleteFileBuilder(spec) .ofPositionDeletes() - .withPath("/path/to/data-unpartitioned-pos-deletes.parquet") + .withPath(UUID.randomUUID() + "/path/to/data-unpartitioned-pos-deletes.parquet") + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + } + + private static DeleteFile partitionedPosDeletes(PartitionSpec spec, StructLike partition) { + return FileMetadata.deleteFileBuilder(spec) + .ofPositionDeletes() + .withPartition(partition) + .withPath(UUID.randomUUID() + "/path/to/data-partitioned-pos-deletes.parquet") .withFileSizeInBytes(10) .withRecordCount(1) .build(); @@ -81,38 +86,81 @@ private static DeleteFile unpartitionedPosDeletes(PartitionSpec spec) { private static DeleteFile unpartitionedEqDeletes(PartitionSpec spec) { return FileMetadata.deleteFileBuilder(spec) .ofEqualityDeletes() - .withPath("/path/to/data-unpartitioned-eq-deletes.parquet") + .withPath(UUID.randomUUID() + "/path/to/data-unpartitioned-eq-deletes.parquet") + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + } + + private static DeleteFile partitionedEqDeletes(PartitionSpec spec, StructLike partition) { + return FileMetadata.deleteFileBuilder(spec) + .ofEqualityDeletes() + .withPartition(partition) + .withPath(UUID.randomUUID() + "/path/to/data-partitioned-eq-deletes.parquet") .withFileSizeInBytes(10) .withRecordCount(1) .build(); } + @SuppressWarnings("unchecked") + private static > F withDataSequenceNumber(long seq, F file) { + BaseFile baseFile = (BaseFile) file; + baseFile.setDataSequenceNumber(seq); + return file; + } + + @Test + public void testMinSequenceNumberFilteringForFiles() { + PartitionSpec partSpec = PartitionSpec.unpartitioned(); + + DeleteFile[] deleteFiles = { + withDataSequenceNumber(4, unpartitionedEqDeletes(partSpec)), + withDataSequenceNumber(6, unpartitionedEqDeletes(partSpec)) + }; + + DeleteFileIndex index = + DeleteFileIndex.builderFor(Arrays.asList(deleteFiles)) + .specsById(ImmutableMap.of(partSpec.specId(), partSpec, 1, SPEC)) + .afterSequenceNumber(4) + .build(); + + DataFile file = unpartitionedFile(partSpec); + + Assert.assertEquals("Only one delete file should apply", 1, index.forDataFile(0, file).length); + } + @Test public void testUnpartitionedDeletes() { PartitionSpec partSpec = PartitionSpec.unpartitioned(); + + DeleteFile[] deleteFiles = { + withDataSequenceNumber(4, unpartitionedEqDeletes(partSpec)), + withDataSequenceNumber(6, unpartitionedEqDeletes(partSpec)), + withDataSequenceNumber(5, unpartitionedPosDeletes(partSpec)), + withDataSequenceNumber(6, unpartitionedPosDeletes(partSpec)) + }; + DeleteFileIndex index = - new DeleteFileIndex( - ImmutableMap.of(partSpec.specId(), partSpec, 1, SPEC), - new long[] {3, 5, 5, 6}, - DELETE_FILES, - ImmutableMap.of()); + DeleteFileIndex.builderFor(Arrays.asList(deleteFiles)) + .specsById(ImmutableMap.of(partSpec.specId(), partSpec, 1, SPEC)) + .build(); DataFile unpartitionedFile = unpartitionedFile(partSpec); Assert.assertArrayEquals( - "All deletes should apply to seq 0", DELETE_FILES, index.forDataFile(0, unpartitionedFile)); + "All deletes should apply to seq 0", deleteFiles, index.forDataFile(0, unpartitionedFile)); Assert.assertArrayEquals( - "All deletes should apply to seq 3", DELETE_FILES, index.forDataFile(3, unpartitionedFile)); + "All deletes should apply to seq 3", deleteFiles, index.forDataFile(3, unpartitionedFile)); Assert.assertArrayEquals( "Last 3 deletes should apply to seq 4", - Arrays.copyOfRange(DELETE_FILES, 1, 4), + Arrays.copyOfRange(deleteFiles, 1, 4), index.forDataFile(4, unpartitionedFile)); Assert.assertArrayEquals( "Last 3 deletes should apply to seq 5", - Arrays.copyOfRange(DELETE_FILES, 1, 4), + Arrays.copyOfRange(deleteFiles, 1, 4), index.forDataFile(5, unpartitionedFile)); Assert.assertArrayEquals( "Last delete should apply to seq 6", - Arrays.copyOfRange(DELETE_FILES, 3, 4), + Arrays.copyOfRange(deleteFiles, 3, 4), index.forDataFile(6, unpartitionedFile)); Assert.assertArrayEquals( "No deletes should apply to seq 7", @@ -127,43 +175,40 @@ public void testUnpartitionedDeletes() { DataFile partitionedFileA = FILE_A.copy(); ((BaseFile) partitionedFileA).setSpecId(1); Assert.assertArrayEquals( - "All global deletes should apply to a partitioned file", - DELETE_FILES, + "All global equality deletes should apply to a partitioned file", + Arrays.copyOfRange(deleteFiles, 0, 2), index.forDataFile(0, partitionedFileA)); } @Test public void testPartitionedDeleteIndex() { + DeleteFile[] deleteFiles = { + withDataSequenceNumber(4, partitionedEqDeletes(SPEC, FILE_A.partition())), + withDataSequenceNumber(6, partitionedEqDeletes(SPEC, FILE_A.partition())), + withDataSequenceNumber(5, partitionedPosDeletes(SPEC, FILE_A.partition())), + withDataSequenceNumber(6, partitionedPosDeletes(SPEC, FILE_A.partition())) + }; + DeleteFileIndex index = - new DeleteFileIndex( - ImmutableMap.of(SPEC.specId(), SPEC, 1, PartitionSpec.unpartitioned()), - null, - null, - ImmutableMap.of( - Pair.of( - SPEC.specId(), - StructLikeWrapper.forType(SPEC.partitionType()).set(FILE_A.partition())), - Pair.of(new long[] {3, 5, 5, 6}, DELETE_FILES), - Pair.of( - SPEC.specId(), - StructLikeWrapper.forType(SPEC.partitionType()).set(FILE_C.partition())), - Pair.of(new long[0], new DeleteFile[0]))); + DeleteFileIndex.builderFor(Arrays.asList(deleteFiles)) + .specsById(ImmutableMap.of(SPEC.specId(), SPEC, 1, PartitionSpec.unpartitioned())) + .build(); Assert.assertArrayEquals( - "All deletes should apply to seq 0", DELETE_FILES, index.forDataFile(0, FILE_A)); + "All deletes should apply to seq 0", deleteFiles, index.forDataFile(0, FILE_A)); Assert.assertArrayEquals( - "All deletes should apply to seq 3", DELETE_FILES, index.forDataFile(3, FILE_A)); + "All deletes should apply to seq 3", deleteFiles, index.forDataFile(3, FILE_A)); Assert.assertArrayEquals( "Last 3 deletes should apply to seq 4", - Arrays.copyOfRange(DELETE_FILES, 1, 4), + Arrays.copyOfRange(deleteFiles, 1, 4), index.forDataFile(4, FILE_A)); Assert.assertArrayEquals( "Last 3 deletes should apply to seq 5", - Arrays.copyOfRange(DELETE_FILES, 1, 4), + Arrays.copyOfRange(deleteFiles, 1, 4), index.forDataFile(5, FILE_A)); Assert.assertArrayEquals( "Last delete should apply to seq 6", - Arrays.copyOfRange(DELETE_FILES, 3, 4), + Arrays.copyOfRange(deleteFiles, 3, 4), index.forDataFile(6, FILE_A)); Assert.assertArrayEquals( "No deletes should apply to seq 7", new DataFile[0], index.forDataFile(7, FILE_A)); diff --git a/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java b/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java index f5370d1aa8ca..4e4565306c00 100644 --- a/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java +++ b/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java @@ -34,6 +34,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.StructLikeWrapper; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Assume; import org.junit.Test; @@ -284,15 +285,14 @@ public void testCannotDeleteFileWhereNotAllRowsMatchPartitionFilter() { commit(table, table.newFastAppend().appendFile(dataFile), branch); - AssertHelpers.assertThrows( - "Should reject as not all rows match filter", - ValidationException.class, - "Cannot delete file where some, but not all, rows match filter", - () -> - commit( - table, - table.newDelete().deleteFromRowFilter(Expressions.equal("data", "aa")), - branch)); + Assertions.assertThatThrownBy( + () -> + commit( + table, + table.newDelete().deleteFromRowFilter(Expressions.equal("data", "aa")), + branch)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Cannot delete file where some, but not all, rows match filter"); } @Test @@ -301,21 +301,19 @@ public void testDeleteCaseSensitivity() { Expression rowFilter = Expressions.lessThan("iD", 5); - AssertHelpers.assertThrows( - "Should use case sensitive binding by default", - ValidationException.class, - "Cannot find field 'iD'", - () -> commit(table, table.newDelete().deleteFromRowFilter(rowFilter), branch)); - - AssertHelpers.assertThrows( - "Should fail with case sensitive binding", - ValidationException.class, - "Cannot find field 'iD'", - () -> - commit( - table, - table.newDelete().deleteFromRowFilter(rowFilter).caseSensitive(true), - branch)); + Assertions.assertThatThrownBy( + () -> commit(table, table.newDelete().deleteFromRowFilter(rowFilter), branch)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Cannot find field 'iD'"); + + Assertions.assertThatThrownBy( + () -> + commit( + table, + table.newDelete().deleteFromRowFilter(rowFilter).caseSensitive(true), + branch)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Cannot find field 'iD'"); Snapshot deleteSnapshot = commit( diff --git a/core/src/test/java/org/apache/iceberg/TestEnvironmentContext.java b/core/src/test/java/org/apache/iceberg/TestEnvironmentContext.java index f6cb9c699bd2..52d38fa97061 100644 --- a/core/src/test/java/org/apache/iceberg/TestEnvironmentContext.java +++ b/core/src/test/java/org/apache/iceberg/TestEnvironmentContext.java @@ -19,7 +19,7 @@ package org.apache.iceberg; import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestEnvironmentContext { diff --git a/core/src/test/java/org/apache/iceberg/TestFastAppend.java b/core/src/test/java/org/apache/iceberg/TestFastAppend.java index 13445d8e338e..6ed52fd5ad35 100644 --- a/core/src/test/java/org/apache/iceberg/TestFastAppend.java +++ b/core/src/test/java/org/apache/iceberg/TestFastAppend.java @@ -239,11 +239,9 @@ public void testFailure() { ManifestFile newManifest = pending.allManifests(FILE_IO).get(0); Assert.assertTrue("Should create new manifest", new File(newManifest.path()).exists()); - AssertHelpers.assertThrows( - "Should retry 4 times and throw last failure", - CommitFailedException.class, - "Injected failure", - append::commit); + Assertions.assertThatThrownBy(append::commit) + .isInstanceOf(CommitFailedException.class) + .hasMessage("Injected failure"); Assert.assertFalse("Should clean up new manifest", new File(newManifest.path()).exists()); } @@ -260,11 +258,9 @@ public void testAppendManifestCleanup() throws IOException { ManifestFile newManifest = pending.allManifests(FILE_IO).get(0); Assert.assertTrue("Should create new manifest", new File(newManifest.path()).exists()); - AssertHelpers.assertThrows( - "Should retry 4 times and throw last failure", - CommitFailedException.class, - "Injected failure", - append::commit); + Assertions.assertThatThrownBy(append::commit) + .isInstanceOf(CommitFailedException.class) + .hasMessage("Injected failure"); Assert.assertFalse("Should clean up new manifest", new File(newManifest.path()).exists()); } @@ -376,8 +372,9 @@ public void testAppendManifestFailureWithSnapshotIdInheritance() throws IOExcept AppendFiles append = table.newAppend(); append.appendManifest(manifest); - AssertHelpers.assertThrows( - "Should reject commit", CommitFailedException.class, "Injected failure", append::commit); + Assertions.assertThatThrownBy(append::commit) + .isInstanceOf(CommitFailedException.class) + .hasMessage("Injected failure"); Assert.assertTrue("Append manifest should not be deleted", new File(manifest.path()).exists()); } @@ -391,19 +388,17 @@ public void testInvalidAppendManifest() throws IOException { ManifestFile manifestWithExistingFiles = writeManifest("manifest-file-1.avro", manifestEntry(Status.EXISTING, null, FILE_A)); - AssertHelpers.assertThrows( - "Should reject commit", - IllegalArgumentException.class, - "Cannot append manifest with existing files", - () -> table.newFastAppend().appendManifest(manifestWithExistingFiles).commit()); + Assertions.assertThatThrownBy( + () -> table.newFastAppend().appendManifest(manifestWithExistingFiles).commit()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot append manifest with existing files"); ManifestFile manifestWithDeletedFiles = writeManifest("manifest-file-2.avro", manifestEntry(Status.DELETED, null, FILE_A)); - AssertHelpers.assertThrows( - "Should reject commit", - IllegalArgumentException.class, - "Cannot append manifest with deleted files", - () -> table.newFastAppend().appendManifest(manifestWithDeletedFiles).commit()); + Assertions.assertThatThrownBy( + () -> table.newFastAppend().appendManifest(manifestWithDeletedFiles).commit()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot append manifest with deleted files"); } @Test diff --git a/core/src/test/java/org/apache/iceberg/TestFileScanTaskParser.java b/core/src/test/java/org/apache/iceberg/TestFileScanTaskParser.java new file mode 100644 index 000000000000..42785c7a778e --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestFileScanTaskParser.java @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import org.apache.iceberg.expressions.ExpressionUtil; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.ResidualEvaluator; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +public class TestFileScanTaskParser { + @Test + public void testNullArguments() { + Assertions.assertThatThrownBy(() -> FileScanTaskParser.toJson(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid file scan task: null"); + + Assertions.assertThatThrownBy(() -> FileScanTaskParser.fromJson(null, true)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid JSON string for file scan task: null"); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testParser(boolean caseSensitive) { + PartitionSpec spec = TableTestBase.SPEC; + FileScanTask fileScanTask = createScanTask(spec, caseSensitive); + String jsonStr = FileScanTaskParser.toJson(fileScanTask); + Assertions.assertThat(jsonStr).isEqualTo(expectedFileScanTaskJson()); + FileScanTask deserializedTask = FileScanTaskParser.fromJson(jsonStr, caseSensitive); + assertFileScanTaskEquals(fileScanTask, deserializedTask, spec, caseSensitive); + } + + private FileScanTask createScanTask(PartitionSpec spec, boolean caseSensitive) { + ResidualEvaluator residualEvaluator; + if (spec.isUnpartitioned()) { + residualEvaluator = ResidualEvaluator.unpartitioned(Expressions.alwaysTrue()); + } else { + residualEvaluator = ResidualEvaluator.of(spec, Expressions.equal("id", 1), caseSensitive); + } + + return new BaseFileScanTask( + TableTestBase.FILE_A, + new DeleteFile[] {TableTestBase.FILE_A_DELETES, TableTestBase.FILE_A2_DELETES}, + SchemaParser.toJson(TableTestBase.SCHEMA), + PartitionSpecParser.toJson(spec), + residualEvaluator); + } + + private String expectedFileScanTaskJson() { + return "{\"schema\":{\"type\":\"struct\",\"schema-id\":0,\"fields\":[" + + "{\"id\":3,\"name\":\"id\",\"required\":true,\"type\":\"int\"}," + + "{\"id\":4,\"name\":\"data\",\"required\":true,\"type\":\"string\"}]}," + + "\"spec\":{\"spec-id\":0,\"fields\":[{\"name\":\"data_bucket\"," + + "\"transform\":\"bucket[16]\",\"source-id\":4,\"field-id\":1000}]}," + + "\"data-file\":{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/path/to/data-a.parquet\"," + + "\"file-format\":\"PARQUET\",\"partition\":{\"1000\":0}," + + "\"file-size-in-bytes\":10,\"record-count\":1,\"sort-order-id\":0}," + + "\"start\":0,\"length\":10," + + "\"delete-files\":[{\"spec-id\":0,\"content\":\"POSITION_DELETES\"," + + "\"file-path\":\"/path/to/data-a-deletes.parquet\",\"file-format\":\"PARQUET\"," + + "\"partition\":{\"1000\":0},\"file-size-in-bytes\":10,\"record-count\":1}," + + "{\"spec-id\":0,\"content\":\"EQUALITY_DELETES\",\"file-path\":\"/path/to/data-a2-deletes.parquet\"," + + "\"file-format\":\"PARQUET\",\"partition\":{\"1000\":0},\"file-size-in-bytes\":10," + + "\"record-count\":1,\"equality-ids\":[1],\"sort-order-id\":0}]," + + "\"residual-filter\":{\"type\":\"eq\",\"term\":\"id\",\"value\":1}}"; + } + + private static void assertFileScanTaskEquals( + FileScanTask expected, FileScanTask actual, PartitionSpec spec, boolean caseSensitive) { + TestContentFileParser.assertContentFileEquals(expected.file(), actual.file(), spec); + Assertions.assertThat(actual.deletes()).hasSameSizeAs(expected.deletes()); + for (int pos = 0; pos < expected.deletes().size(); ++pos) { + TestContentFileParser.assertContentFileEquals( + expected.deletes().get(pos), actual.deletes().get(pos), spec); + } + + Assertions.assertThat(expected.schema().sameSchema(actual.schema())) + .as("Schema should match") + .isTrue(); + Assertions.assertThat(actual.spec()).isEqualTo(expected.spec()); + Assertions.assertThat( + ExpressionUtil.equivalent( + expected.residual(), + actual.residual(), + TableTestBase.SCHEMA.asStruct(), + caseSensitive)) + .as("Residual expression should match") + .isTrue(); + } +} diff --git a/core/src/test/java/org/apache/iceberg/TestFixedSizeSplitScanTaskIterator.java b/core/src/test/java/org/apache/iceberg/TestFixedSizeSplitScanTaskIterator.java index b08449e3a7d3..314e805b827e 100644 --- a/core/src/test/java/org/apache/iceberg/TestFixedSizeSplitScanTaskIterator.java +++ b/core/src/test/java/org/apache/iceberg/TestFixedSizeSplitScanTaskIterator.java @@ -21,8 +21,8 @@ import java.util.List; import org.apache.iceberg.BaseFileScanTask.SplitScanTask; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.Test; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; public class TestFixedSizeSplitScanTaskIterator { @Test @@ -57,8 +57,8 @@ private static void verify(long splitSize, long fileLen, List> offset List split = offsetLenPairs.get(i); long offset = split.get(0); long length = split.get(1); - Assert.assertEquals(offset, task.start()); - Assert.assertEquals(length, task.length()); + Assertions.assertThat(task.start()).isEqualTo(offset); + Assertions.assertThat(task.length()).isEqualTo(length); } } diff --git a/core/src/test/java/org/apache/iceberg/TestFormatVersions.java b/core/src/test/java/org/apache/iceberg/TestFormatVersions.java index f3d31e4db33e..b4f80088d2cc 100644 --- a/core/src/test/java/org/apache/iceberg/TestFormatVersions.java +++ b/core/src/test/java/org/apache/iceberg/TestFormatVersions.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Test; @@ -48,11 +49,9 @@ public void testFormatVersionDowngrade() { Assert.assertEquals("Should report v2", 2, ops.current().formatVersion()); - AssertHelpers.assertThrows( - "Should reject a version downgrade", - IllegalArgumentException.class, - "Cannot downgrade", - () -> ops.current().upgradeToFormatVersion(1)); + Assertions.assertThatThrownBy(() -> ops.current().upgradeToFormatVersion(1)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot downgrade v2 table to v1"); Assert.assertEquals("Should report v2", 2, ops.current().formatVersion()); } @@ -61,14 +60,14 @@ public void testFormatVersionDowngrade() { public void testFormatVersionUpgradeNotSupported() { TableOperations ops = table.ops(); TableMetadata base = ops.current(); - AssertHelpers.assertThrows( - "Should reject an unsupported version upgrade", - IllegalArgumentException.class, - "Cannot upgrade table to unsupported format version", - () -> - ops.commit( - base, - base.upgradeToFormatVersion(TableMetadata.SUPPORTED_TABLE_FORMAT_VERSION + 1))); + + Assertions.assertThatThrownBy( + () -> + ops.commit( + base, + base.upgradeToFormatVersion(TableMetadata.SUPPORTED_TABLE_FORMAT_VERSION + 1))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot upgrade table to unsupported format version: v3 (supported: v2)"); Assert.assertEquals("Should report v1", 1, ops.current().formatVersion()); } diff --git a/core/src/test/java/org/apache/iceberg/TestIncrementalDataTableScan.java b/core/src/test/java/org/apache/iceberg/TestIncrementalDataTableScan.java index 1e6678fc339c..63f0ed6dd727 100644 --- a/core/src/test/java/org/apache/iceberg/TestIncrementalDataTableScan.java +++ b/core/src/test/java/org/apache/iceberg/TestIncrementalDataTableScan.java @@ -31,6 +31,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.Sets; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -56,26 +57,20 @@ public void setupTableProperties() { @Test public void testInvalidScans() { add(table.newAppend(), files("A")); - AssertHelpers.assertThrows( - "from and to snapshots cannot be the same, since from snapshot is exclusive and not part of the scan", - IllegalArgumentException.class, - "from and to snapshot ids cannot be the same", - () -> appendsBetweenScan(1, 1)); + Assertions.assertThatThrownBy(() -> appendsBetweenScan(1, 1)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("from and to snapshot ids cannot be the same"); add(table.newAppend(), files("B")); add(table.newAppend(), files("C")); add(table.newAppend(), files("D")); add(table.newAppend(), files("E")); - AssertHelpers.assertThrows( - "Check refinement api", - IllegalArgumentException.class, - "from snapshot id 1 not in existing snapshot ids range (2, 4]", - () -> table.newScan().appendsBetween(2, 5).appendsBetween(1, 4)); - AssertHelpers.assertThrows( - "Check refinement api", - IllegalArgumentException.class, - "to snapshot id 3 not in existing snapshot ids range (1, 2]", - () -> table.newScan().appendsBetween(1, 2).appendsBetween(1, 3)); + Assertions.assertThatThrownBy(() -> table.newScan().appendsBetween(2, 5).appendsBetween(1, 4)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("from snapshot id 1 not in existing snapshot ids range (2, 4]"); + Assertions.assertThatThrownBy(() -> table.newScan().appendsBetween(1, 2).appendsBetween(1, 3)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("to snapshot id 3 not in existing snapshot ids range (1, 2]"); } @Test @@ -140,11 +135,10 @@ public void testReplaceOverwritesDeletes() { filesMatch(Lists.newArrayList("I"), appendsBetweenScan(7, 8)); overwrite(table.newOverwrite(), files("H"), files("E")); // 9 - AssertHelpers.assertThrows( - "Overwrites are not supported for Incremental scan", - UnsupportedOperationException.class, - "Found overwrite operation, cannot support incremental data in snapshots (8, 9]", - () -> appendsBetweenScan(8, 9)); + Assertions.assertThatThrownBy(() -> appendsBetweenScan(8, 9)) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage( + "Found overwrite operation, cannot support incremental data in snapshots (8, 9]"); } @Test diff --git a/core/src/test/java/org/apache/iceberg/TestLocationProvider.java b/core/src/test/java/org/apache/iceberg/TestLocationProvider.java index 2b9cc47e9389..6afc7f0fe715 100644 --- a/core/src/test/java/org/apache/iceberg/TestLocationProvider.java +++ b/core/src/test/java/org/apache/iceberg/TestLocationProvider.java @@ -22,6 +22,7 @@ import java.util.Map; import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.relocated.com.google.common.base.Splitter; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; @@ -172,13 +173,16 @@ public void testDynamicallyLoadedLocationProviderNotFound() { .set(TableProperties.WRITE_LOCATION_PROVIDER_IMPL, nonExistentImpl) .commit(); - AssertHelpers.assertThrows( - "Non-existent implementation should fail on finding constructor", - IllegalArgumentException.class, - String.format( - "Unable to find a constructor for implementation %s of %s. ", - nonExistentImpl, LocationProvider.class), - () -> table.locationProvider()); + Assertions.assertThatThrownBy(() -> table.locationProvider()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith( + String.format( + "Unable to find a constructor for implementation %s of %s. ", + nonExistentImpl, LocationProvider.class)) + .hasMessageEndingWith( + "Make sure the implementation is in classpath, and that it either " + + "has a public no-arg constructor or a two-arg constructor " + + "taking in the string base table location and its property string map."); } @Test @@ -193,13 +197,12 @@ public void testInvalidNoInterfaceDynamicallyLoadedLocationProvider() { .set(TableProperties.WRITE_LOCATION_PROVIDER_IMPL, invalidImpl) .commit(); - AssertHelpers.assertThrows( - "Class with missing interface implementation should fail on instantiation.", - IllegalArgumentException.class, - String.format( - "Provided implementation for dynamic instantiation should implement %s", - LocationProvider.class), - () -> table.locationProvider()); + Assertions.assertThatThrownBy(() -> table.locationProvider()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + String.format( + "Provided implementation for dynamic instantiation should implement %s.", + LocationProvider.class)); } @Test @@ -214,13 +217,12 @@ public void testInvalidArgTypesDynamicallyLoadedLocationProvider() { .set(TableProperties.WRITE_LOCATION_PROVIDER_IMPL, invalidImpl) .commit(); - AssertHelpers.assertThrows( - "Implementation with invalid arg types should fail on finding constructor", - IllegalArgumentException.class, - String.format( - "Unable to find a constructor for implementation %s of %s. ", - invalidImpl, LocationProvider.class), - () -> table.locationProvider()); + Assertions.assertThatThrownBy(() -> table.locationProvider()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith( + String.format( + "Unable to find a constructor for implementation %s of %s. ", + invalidImpl, LocationProvider.class)); } @Test diff --git a/core/src/test/java/org/apache/iceberg/TestManifestListVersions.java b/core/src/test/java/org/apache/iceberg/TestManifestListVersions.java index eab3d3109421..a95379e4ca33 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestListVersions.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestListVersions.java @@ -23,7 +23,9 @@ import java.nio.ByteBuffer; import java.util.Collection; import java.util.List; +import org.apache.avro.AvroRuntimeException; import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecordBuilder; import org.apache.iceberg.avro.Avro; import org.apache.iceberg.avro.AvroSchemaUtil; @@ -37,6 +39,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Conversions; import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -99,11 +102,9 @@ public class TestManifestListVersions { @Test public void testV1WriteDeleteManifest() { - AssertHelpers.assertThrows( - "Should fail to write a DELETE manifest to v1", - IllegalArgumentException.class, - "Cannot store delete manifests in a v1 table", - () -> writeManifestList(TEST_DELETE_MANIFEST, 1)); + Assertions.assertThatThrownBy(() -> writeManifestList(TEST_DELETE_MANIFEST, 1)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot store delete manifests in a v1 table"); } @Test @@ -173,9 +174,9 @@ public void testV1ForwardCompatibility() throws IOException { "Existing rows count", EXISTING_ROWS, (long) generic.get("existing_rows_count")); Assert.assertEquals( "Deleted rows count", DELETED_ROWS, (long) generic.get("deleted_rows_count")); - AssertHelpers.assertEmptyAvroField(generic, ManifestFile.MANIFEST_CONTENT.name()); - AssertHelpers.assertEmptyAvroField(generic, ManifestFile.SEQUENCE_NUMBER.name()); - AssertHelpers.assertEmptyAvroField(generic, ManifestFile.MIN_SEQUENCE_NUMBER.name()); + assertEmptyAvroField(generic, ManifestFile.MANIFEST_CONTENT.name()); + assertEmptyAvroField(generic, ManifestFile.SEQUENCE_NUMBER.name()); + assertEmptyAvroField(generic, ManifestFile.MIN_SEQUENCE_NUMBER.name()); } @Test @@ -201,9 +202,9 @@ public void testV2ForwardCompatibility() throws IOException { "Existing rows count", EXISTING_ROWS, (long) generic.get("existing_rows_count")); Assert.assertEquals( "Deleted rows count", DELETED_ROWS, (long) generic.get("deleted_rows_count")); - AssertHelpers.assertEmptyAvroField(generic, ManifestFile.MANIFEST_CONTENT.name()); - AssertHelpers.assertEmptyAvroField(generic, ManifestFile.SEQUENCE_NUMBER.name()); - AssertHelpers.assertEmptyAvroField(generic, ManifestFile.MIN_SEQUENCE_NUMBER.name()); + assertEmptyAvroField(generic, ManifestFile.MANIFEST_CONTENT.name()); + assertEmptyAvroField(generic, ManifestFile.SEQUENCE_NUMBER.name()); + assertEmptyAvroField(generic, ManifestFile.MIN_SEQUENCE_NUMBER.name()); } @Test @@ -360,4 +361,10 @@ private ManifestFile writeAndReadManifestList(int formatVersion) throws IOExcept Assert.assertEquals("Should contain one manifest", 1, manifests.size()); return manifests.get(0); } + + private void assertEmptyAvroField(GenericRecord record, String field) { + Assertions.assertThatThrownBy(() -> record.get(field)) + .isInstanceOf(AvroRuntimeException.class) + .hasMessage("Not a valid schema field: " + field); + } } diff --git a/core/src/test/java/org/apache/iceberg/TestManifestReader.java b/core/src/test/java/org/apache/iceberg/TestManifestReader.java index dfc84200fdb2..09ed1e448560 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestReader.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestReader.java @@ -27,6 +27,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Streams; import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Assume; import org.junit.Test; @@ -76,11 +77,9 @@ public void testReaderWithFilterWithoutSelect() throws IOException { @Test public void testInvalidUsage() throws IOException { ManifestFile manifest = writeManifest(FILE_A, FILE_B); - AssertHelpers.assertThrows( - "Should not be possible to read manifest without explicit snapshot ids and inheritable metadata", - IllegalArgumentException.class, - "Cannot read from ManifestFile with null (unassigned) snapshot ID", - () -> ManifestFiles.read(manifest, FILE_IO)); + Assertions.assertThatThrownBy(() -> ManifestFiles.read(manifest, FILE_IO)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot read from ManifestFile with null (unassigned) snapshot ID"); } @Test diff --git a/core/src/test/java/org/apache/iceberg/TestManifestReaderStats.java b/core/src/test/java/org/apache/iceberg/TestManifestReaderStats.java index e794539d6d5f..082800238a0a 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestReaderStats.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestReaderStats.java @@ -275,9 +275,6 @@ private void assertStatsDropped(DataFile dataFile) { private void assertNullRecordCount(DataFile dataFile) { // record count is a primitive type, accessing null record count will throw NPE - AssertHelpers.assertThrows( - "Should throw NPE when accessing non-populated record count field", - NullPointerException.class, - dataFile::recordCount); + Assertions.assertThatThrownBy(dataFile::recordCount).isInstanceOf(NullPointerException.class); } } diff --git a/core/src/test/java/org/apache/iceberg/TestManifestWriter.java b/core/src/test/java/org/apache/iceberg/TestManifestWriter.java index 245ad1b8176c..17a41f418a8e 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestWriter.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestWriter.java @@ -20,12 +20,15 @@ import java.io.File; import java.io.IOException; +import java.io.UncheckedIOException; import java.util.List; import java.util.UUID; import org.apache.iceberg.ManifestEntry.Status; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.types.Conversions; import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; +import org.assertj.core.api.Assumptions; import org.junit.Assert; import org.junit.Assume; import org.junit.Test; @@ -43,6 +46,9 @@ public TestManifestWriter(int formatVersion) { super(formatVersion); } + private static final int FILE_SIZE_CHECK_ROWS_DIVISOR = 250; + private static final long SMALL_FILE_SIZE = 10L; + @Test public void testManifestStats() throws IOException { ManifestFile manifest = @@ -218,6 +224,166 @@ public void testCommitManifestWithExistingEntriesWithoutFileSequenceNumber() thr statuses(Status.EXISTING, Status.EXISTING)); } + @Test + public void testRollingManifestWriterNoRecords() throws IOException { + RollingManifestWriter writer = newRollingWriteManifest(SMALL_FILE_SIZE); + + writer.close(); + Assertions.assertThat(writer.toManifestFiles()).isEmpty(); + + writer.close(); + Assertions.assertThat(writer.toManifestFiles()).isEmpty(); + } + + @Test + public void testRollingDeleteManifestWriterNoRecords() throws IOException { + Assumptions.assumeThat(formatVersion).isGreaterThan(1); + RollingManifestWriter writer = newRollingWriteDeleteManifest(SMALL_FILE_SIZE); + + writer.close(); + Assertions.assertThat(writer.toManifestFiles()).isEmpty(); + + writer.close(); + Assertions.assertThat(writer.toManifestFiles()).isEmpty(); + } + + @Test + public void testRollingManifestWriterSplitFiles() throws IOException { + RollingManifestWriter writer = newRollingWriteManifest(SMALL_FILE_SIZE); + + int[] addedFileCounts = new int[3]; + int[] existingFileCounts = new int[3]; + int[] deletedFileCounts = new int[3]; + long[] addedRowCounts = new long[3]; + long[] existingRowCounts = new long[3]; + long[] deletedRowCounts = new long[3]; + + for (int i = 0; i < FILE_SIZE_CHECK_ROWS_DIVISOR * 3; i++) { + int type = i % 3; + int fileIndex = i / FILE_SIZE_CHECK_ROWS_DIVISOR; + if (type == 0) { + writer.add(newFile(i)); + addedFileCounts[fileIndex] += 1; + addedRowCounts[fileIndex] += i; + } else if (type == 1) { + writer.existing(newFile(i), 1, 1, null); + existingFileCounts[fileIndex] += 1; + existingRowCounts[fileIndex] += i; + } else { + writer.delete(newFile(i), 1, null); + deletedFileCounts[fileIndex] += 1; + deletedRowCounts[fileIndex] += i; + } + } + + writer.close(); + List manifestFiles = writer.toManifestFiles(); + Assertions.assertThat(manifestFiles.size()).isEqualTo(3); + + checkManifests( + manifestFiles, + addedFileCounts, + existingFileCounts, + deletedFileCounts, + addedRowCounts, + existingRowCounts, + deletedRowCounts); + + writer.close(); + manifestFiles = writer.toManifestFiles(); + Assertions.assertThat(manifestFiles.size()).isEqualTo(3); + + checkManifests( + manifestFiles, + addedFileCounts, + existingFileCounts, + deletedFileCounts, + addedRowCounts, + existingRowCounts, + deletedRowCounts); + } + + @Test + public void testRollingDeleteManifestWriterSplitFiles() throws IOException { + Assumptions.assumeThat(formatVersion).isGreaterThan(1); + RollingManifestWriter writer = newRollingWriteDeleteManifest(SMALL_FILE_SIZE); + + int[] addedFileCounts = new int[3]; + int[] existingFileCounts = new int[3]; + int[] deletedFileCounts = new int[3]; + long[] addedRowCounts = new long[3]; + long[] existingRowCounts = new long[3]; + long[] deletedRowCounts = new long[3]; + for (int i = 0; i < 3 * FILE_SIZE_CHECK_ROWS_DIVISOR; i++) { + int type = i % 3; + int fileIndex = i / FILE_SIZE_CHECK_ROWS_DIVISOR; + if (type == 0) { + writer.add(newPosDeleteFile(i)); + addedFileCounts[fileIndex] += 1; + addedRowCounts[fileIndex] += i; + } else if (type == 1) { + writer.existing(newPosDeleteFile(i), 1, 1, null); + existingFileCounts[fileIndex] += 1; + existingRowCounts[fileIndex] += i; + } else { + writer.delete(newPosDeleteFile(i), 1, null); + deletedFileCounts[fileIndex] += 1; + deletedRowCounts[fileIndex] += i; + } + } + + writer.close(); + List manifestFiles = writer.toManifestFiles(); + Assertions.assertThat(manifestFiles.size()).isEqualTo(3); + + checkManifests( + manifestFiles, + addedFileCounts, + existingFileCounts, + deletedFileCounts, + addedRowCounts, + existingRowCounts, + deletedRowCounts); + + writer.close(); + manifestFiles = writer.toManifestFiles(); + Assertions.assertThat(manifestFiles.size()).isEqualTo(3); + + checkManifests( + manifestFiles, + addedFileCounts, + existingFileCounts, + deletedFileCounts, + addedRowCounts, + existingRowCounts, + deletedRowCounts); + } + + private void checkManifests( + List manifests, + int[] addedFileCounts, + int[] existingFileCounts, + int[] deletedFileCounts, + long[] addedRowCounts, + long[] existingRowCounts, + long[] deletedRowCounts) { + for (int i = 0; i < manifests.size(); i++) { + ManifestFile manifest = manifests.get(i); + + Assertions.assertThat(manifest.hasAddedFiles()).isTrue(); + Assertions.assertThat(manifest.addedFilesCount()).isEqualTo(addedFileCounts[i]); + Assertions.assertThat(manifest.addedRowsCount()).isEqualTo(addedRowCounts[i]); + + Assertions.assertThat(manifest.hasExistingFiles()).isTrue(); + Assertions.assertThat(manifest.existingFilesCount()).isEqualTo(existingFileCounts[i]); + Assertions.assertThat(manifest.existingRowsCount()).isEqualTo(existingRowCounts[i]); + + Assertions.assertThat(manifest.hasDeletedFiles()).isTrue(); + Assertions.assertThat(manifest.deletedFilesCount()).isEqualTo(deletedFileCounts[i]); + Assertions.assertThat(manifest.deletedRowsCount()).isEqualTo(deletedRowCounts[i]); + } + } + private DataFile newFile(long recordCount) { return newFile(recordCount, null); } @@ -234,4 +400,39 @@ private DataFile newFile(long recordCount, StructLike partition) { } return builder.build(); } + + private DeleteFile newPosDeleteFile(long recordCount) { + return FileMetadata.deleteFileBuilder(SPEC) + .ofPositionDeletes() + .withPath("/path/to/delete-" + UUID.randomUUID() + ".parquet") + .withFileSizeInBytes(10) + .withRecordCount(recordCount) + .build(); + } + + private RollingManifestWriter newRollingWriteManifest(long targetFileSize) { + return new RollingManifestWriter<>( + () -> { + OutputFile newManifestFile = newManifestFile(); + return ManifestFiles.write(formatVersion, SPEC, newManifestFile, null); + }, + targetFileSize); + } + + private RollingManifestWriter newRollingWriteDeleteManifest(long targetFileSize) { + return new RollingManifestWriter<>( + () -> { + OutputFile newManifestFile = newManifestFile(); + return ManifestFiles.writeDeleteManifest(formatVersion, SPEC, newManifestFile, null); + }, + targetFileSize); + } + + private OutputFile newManifestFile() { + try { + return Files.localOutput(FileFormat.AVRO.addExtension(temp.newFile().toString())); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } } diff --git a/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java b/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java index 7184b4969cfc..08b27d7460da 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java @@ -33,6 +33,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Conversions; import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -77,7 +78,7 @@ public class TestManifestWriterVersions { private static final DataFile DATA_FILE = new GenericDataFile( - 0, PATH, FORMAT, PARTITION, 150972L, METRICS, null, OFFSETS, SORT_ORDER_ID); + 0, PATH, FORMAT, PARTITION, 150972L, METRICS, null, OFFSETS, null, SORT_ORDER_ID); private static final List EQUALITY_IDS = ImmutableList.of(1); private static final int[] EQUALITY_ID_ARR = new int[] {1}; @@ -111,11 +112,9 @@ public void testV1Write() throws IOException { @Test public void testV1WriteDelete() { - AssertHelpers.assertThrows( - "Should fail to write a delete manifest for v1", - IllegalArgumentException.class, - "Cannot write delete files in a v1 table", - () -> writeDeleteManifest(1)); + Assertions.assertThatThrownBy(() -> writeDeleteManifest(1)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot write delete files in a v1 table"); } @Test diff --git a/core/src/test/java/org/apache/iceberg/TestMergeAppend.java b/core/src/test/java/org/apache/iceberg/TestMergeAppend.java index 46dd9ecbc59f..892d92634c09 100644 --- a/core/src/test/java/org/apache/iceberg/TestMergeAppend.java +++ b/core/src/test/java/org/apache/iceberg/TestMergeAppend.java @@ -33,6 +33,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; @@ -945,11 +946,9 @@ public void testFailure() { ids(pending.snapshotId(), baseId), concat(files(FILE_B), files(initialManifest))); - AssertHelpers.assertThrows( - "Should retry 4 times and throw last failure", - CommitFailedException.class, - "Injected failure", - () -> commit(table, append, branch)); + Assertions.assertThatThrownBy(() -> commit(table, append, branch)) + .isInstanceOf(CommitFailedException.class) + .hasMessage("Injected failure"); V2Assert.assertEquals( "Last sequence number should be 1", 1, readMetadata().lastSequenceNumber()); @@ -983,11 +982,9 @@ public void testAppendManifestCleanup() throws IOException { ManifestFile newManifest = pending.allManifests(table.io()).get(0); Assert.assertTrue("Should create new manifest", new File(newManifest.path()).exists()); - AssertHelpers.assertThrows( - "Should retry 4 times and throw last failure", - CommitFailedException.class, - "Injected failure", - () -> commit(table, append, branch)); + Assertions.assertThatThrownBy(() -> commit(table, append, branch)) + .isInstanceOf(CommitFailedException.class) + .hasMessage("Injected failure"); V2Assert.assertEquals( "Last sequence number should be 0", 0, readMetadata().lastSequenceNumber()); V1Assert.assertEquals( @@ -1186,11 +1183,9 @@ public void testAppendManifestFailureWithSnapshotIdInheritance() throws IOExcept AppendFiles append = table.newAppend(); append.appendManifest(manifest); - AssertHelpers.assertThrows( - "Should reject commit", - CommitFailedException.class, - "Injected failure", - () -> commit(table, append, branch)); + Assertions.assertThatThrownBy(() -> commit(table, append, branch)) + .isInstanceOf(CommitFailedException.class) + .hasMessage("Injected failure"); Assert.assertEquals("Last sequence number should be 0", 0, readMetadata().lastSequenceNumber()); Assert.assertTrue("Append manifest should not be deleted", new File(manifest.path()).exists()); @@ -1205,20 +1200,19 @@ public void testInvalidAppendManifest() throws IOException { ManifestFile manifestWithExistingFiles = writeManifest("manifest-file-1.avro", manifestEntry(Status.EXISTING, null, FILE_A)); - AssertHelpers.assertThrows( - "Should reject commit", - IllegalArgumentException.class, - "Cannot append manifest with existing files", - () -> commit(table, table.newAppend().appendManifest(manifestWithExistingFiles), branch)); + Assertions.assertThatThrownBy( + () -> + commit(table, table.newAppend().appendManifest(manifestWithExistingFiles), branch)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot append manifest with existing files"); Assert.assertEquals("Last sequence number should be 0", 0, readMetadata().lastSequenceNumber()); ManifestFile manifestWithDeletedFiles = writeManifest("manifest-file-2.avro", manifestEntry(Status.DELETED, null, FILE_A)); - AssertHelpers.assertThrows( - "Should reject commit", - IllegalArgumentException.class, - "Cannot append manifest with deleted files", - () -> commit(table, table.newAppend().appendManifest(manifestWithDeletedFiles), branch)); + Assertions.assertThatThrownBy( + () -> commit(table, table.newAppend().appendManifest(manifestWithDeletedFiles), branch)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot append manifest with deleted files"); Assert.assertEquals("Last sequence number should be 0", 0, readMetadata().lastSequenceNumber()); } diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java b/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java index 9f3ee5c9c3f4..2e34f2e6da4d 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java @@ -317,18 +317,18 @@ public void testPartitionsTableScanNoFilter() { TableScan scanNoFilter = partitionsTable.newScan().select("partition.data_bucket"); Assert.assertEquals(expected, scanNoFilter.schema().asStruct()); - CloseableIterable> files = - PartitionsTable.planFiles((StaticTableScan) scanNoFilter); + CloseableIterable> entries = + PartitionsTable.planEntries((StaticTableScan) scanNoFilter); if (formatVersion == 2) { - Assert.assertEquals(8, Iterators.size(files.iterator())); + Assert.assertEquals(8, Iterators.size(entries.iterator())); } else { - Assert.assertEquals(4, Iterators.size(files.iterator())); + Assert.assertEquals(4, Iterators.size(entries.iterator())); } - validateSingleFieldPartition(files, 0); - validateSingleFieldPartition(files, 1); - validateSingleFieldPartition(files, 2); - validateSingleFieldPartition(files, 3); + validateSingleFieldPartition(entries, 0); + validateSingleFieldPartition(entries, 1); + validateSingleFieldPartition(entries, 2); + validateSingleFieldPartition(entries, 3); } @Test @@ -342,18 +342,18 @@ public void testPartitionsTableScanWithProjection() { TableScan scanWithProjection = partitionsTable.newScan().select("file_count"); Assert.assertEquals(expected, scanWithProjection.schema().asStruct()); - CloseableIterable> files = - PartitionsTable.planFiles((StaticTableScan) scanWithProjection); + CloseableIterable> entries = + PartitionsTable.planEntries((StaticTableScan) scanWithProjection); if (formatVersion == 2) { - Assert.assertEquals(8, Iterators.size(files.iterator())); + Assert.assertEquals(8, Iterators.size(entries.iterator())); } else { - Assert.assertEquals(4, Iterators.size(files.iterator())); + Assert.assertEquals(4, Iterators.size(entries.iterator())); } - validateSingleFieldPartition(files, 0); - validateSingleFieldPartition(files, 1); - validateSingleFieldPartition(files, 2); - validateSingleFieldPartition(files, 3); + validateSingleFieldPartition(entries, 0); + validateSingleFieldPartition(entries, 1); + validateSingleFieldPartition(entries, 2); + validateSingleFieldPartition(entries, 3); } @Test @@ -361,14 +361,15 @@ public void testPartitionsTableScanNoStats() { table.newFastAppend().appendFile(FILE_WITH_STATS).commit(); Table partitionsTable = new PartitionsTable(table); - CloseableIterable> tasksAndEq = - PartitionsTable.planFiles((StaticTableScan) partitionsTable.newScan()); - for (ContentFile file : tasksAndEq) { - Assert.assertNull(file.columnSizes()); - Assert.assertNull(file.valueCounts()); - Assert.assertNull(file.nullValueCounts()); - Assert.assertNull(file.lowerBounds()); - Assert.assertNull(file.upperBounds()); + CloseableIterable> tasksAndEq = + PartitionsTable.planEntries((StaticTableScan) partitionsTable.newScan()); + for (ManifestEntry> task : tasksAndEq) { + Assert.assertNull(task.file().columnSizes()); + Assert.assertNull(task.file().valueCounts()); + Assert.assertNull(task.file().nullValueCounts()); + Assert.assertNull(task.file().nanValueCounts()); + Assert.assertNull(task.file().lowerBounds()); + Assert.assertNull(task.file().upperBounds()); } } @@ -383,15 +384,15 @@ public void testPartitionsTableScanAndFilter() { Expressions.equal("partition.data_bucket", 0), Expressions.greaterThan("record_count", 0)); TableScan scanAndEq = partitionsTable.newScan().filter(andEquals); - CloseableIterable> files = - PartitionsTable.planFiles((StaticTableScan) scanAndEq); + CloseableIterable> entries = + PartitionsTable.planEntries((StaticTableScan) scanAndEq); if (formatVersion == 2) { - Assert.assertEquals(2, Iterators.size(files.iterator())); + Assert.assertEquals(2, Iterators.size(entries.iterator())); } else { - Assert.assertEquals(1, Iterators.size(files.iterator())); + Assert.assertEquals(1, Iterators.size(entries.iterator())); } - validateSingleFieldPartition(files, 0); + validateSingleFieldPartition(entries, 0); } @Test @@ -405,16 +406,16 @@ public void testPartitionsTableScanLtFilter() { Expressions.lessThan("partition.data_bucket", 2), Expressions.greaterThan("record_count", 0)); TableScan scanLtAnd = partitionsTable.newScan().filter(ltAnd); - CloseableIterable> files = - PartitionsTable.planFiles((StaticTableScan) scanLtAnd); + CloseableIterable> entries = + PartitionsTable.planEntries((StaticTableScan) scanLtAnd); if (formatVersion == 2) { - Assert.assertEquals(4, Iterators.size(files.iterator())); + Assert.assertEquals(4, Iterators.size(entries.iterator())); } else { - Assert.assertEquals(2, Iterators.size(files.iterator())); + Assert.assertEquals(2, Iterators.size(entries.iterator())); } - validateSingleFieldPartition(files, 0); - validateSingleFieldPartition(files, 1); + validateSingleFieldPartition(entries, 0); + validateSingleFieldPartition(entries, 1); } @Test @@ -429,17 +430,18 @@ public void testPartitionsTableScanOrFilter() { Expressions.greaterThan("record_count", 0)); TableScan scanOr = partitionsTable.newScan().filter(or); - CloseableIterable> files = PartitionsTable.planFiles((StaticTableScan) scanOr); + CloseableIterable> entries = + PartitionsTable.planEntries((StaticTableScan) scanOr); if (formatVersion == 2) { - Assert.assertEquals(8, Iterators.size(files.iterator())); + Assert.assertEquals(8, Iterators.size(entries.iterator())); } else { - Assert.assertEquals(4, Iterators.size(files.iterator())); + Assert.assertEquals(4, Iterators.size(entries.iterator())); } - validateSingleFieldPartition(files, 0); - validateSingleFieldPartition(files, 1); - validateSingleFieldPartition(files, 2); - validateSingleFieldPartition(files, 3); + validateSingleFieldPartition(entries, 0); + validateSingleFieldPartition(entries, 1); + validateSingleFieldPartition(entries, 2); + validateSingleFieldPartition(entries, 3); } @Test @@ -449,15 +451,16 @@ public void testPartitionsScanNotFilter() { Expression not = Expressions.not(Expressions.lessThan("partition.data_bucket", 2)); TableScan scanNot = partitionsTable.newScan().filter(not); - CloseableIterable> files = PartitionsTable.planFiles((StaticTableScan) scanNot); + CloseableIterable> entries = + PartitionsTable.planEntries((StaticTableScan) scanNot); if (formatVersion == 2) { - Assert.assertEquals(4, Iterators.size(files.iterator())); + Assert.assertEquals(4, Iterators.size(entries.iterator())); } else { - Assert.assertEquals(2, Iterators.size(files.iterator())); + Assert.assertEquals(2, Iterators.size(entries.iterator())); } - validateSingleFieldPartition(files, 2); - validateSingleFieldPartition(files, 3); + validateSingleFieldPartition(entries, 2); + validateSingleFieldPartition(entries, 3); } @Test @@ -468,15 +471,16 @@ public void testPartitionsTableScanInFilter() { Expression set = Expressions.in("partition.data_bucket", 2, 3); TableScan scanSet = partitionsTable.newScan().filter(set); - CloseableIterable> files = PartitionsTable.planFiles((StaticTableScan) scanSet); + CloseableIterable> entries = + PartitionsTable.planEntries((StaticTableScan) scanSet); if (formatVersion == 2) { - Assert.assertEquals(4, Iterators.size(files.iterator())); + Assert.assertEquals(4, Iterators.size(entries.iterator())); } else { - Assert.assertEquals(2, Iterators.size(files.iterator())); + Assert.assertEquals(2, Iterators.size(entries.iterator())); } - validateSingleFieldPartition(files, 2); - validateSingleFieldPartition(files, 3); + validateSingleFieldPartition(entries, 2); + validateSingleFieldPartition(entries, 3); } @Test @@ -487,18 +491,18 @@ public void testPartitionsTableScanNotNullFilter() { Expression unary = Expressions.notNull("partition.data_bucket"); TableScan scanUnary = partitionsTable.newScan().filter(unary); - CloseableIterable> files = - PartitionsTable.planFiles((StaticTableScan) scanUnary); + CloseableIterable> entries = + PartitionsTable.planEntries((StaticTableScan) scanUnary); if (formatVersion == 2) { - Assert.assertEquals(8, Iterators.size(files.iterator())); + Assert.assertEquals(8, Iterators.size(entries.iterator())); } else { - Assert.assertEquals(4, Iterators.size(files.iterator())); + Assert.assertEquals(4, Iterators.size(entries.iterator())); } - validateSingleFieldPartition(files, 0); - validateSingleFieldPartition(files, 1); - validateSingleFieldPartition(files, 2); - validateSingleFieldPartition(files, 3); + validateSingleFieldPartition(entries, 0); + validateSingleFieldPartition(entries, 1); + validateSingleFieldPartition(entries, 2); + validateSingleFieldPartition(entries, 3); } @Test @@ -788,13 +792,14 @@ public void testPartitionSpecEvolutionAdditive() { Expressions.and( Expressions.equal("partition.id", 10), Expressions.greaterThan("record_count", 0)); TableScan scan = metadataTable.newScan().filter(filter); - CloseableIterable> files = PartitionsTable.planFiles((StaticTableScan) scan); + CloseableIterable> entries = + PartitionsTable.planEntries((StaticTableScan) scan); if (formatVersion == 2) { // Four data files and delete files of old spec, one new data file of new spec - Assert.assertEquals(9, Iterables.size(files)); + Assert.assertEquals(9, Iterables.size(entries)); } else { // Four data files of old spec, one new data file of new spec - Assert.assertEquals(5, Iterables.size(files)); + Assert.assertEquals(5, Iterables.size(entries)); } filter = @@ -802,15 +807,15 @@ public void testPartitionSpecEvolutionAdditive() { Expressions.equal("partition.data_bucket", 0), Expressions.greaterThan("record_count", 0)); scan = metadataTable.newScan().filter(filter); - files = PartitionsTable.planFiles((StaticTableScan) scan); + entries = PartitionsTable.planEntries((StaticTableScan) scan); if (formatVersion == 2) { // 1 original data file and delete file written by old spec, plus 1 new data file written by // new spec - Assert.assertEquals(3, Iterables.size(files)); + Assert.assertEquals(3, Iterables.size(entries)); } else { // 1 original data file written by old spec, plus 1 new data file written by new spec - Assert.assertEquals(2, Iterables.size(files)); + Assert.assertEquals(2, Iterables.size(entries)); } } @@ -852,14 +857,15 @@ public void testPartitionSpecEvolutionRemoval() { Expressions.and( Expressions.equal("partition.id", 10), Expressions.greaterThan("record_count", 0)); TableScan scan = metadataTable.newScan().filter(filter); - CloseableIterable> files = PartitionsTable.planFiles((StaticTableScan) scan); + CloseableIterable> entries = + PartitionsTable.planEntries((StaticTableScan) scan); if (formatVersion == 2) { // Four data and delete files of original spec, one data file written by new spec - Assert.assertEquals(9, Iterables.size(files)); + Assert.assertEquals(9, Iterables.size(entries)); } else { // Four data files of original spec, one data file written by new spec - Assert.assertEquals(5, Iterables.size(files)); + Assert.assertEquals(5, Iterables.size(entries)); } // Filter for a dropped partition spec field. Correct behavior is that only old partitions are @@ -869,11 +875,11 @@ public void testPartitionSpecEvolutionRemoval() { Expressions.equal("partition.data_bucket", 0), Expressions.greaterThan("record_count", 0)); scan = metadataTable.newScan().filter(filter); - files = PartitionsTable.planFiles((StaticTableScan) scan); + entries = PartitionsTable.planEntries((StaticTableScan) scan); if (formatVersion == 1) { // 1 original data file written by old spec - Assert.assertEquals(1, Iterables.size(files)); + Assert.assertEquals(1, Iterables.size(entries)); } else { // 1 original data and 1 delete files written by old spec, plus both of new data file/delete // file written by new spec @@ -888,7 +894,7 @@ public void testPartitionSpecEvolutionRemoval() { // schema. // The Partition table final schema is a union of fields of all specs, including dropped // fields. - Assert.assertEquals(4, Iterables.size(files)); + Assert.assertEquals(4, Iterables.size(entries)); } } @@ -939,10 +945,10 @@ public void testPartitionColumnNamedPartition() throws Exception { Expressions.equal("partition.partition", 0), Expressions.greaterThan("record_count", 0)); TableScan scanAndEq = partitionsTable.newScan().filter(andEquals); - CloseableIterable> files = - PartitionsTable.planFiles((StaticTableScan) scanAndEq); - Assert.assertEquals(1, Iterators.size(files.iterator())); - validateSingleFieldPartition(files, 0); + CloseableIterable> entries = + PartitionsTable.planEntries((StaticTableScan) scanAndEq); + Assert.assertEquals(1, Iterators.size(entries.iterator())); + validateSingleFieldPartition(entries, 0); } @Test @@ -1010,11 +1016,12 @@ public void testPartitionsTableScanWithPlanExecutor() { true); // daemon threads will be terminated abruptly when the JVM exits return thread; })); - CloseableIterable> files = PartitionsTable.planFiles((StaticTableScan) scan); + CloseableIterable> entries = + PartitionsTable.planEntries((StaticTableScan) scan); if (formatVersion == 2) { - Assert.assertEquals(8, Iterators.size(files.iterator())); + Assert.assertEquals(8, Iterators.size(entries.iterator())); } else { - Assert.assertEquals(4, Iterators.size(files.iterator())); + Assert.assertEquals(4, Iterators.size(entries.iterator())); } Assert.assertTrue("Thread should be created in provided pool", planThreadsIndex.get() > 0); diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataTableScansWithPartitionEvolution.java b/core/src/test/java/org/apache/iceberg/TestMetadataTableScansWithPartitionEvolution.java index b6e1ecae4ffe..92fa080dfec3 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataTableScansWithPartitionEvolution.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataTableScansWithPartitionEvolution.java @@ -140,7 +140,7 @@ public void testAllManifestsTableWithAddPartitionOnNestedField() throws IOExcept } @Test - public void testPartitionsTableScanWithAddPartitionOnNestedField() throws IOException { + public void testPartitionsTableScanWithAddPartitionOnNestedField() { Table partitionsTable = new PartitionsTable(table); Types.StructType idPartition = new Schema( @@ -154,15 +154,15 @@ public void testPartitionsTableScanWithAddPartitionOnNestedField() throws IOExce TableScan scanNoFilter = partitionsTable.newScan().select("partition"); Assert.assertEquals(idPartition, scanNoFilter.schema().asStruct()); - CloseableIterable> files = - PartitionsTable.planFiles((StaticTableScan) scanNoFilter); - Assert.assertEquals(4, Iterators.size(files.iterator())); - validatePartition(files, 0, 0); - validatePartition(files, 0, 1); - validatePartition(files, 0, 2); - validatePartition(files, 0, 3); - validatePartition(files, 1, 2); - validatePartition(files, 1, 3); + CloseableIterable> entries = + PartitionsTable.planEntries((StaticTableScan) scanNoFilter); + Assert.assertEquals(4, Iterators.size(entries.iterator())); + validatePartition(entries, 0, 0); + validatePartition(entries, 0, 1); + validatePartition(entries, 0, 2); + validatePartition(entries, 0, 3); + validatePartition(entries, 1, 2); + validatePartition(entries, 1, 3); } @Test @@ -241,16 +241,16 @@ public void testPartitionSpecEvolutionToUnpartitioned() throws IOException { // must contain the partition column even when the current spec is non-partitioned. Assertions.assertThat(partitionsTable.schema().findField("partition")).isNotNull(); - try (CloseableIterable> files = - PartitionsTable.planFiles((StaticTableScan) partitionsTable.newScan())) { + try (CloseableIterable> entries = + PartitionsTable.planEntries((StaticTableScan) partitionsTable.newScan())) { // four partitioned data files and one non-partitioned data file. - Assertions.assertThat(files).hasSize(5); + Assertions.assertThat(entries).hasSize(5); // check for null partition value. - Assertions.assertThat(StreamSupport.stream(files.spliterator(), false)) + Assertions.assertThat(StreamSupport.stream(entries.spliterator(), false)) .anyMatch( - file -> { - StructLike partition = file.partition(); + entry -> { + StructLike partition = entry.file().partition(); return Objects.equals(null, partition.get(0, Object.class)); }); } diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataUpdateParser.java b/core/src/test/java/org/apache/iceberg/TestMetadataUpdateParser.java index aa7c120aebe8..09259aaa3752 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataUpdateParser.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataUpdateParser.java @@ -56,11 +56,9 @@ public void testMetadataUpdateWithoutActionCannotDeserialize() { ImmutableList.of("{\"action\":null,\"format-version\":2}", "{\"format-version\":2}"); for (String json : invalidJson) { - AssertHelpers.assertThrows( - "MetadataUpdate without a recognized action should fail to deserialize", - IllegalArgumentException.class, - "Cannot parse metadata update. Missing field: action", - () -> MetadataUpdateParser.fromJson(json)); + Assertions.assertThatThrownBy(() -> MetadataUpdateParser.fromJson(json)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse metadata update. Missing field: action"); } } @@ -702,11 +700,9 @@ public void testSetPropertiesFromJsonFailsWhenDeserializingNullValues() { props.put("prop2", null); String propsMap = "{\"prop1\":\"val1\",\"prop2\":null}"; String json = String.format("{\"action\":\"%s\",\"updated\":%s}", action, propsMap); - AssertHelpers.assertThrows( - "Parsing updates from SetProperties with a property set to null should throw", - IllegalArgumentException.class, - "Cannot parse to a string value: prop2: null", - () -> MetadataUpdateParser.fromJson(json)); + Assertions.assertThatThrownBy(() -> MetadataUpdateParser.fromJson(json)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse to a string value: prop2: null"); } @Test diff --git a/core/src/test/java/org/apache/iceberg/TestMetricsTruncation.java b/core/src/test/java/org/apache/iceberg/TestMetricsTruncation.java index e314385f7e4f..3fe974f086a1 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetricsTruncation.java +++ b/core/src/test/java/org/apache/iceberg/TestMetricsTruncation.java @@ -27,8 +27,8 @@ import java.nio.ByteBuffer; import java.util.Comparator; import org.apache.iceberg.expressions.Literal; -import org.junit.Assert; -import org.junit.Test; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; @SuppressWarnings("checkstyle:LocalVariableName") public class TestMetricsTruncation { @@ -39,30 +39,34 @@ public void testTruncateBinary() { ByteBuffer emptyByteBuffer = ByteBuffer.allocate(0); Comparator cmp = Literal.of(original).comparator(); - Assert.assertEquals( - "Truncating to a length of zero should return an empty ByteBuffer", - 0, - cmp.compare(truncateBinary(original, 0), emptyByteBuffer)); - Assert.assertEquals( - "Truncating to the original buffer's remaining size should return the original buffer", - original, - truncateBinary(original, original.remaining())); - Assert.assertEquals( - "Truncating with a length greater than the input's remaining size should return the input", - original, - truncateBinary(original, 16)); + Assertions.assertThat(cmp.compare(truncateBinary(original, 0), emptyByteBuffer)) + .as("Truncating to a length of zero should return an empty ByteBuffer") + .isEqualTo(0); + Assertions.assertThat(truncateBinary(original, original.remaining())) + .as("Truncating to the original buffer's remaining size should return the original buffer") + .isEqualTo(original); + Assertions.assertThat(truncateBinary(original, 16)) + .as( + "Truncating with a length greater than the input's remaining size should return the input") + .isEqualTo(original); ByteBuffer truncated = truncateBinary(original, 2); - Assert.assertTrue( - "Truncating with a length less than the input's remaining size should truncate properly", - truncated.remaining() == 2 && truncated.position() == 0); - Assert.assertTrue( - "Truncating should not modify the input buffer", - original.remaining() == 4 && original.position() == 0); - AssertHelpers.assertThrows( - "Should not allow a negative truncation length", - IllegalArgumentException.class, - "length should be non-negative", - () -> truncateBinary(original, -1)); + Assertions.assertThat(truncated.remaining()) + .as( + "Truncating with a length less than the input's remaining size should truncate properly") + .isEqualTo(2); + Assertions.assertThat(truncated.position()) + .as( + "Truncating with a length less than the input's remaining size should truncate properly") + .isEqualTo(0); + Assertions.assertThat(original.remaining()) + .as("Truncating should not modify the input buffer") + .isEqualTo(4); + Assertions.assertThat(original.position()) + .as("Truncating should not modify the input buffer") + .isEqualTo(0); + Assertions.assertThatThrownBy(() -> truncateBinary(original, -1)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Truncate length should be non-negative"); } @Test @@ -74,22 +78,24 @@ public void testTruncateBinaryMin() { ByteBuffer test2_2 = ByteBuffer.wrap(new byte[] {(byte) 0xFF, (byte) 0xFF}); Comparator cmp = Literal.of(test1).comparator(); - Assert.assertTrue( - "Truncated lower bound should be lower than or equal to the actual lower bound", - cmp.compare(truncateBinaryMin(Literal.of(test1), 2).value(), test1) <= 0); - Assert.assertTrue( - "Output must have the first two bytes of the input", - cmp.compare(truncateBinaryMin(Literal.of(test1), 2).value(), test1_2_expected) == 0); - Assert.assertTrue( - "No truncation required as truncate length is greater than the input size", - cmp.compare(truncateBinaryMin(Literal.of(test1), 5).value(), test1) == 0); - Assert.assertTrue( - "Truncated lower bound should be lower than or equal to the actual lower bound", - cmp.compare(truncateBinaryMin(Literal.of(test2), 2).value(), test2) <= 0); - Assert.assertTrue( - "Output must have the first two bytes of the input. A lower bound exists " - + "even though the first two bytes are the max value", - cmp.compare(truncateBinaryMin(Literal.of(test2), 2).value(), test2_2) == 0); + Assertions.assertThat(cmp.compare(truncateBinaryMin(Literal.of(test1), 2).value(), test1)) + .as("Truncated lower bound should be lower than or equal to the actual lower bound") + .isLessThanOrEqualTo(0); + Assertions.assertThat( + cmp.compare(truncateBinaryMin(Literal.of(test1), 2).value(), test1_2_expected)) + .as("Output must have the first two bytes of the input") + .isEqualTo(0); + Assertions.assertThat(cmp.compare(truncateBinaryMin(Literal.of(test1), 5).value(), test1)) + .as("No truncation required as truncate length is greater than the input size") + .isEqualTo(0); + Assertions.assertThat(cmp.compare(truncateBinaryMin(Literal.of(test2), 2).value(), test2)) + .as("Truncated lower bound should be lower than or equal to the actual lower bound") + .isLessThanOrEqualTo(0); + Assertions.assertThat(cmp.compare(truncateBinaryMin(Literal.of(test2), 2).value(), test2_2)) + .as( + "Output must have the first two bytes of the input. A lower bound exists " + + "even though the first two bytes are the max value") + .isEqualTo(0); } @Test @@ -101,32 +107,37 @@ public void testTruncateBinaryMax() { ByteBuffer expectedOutput = ByteBuffer.wrap(new byte[] {1, 2}); Comparator cmp = Literal.of(test1).comparator(); - Assert.assertTrue( - "Truncated upper bound should be greater than or equal to the actual upper bound", - cmp.compare(truncateBinaryMax(Literal.of(test1), 2).value(), test1) >= 0); - Assert.assertTrue( - "Output must have two bytes and the second byte of the input must be incremented", - cmp.compare(truncateBinaryMax(Literal.of(test1), 2).value(), expectedOutput) == 0); - Assert.assertTrue( - "Truncated upper bound should be greater than or equal to the actual upper bound", - cmp.compare(truncateBinaryMax(Literal.of(test2), 2).value(), test2) >= 0); - Assert.assertTrue( - "Since the third byte is already the max value, output must have two bytes " - + "with the second byte incremented ", - cmp.compare(truncateBinaryMax(Literal.of(test2), 3).value(), expectedOutput) == 0); - Assert.assertTrue( - "No truncation required as truncate length is greater than the input size", - cmp.compare(truncateBinaryMax(Literal.of(test3), 5).value(), test3) == 0); - Assert.assertNull( - "An upper bound doesn't exist since the first two bytes are the max value", - truncateBinaryMax(Literal.of(test3), 2)); - Assert.assertTrue( - "Truncated upper bound should be greater than or equal to the actual upper bound", - cmp.compare(truncateBinaryMax(Literal.of(test4), 2).value(), test4) >= 0); - Assert.assertTrue( - "Since a shorter sequence is considered smaller, output must have two bytes " - + "and the second byte of the input must be incremented", - cmp.compare(truncateBinaryMax(Literal.of(test4), 2).value(), expectedOutput) == 0); + Assertions.assertThat(cmp.compare(truncateBinaryMax(Literal.of(test1), 2).value(), test1)) + .as("Truncated upper bound should be greater than or equal to the actual upper bound") + .isGreaterThanOrEqualTo(0); + Assertions.assertThat( + cmp.compare(truncateBinaryMax(Literal.of(test1), 2).value(), expectedOutput)) + .as("Output must have two bytes and the second byte of the input must be incremented") + .isEqualTo(0); + Assertions.assertThat(cmp.compare(truncateBinaryMax(Literal.of(test2), 2).value(), test2)) + .as("Truncated upper bound should be greater than or equal to the actual upper bound") + .isGreaterThanOrEqualTo(0); + Assertions.assertThat( + cmp.compare(truncateBinaryMax(Literal.of(test2), 3).value(), expectedOutput)) + .as( + "Since the third byte is already the max value, output must have two bytes " + + "with the second byte incremented ") + .isEqualTo(0); + Assertions.assertThat(cmp.compare(truncateBinaryMax(Literal.of(test3), 5).value(), test3)) + .as("No truncation required as truncate length is greater than the input size") + .isEqualTo(0); + Assertions.assertThat(truncateBinaryMax(Literal.of(test3), 2)) + .as("An upper bound doesn't exist since the first two bytes are the max value") + .isNull(); + Assertions.assertThat(cmp.compare(truncateBinaryMax(Literal.of(test4), 2).value(), test4)) + .as("Truncated upper bound should be greater than or equal to the actual upper bound") + .isGreaterThanOrEqualTo(0); + Assertions.assertThat( + cmp.compare(truncateBinaryMax(Literal.of(test4), 2).value(), expectedOutput)) + .as( + "Since a shorter sequence is considered smaller, output must have two bytes " + + "and the second byte of the input must be incremented") + .isEqualTo(0); } @SuppressWarnings("checkstyle:AvoidEscapedUnicodeCharacters") @@ -144,36 +155,40 @@ public void testTruncateStringMin() { String test4 = "\uD800\uDC00\uD800\uDC00"; String test4_1_expected = "\uD800\uDC00"; Comparator cmp = Literal.of(test1).comparator(); - Assert.assertTrue( - "Truncated lower bound should be lower than or equal to the actual lower bound", - cmp.compare(truncateStringMin(Literal.of(test1), 3).value(), test1) <= 0); - Assert.assertTrue( - "No truncation required as truncate length is greater than the input size", - cmp.compare(truncateStringMin(Literal.of(test1), 8).value(), test1) == 0); - Assert.assertTrue( - "Output must have the first two characters of the input", - cmp.compare(truncateStringMin(Literal.of(test1), 2).value(), test1_2_expected) == 0); - Assert.assertTrue( - "Output must have the first three characters of the input", - cmp.compare(truncateStringMin(Literal.of(test1), 3).value(), test1_3_expected) == 0); - Assert.assertTrue( - "Truncated lower bound should be lower than or equal to the actual lower bound", - cmp.compare(truncateStringMin(Literal.of(test2), 16).value(), test2) <= 0); - Assert.assertTrue( - "Output must have the first seven characters of the input", - cmp.compare(truncateStringMin(Literal.of(test2), 7).value(), test2_7_expected) == 0); - Assert.assertTrue( - "Truncated lower bound should be lower than or equal to the actual lower bound", - cmp.compare(truncateStringMin(Literal.of(test3), 2).value(), test3) <= 0); - Assert.assertTrue( - "No truncation required as truncate length is equal to the input size", - cmp.compare(truncateStringMin(Literal.of(test3), 2).value(), test3) == 0); - Assert.assertTrue( - "Truncated lower bound should be lower than or equal to the actual lower bound", - cmp.compare(truncateStringMin(Literal.of(test4), 1).value(), test4) <= 0); - Assert.assertTrue( - "Output must have the first 4 byte UTF-8 character of the input", - cmp.compare(truncateStringMin(Literal.of(test4), 1).value(), test4_1_expected) == 0); + Assertions.assertThat(cmp.compare(truncateStringMin(Literal.of(test1), 3).value(), test1)) + .as("Truncated lower bound should be lower than or equal to the actual lower bound") + .isLessThanOrEqualTo(0); + Assertions.assertThat(cmp.compare(truncateStringMin(Literal.of(test1), 8).value(), test1)) + .as("No truncation required as truncate length is greater than the input size") + .isEqualTo(0); + Assertions.assertThat( + cmp.compare(truncateStringMin(Literal.of(test1), 2).value(), test1_2_expected)) + .as("Output must have the first two characters of the input") + .isEqualTo(0); + Assertions.assertThat( + cmp.compare(truncateStringMin(Literal.of(test1), 3).value(), test1_3_expected)) + .as("Output must have the first three characters of the input") + .isEqualTo(0); + Assertions.assertThat(cmp.compare(truncateStringMin(Literal.of(test2), 16).value(), test2)) + .as("Truncated lower bound should be lower than or equal to the actual lower bound") + .isLessThanOrEqualTo(0); + Assertions.assertThat( + cmp.compare(truncateStringMin(Literal.of(test2), 7).value(), test2_7_expected)) + .as("Output must have the first seven characters of the input") + .isEqualTo(0); + Assertions.assertThat(cmp.compare(truncateStringMin(Literal.of(test3), 2).value(), test3)) + .as("Truncated lower bound should be lower than or equal to the actual lower bound") + .isLessThanOrEqualTo(0); + Assertions.assertThat(cmp.compare(truncateStringMin(Literal.of(test3), 2).value(), test3)) + .as("No truncation required as truncate length is equal to the input size") + .isEqualTo(0); + Assertions.assertThat(cmp.compare(truncateStringMin(Literal.of(test4), 1).value(), test4)) + .as("Truncated lower bound should be lower than or equal to the actual lower bound") + .isLessThanOrEqualTo(0); + Assertions.assertThat( + cmp.compare(truncateStringMin(Literal.of(test4), 1).value(), test4_1_expected)) + .as("Output must have the first 4 byte UTF-8 character of the input") + .isEqualTo(0); } @SuppressWarnings("checkstyle:AvoidEscapedUnicodeCharacters") @@ -200,65 +215,78 @@ public void testTruncateStringMax() { String test7_1_expected = "\uD83D\uDE03"; Comparator cmp = Literal.of(test1).comparator(); - Assert.assertTrue( - "Truncated upper bound should be greater than or equal to the actual upper bound", - cmp.compare(truncateStringMax(Literal.of(test1), 4).value(), test1) >= 0); - Assert.assertTrue( - "No truncation required as truncate length is equal to the input size", - cmp.compare(truncateStringMax(Literal.of(test1), 7).value(), test1) == 0); - Assert.assertTrue( - "Output must have two characters and the second character of the input must " - + "be incremented", - cmp.compare(truncateStringMax(Literal.of(test1), 2).value(), test1_2_expected) == 0); - Assert.assertTrue( - "Output must have three characters and the third character of the input must " - + "be incremented", - cmp.compare(truncateStringMax(Literal.of(test1), 3).value(), test1_3_expected) == 0); - Assert.assertTrue( - "No truncation required as truncate length is greater than the input size", - cmp.compare(truncateStringMax(Literal.of(test1), 8).value(), test1) == 0); - Assert.assertTrue( - "Truncated upper bound should be greater than or equal to the actual upper " + "bound", - cmp.compare(truncateStringMax(Literal.of(test2), 8).value(), test2) >= 0); - Assert.assertTrue( - "Output must have seven characters and the seventh character of the input " - + "must be incremented", - cmp.compare(truncateStringMax(Literal.of(test2), 7).value(), test2_7_expected) == 0); - Assert.assertTrue( - "Truncated upper bound should be greater than or equal to the actual upper " + "bound", - cmp.compare(truncateStringMax(Literal.of(test3), 3).value(), test3) >= 0); - Assert.assertTrue( - "Output must have three characters and the third character of the input must " - + "be incremented. The second perceivable character in this string is actually a glyph. It consists of " - + "two unicode characters", - cmp.compare(truncateStringMax(Literal.of(test3), 3).value(), test3_3_expected) == 0); - Assert.assertTrue( - "Truncated upper bound should be greater than or equal to the actual upper bound", - cmp.compare(truncateStringMax(Literal.of(test4), 1).value(), test4) >= 0); - Assert.assertTrue( - "Output must have one character. Since the first character is the max 3 byte " - + "UTF-8 character, it should be incremented to the lowest 4 byte UTF-8 character", - cmp.compare(truncateStringMax(Literal.of(test4), 1).value(), test4_1_expected) == 0); - Assert.assertNull( - "An upper bound doesn't exist since the first two characters are max UTF-8 " + "characters", - truncateStringMax(Literal.of(test5), 1)); - Assert.assertTrue( - "Truncated upper bound should be greater than or equal to the actual upper bound", - cmp.compare(truncateStringMax(Literal.of(test6), 2).value(), test6) >= 0); - Assert.assertTrue( - "Test 4 byte UTF-8 character increment. Output must have one character with " - + "the first character incremented", - cmp.compare(truncateStringMax(Literal.of(test6), 1).value(), test6_2_expected) == 0); - Assert.assertTrue( - "Truncated upper bound should be greater than or equal to the actual upper bound", - cmp.compare(truncateStringMax(Literal.of(test7), 2).value(), test7) >= 0); - Assert.assertTrue( - "Test input with multiple 4 byte UTF-8 character where the second unicode " - + "character should be incremented", - cmp.compare(truncateStringMax(Literal.of(test7), 2).value(), test7_2_expected) == 0); - Assert.assertTrue( - "Test input with multiple 4 byte UTF-8 character where the first unicode " - + "character should be incremented", - cmp.compare(truncateStringMax(Literal.of(test7), 1).value(), test7_1_expected) == 0); + Assertions.assertThat(cmp.compare(truncateStringMax(Literal.of(test1), 4).value(), test1)) + .as("Truncated upper bound should be greater than or equal to the actual upper bound") + .isGreaterThanOrEqualTo(0); + Assertions.assertThat(cmp.compare(truncateStringMax(Literal.of(test1), 7).value(), test1)) + .as("No truncation required as truncate length is equal to the input size") + .isEqualTo(0); + Assertions.assertThat( + cmp.compare(truncateStringMax(Literal.of(test1), 2).value(), test1_2_expected)) + .as( + "Output must have two characters and the second character of the input must " + + "be incremented") + .isEqualTo(0); + Assertions.assertThat( + cmp.compare(truncateStringMax(Literal.of(test1), 3).value(), test1_3_expected)) + .as( + "Output must have three characters and the third character of the input must " + + "be incremented") + .isEqualTo(0); + Assertions.assertThat(cmp.compare(truncateStringMax(Literal.of(test1), 8).value(), test1)) + .as("No truncation required as truncate length is greater than the input size") + .isEqualTo(0); + Assertions.assertThat(cmp.compare(truncateStringMax(Literal.of(test2), 8).value(), test2)) + .as("Truncated upper bound should be greater than or equal to the actual upper bound") + .isGreaterThanOrEqualTo(0); + Assertions.assertThat( + cmp.compare(truncateStringMax(Literal.of(test2), 7).value(), test2_7_expected)) + .as( + "Output must have seven characters and the seventh character of the input must be incremented") + .isEqualTo(0); + Assertions.assertThat(cmp.compare(truncateStringMax(Literal.of(test3), 3).value(), test3)) + .as("Truncated upper bound should be greater than or equal to the actual upper bound") + .isGreaterThanOrEqualTo(0); + Assertions.assertThat( + cmp.compare(truncateStringMax(Literal.of(test3), 3).value(), test3_3_expected)) + .as( + "Output must have three characters and the third character of the input must " + + "be incremented. The second perceivable character in this string is actually a glyph. It consists of " + + "two unicode characters") + .isEqualTo(0); + Assertions.assertThat(cmp.compare(truncateStringMax(Literal.of(test4), 1).value(), test4)) + .as("Truncated upper bound should be greater than or equal to the actual upper bound") + .isGreaterThanOrEqualTo(0); + Assertions.assertThat( + cmp.compare(truncateStringMax(Literal.of(test4), 1).value(), test4_1_expected)) + .as( + "Output must have one character. Since the first character is the max 3 byte " + + "UTF-8 character, it should be incremented to the lowest 4 byte UTF-8 character") + .isEqualTo(0); + Assertions.assertThat(truncateStringMax(Literal.of(test5), 1)) + .as("An upper bound doesn't exist since the first two characters are max UTF-8 characters") + .isNull(); + Assertions.assertThat(cmp.compare(truncateStringMax(Literal.of(test6), 2).value(), test6)) + .as("Truncated upper bound should be greater than or equal to the actual upper bound") + .isGreaterThanOrEqualTo(0); + Assertions.assertThat( + cmp.compare(truncateStringMax(Literal.of(test6), 1).value(), test6_2_expected)) + .as( + "Test 4 byte UTF-8 character increment. Output must have one character with " + + "the first character incremented") + .isEqualTo(0); + Assertions.assertThat(cmp.compare(truncateStringMax(Literal.of(test7), 2).value(), test7)) + .as("Truncated upper bound should be greater than or equal to the actual upper bound") + .isGreaterThanOrEqualTo(0); + Assertions.assertThat( + cmp.compare(truncateStringMax(Literal.of(test7), 2).value(), test7_2_expected)) + .as( + "Test input with multiple 4 byte UTF-8 character where the second unicode character should be incremented") + .isEqualTo(0); + Assertions.assertThat( + cmp.compare(truncateStringMax(Literal.of(test7), 1).value(), test7_1_expected)) + .as( + "Test input with multiple 4 byte UTF-8 character where the first unicode character should be incremented") + .isEqualTo(0); } } diff --git a/core/src/test/java/org/apache/iceberg/TestOffsetsBasedSplitScanTaskIterator.java b/core/src/test/java/org/apache/iceberg/TestOffsetsBasedSplitScanTaskIterator.java index 7ca14a185a05..d9ad4fb3cbbd 100644 --- a/core/src/test/java/org/apache/iceberg/TestOffsetsBasedSplitScanTaskIterator.java +++ b/core/src/test/java/org/apache/iceberg/TestOffsetsBasedSplitScanTaskIterator.java @@ -21,8 +21,8 @@ import java.util.List; import org.apache.iceberg.BaseFileScanTask.SplitScanTask; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.Test; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; public class TestOffsetsBasedSplitScanTaskIterator { @Test @@ -62,15 +62,15 @@ private static void verify( offsetRanges, TestOffsetsBasedSplitScanTaskIterator::createSplitTask); List tasks = Lists.newArrayList(splitTaskIterator); - Assert.assertEquals("Number of tasks don't match", offsetLenPairs.size(), tasks.size()); + Assertions.assertThat(tasks).as("Number of tasks don't match").hasSameSizeAs(offsetLenPairs); for (int i = 0; i < tasks.size(); i++) { FileScanTask task = tasks.get(i); List split = offsetLenPairs.get(i); long offset = split.get(0); long length = split.get(1); - Assert.assertEquals(offset, task.start()); - Assert.assertEquals(length, task.length()); + Assertions.assertThat(task.start()).isEqualTo(offset); + Assertions.assertThat(task.length()).isEqualTo(length); } } diff --git a/core/src/test/java/org/apache/iceberg/TestOverwrite.java b/core/src/test/java/org/apache/iceberg/TestOverwrite.java index 4f6a184e9ec1..e5665d6714fc 100644 --- a/core/src/test/java/org/apache/iceberg/TestOverwrite.java +++ b/core/src/test/java/org/apache/iceberg/TestOverwrite.java @@ -33,6 +33,7 @@ import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -170,11 +171,9 @@ public void testOverwriteFailsDelete() { .newOverwrite() .overwriteByRowFilter(and(equal("date", "2018-06-09"), lessThan("id", 9))); - AssertHelpers.assertThrows( - "Should reject commit with file not matching delete expression", - ValidationException.class, - "Cannot delete file where some, but not all, rows match filter", - () -> commit(table, overwrite, branch)); + Assertions.assertThatThrownBy(() -> commit(table, overwrite, branch)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Cannot delete file where some, but not all, rows match filter"); Assert.assertEquals( "Should not create a new snapshot", baseId, latestSnapshot(base, branch).snapshotId()); @@ -264,11 +263,9 @@ public void testValidatedOverwriteWithAppendOutsideOfDelete() { .addFile(FILE_10_TO_14) // in 2018-06-09, NOT in 2018-06-08 .validateAddedFilesMatchOverwriteFilter(); - AssertHelpers.assertThrows( - "Should reject commit with file not matching delete expression", - ValidationException.class, - "Cannot append file with rows that do not match filter", - () -> commit(table, overwrite, branch)); + Assertions.assertThatThrownBy(() -> commit(table, overwrite, branch)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Cannot append file with rows that do not match filter"); Assert.assertEquals( "Should not create a new snapshot", baseId, latestSnapshot(table, branch).snapshotId()); @@ -287,11 +284,9 @@ public void testValidatedOverwriteWithAppendOutsideOfDeleteMetrics() { .addFile(FILE_10_TO_14) // in 2018-06-09 matches, but IDs are outside range .validateAddedFilesMatchOverwriteFilter(); - AssertHelpers.assertThrows( - "Should reject commit with file not matching delete expression", - ValidationException.class, - "Cannot append file with rows that do not match filter", - () -> commit(table, overwrite, branch)); + Assertions.assertThatThrownBy(() -> commit(table, overwrite, branch)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Cannot append file with rows that do not match filter"); Assert.assertEquals( "Should not create a new snapshot", baseId, latestSnapshot(base, branch).snapshotId()); @@ -310,11 +305,9 @@ public void testValidatedOverwriteWithAppendSuccess() { .addFile(FILE_10_TO_14) // in 2018-06-09 matches and IDs are inside range .validateAddedFilesMatchOverwriteFilter(); - AssertHelpers.assertThrows( - "Should reject commit with file not matching delete expression", - ValidationException.class, - "Cannot append file with rows that do not match filter", - () -> commit(table, overwrite, branch)); + Assertions.assertThatThrownBy(() -> commit(table, overwrite, branch)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Cannot append file with rows that do not match filter"); Assert.assertEquals( "Should not create a new snapshot", baseId, latestSnapshot(base, branch).snapshotId()); diff --git a/core/src/test/java/org/apache/iceberg/TestOverwriteWithValidation.java b/core/src/test/java/org/apache/iceberg/TestOverwriteWithValidation.java index b7d194377ef6..a4ccb4018c9a 100644 --- a/core/src/test/java/org/apache/iceberg/TestOverwriteWithValidation.java +++ b/core/src/test/java/org/apache/iceberg/TestOverwriteWithValidation.java @@ -36,6 +36,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Assume; import org.junit.Before; @@ -339,11 +340,9 @@ public void testOverwriteCompatibleAdditionStrictValidated() { commit(table, table.newAppend().appendFile(FILE_DAY_1), branch); long committedSnapshotId = latestSnapshot(table, branch).snapshotId(); - AssertHelpers.assertThrows( - "Should reject commit", - ValidationException.class, - "Found conflicting files", - () -> commit(table, overwrite, branch)); + Assertions.assertThatThrownBy(() -> commit(table, overwrite, branch)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Found conflicting files"); Assert.assertEquals( "Should not create a new snapshot", @@ -415,11 +414,9 @@ public void testOverwriteIncompatibleAdditionValidated() { commit(table, table.newAppend().appendFile(FILE_DAY_2), branch); long committedSnapshotId = latestSnapshot(table, branch).snapshotId(); - AssertHelpers.assertThrows( - "Should reject commit", - ValidationException.class, - "Found conflicting files", - () -> commit(table, overwrite, branch)); + Assertions.assertThatThrownBy(() -> commit(table, overwrite, branch)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Found conflicting files"); Assert.assertEquals( "Should not create a new snapshot", @@ -446,11 +443,9 @@ public void testOverwriteIncompatibleDeletionValidated() { commit(table, table.newDelete().deleteFile(FILE_DAY_2), branch); long committedSnapshotId = latestSnapshot(table, branch).snapshotId(); - AssertHelpers.assertThrows( - "Should reject commit", - ValidationException.class, - "Missing required files to delete:", - () -> commit(table, overwrite, branch)); + Assertions.assertThatThrownBy(() -> commit(table, overwrite, branch)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Missing required files to delete:"); Assert.assertEquals( "Should not create a new snapshot", @@ -558,11 +553,9 @@ public void testOverwriteIncompatibleExpirationValidated() { table.expireSnapshots().expireSnapshotId(2L).commit(); long committedSnapshotId = latestSnapshot(table, branch).snapshotId(); - AssertHelpers.assertThrows( - "Should reject commit", - ValidationException.class, - "Cannot determine history", - () -> commit(table, overwrite, branch)); + Assertions.assertThatThrownBy(() -> commit(table, overwrite, branch)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Cannot determine history"); Assert.assertEquals( "Should not create a new snapshot", @@ -588,11 +581,9 @@ public void testOverwriteIncompatibleBaseExpirationEmptyTableValidated() { table.expireSnapshots().expireSnapshotId(1L).commit(); long committedSnapshotId = latestSnapshot(table, branch).snapshotId(); - AssertHelpers.assertThrows( - "Should reject commit", - ValidationException.class, - "Cannot determine history", - () -> commit(table, overwrite, branch)); + Assertions.assertThatThrownBy(() -> commit(table, overwrite, branch)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Cannot determine history"); Assert.assertEquals( "Should not create a new snapshot", @@ -681,11 +672,9 @@ public void testTransactionIncompatibleAdditionValidated() { commit(table, overwrite, branch); - AssertHelpers.assertThrows( - "Should reject commit", - ValidationException.class, - "Found conflicting files", - txn::commitTransaction); + Assertions.assertThatThrownBy(txn::commitTransaction) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Found conflicting files"); Assert.assertEquals( "Should not create a new snapshot", @@ -715,11 +704,9 @@ public void testConcurrentConflictingPositionDeletes() { commit(table, table.newRowDelta().addDeletes(FILE_DAY_2_POS_DELETES), branch); - AssertHelpers.assertThrows( - "Should reject commit", - ValidationException.class, - "found new delete", - () -> commit(table, overwrite, branch)); + Assertions.assertThatThrownBy(() -> commit(table, overwrite, branch)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Cannot commit, found new delete for replaced data file"); } @Test @@ -744,11 +731,9 @@ public void testConcurrentConflictingPositionDeletesOverwriteByFilter() { commit(table, table.newRowDelta().addDeletes(FILE_DAY_2_POS_DELETES), branch); - AssertHelpers.assertThrows( - "Should reject commit", - ValidationException.class, - "Found new conflicting delete", - () -> commit(table, overwrite, branch)); + Assertions.assertThatThrownBy(() -> commit(table, overwrite, branch)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Found new conflicting delete"); } @Test @@ -770,11 +755,9 @@ public void testConcurrentConflictingDataFileDeleteOverwriteByFilter() { commit(table, table.newOverwrite().deleteFile(FILE_DAY_2), branch); - AssertHelpers.assertThrows( - "Should reject commit", - ValidationException.class, - "Found conflicting deleted files", - () -> commit(table, overwrite, branch)); + Assertions.assertThatThrownBy(() -> commit(table, overwrite, branch)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Found conflicting deleted files"); } @Test @@ -879,11 +862,9 @@ public void testConcurrentConflictingEqualityDeletes() { commit(table, table.newRowDelta().addDeletes(FILE_DAY_2_EQ_DELETES), branch); - AssertHelpers.assertThrows( - "Should reject commit", - ValidationException.class, - "found new delete", - () -> commit(table, overwrite, branch)); + Assertions.assertThatThrownBy(() -> commit(table, overwrite, branch)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Cannot commit, found new delete for replaced data file"); } @Test @@ -953,50 +934,47 @@ public void testOverwriteCaseSensitivity() { Expression rowFilter = equal("dAtE", "2018-06-09"); - AssertHelpers.assertThrows( - "Should use case sensitive binding by default", - ValidationException.class, - "Cannot find field 'dAtE'", - () -> - commit( - table, - table - .newOverwrite() - .addFile(FILE_DAY_2_MODIFIED) - .conflictDetectionFilter(rowFilter) - .validateNoConflictingData(), - branch)); - - AssertHelpers.assertThrows( - "Should fail with case sensitive binding", - ValidationException.class, - "Cannot find field 'dAtE'", - () -> - commit( - table, - table - .newOverwrite() - .caseSensitive(true) - .addFile(FILE_DAY_2_MODIFIED) - .conflictDetectionFilter(rowFilter) - .validateNoConflictingData(), - branch)); + Assertions.assertThatThrownBy( + () -> + commit( + table, + table + .newOverwrite() + .addFile(FILE_DAY_2_MODIFIED) + .conflictDetectionFilter(rowFilter) + .validateNoConflictingData(), + branch)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Cannot find field 'dAtE'"); + + Assertions.assertThatThrownBy( + () -> + commit( + table, + table + .newOverwrite() + .caseSensitive(true) + .addFile(FILE_DAY_2_MODIFIED) + .conflictDetectionFilter(rowFilter) + .validateNoConflictingData(), + branch)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Cannot find field 'dAtE'"); // binding should succeed and trigger the validation - AssertHelpers.assertThrows( - "Should trigger the validation", - ValidationException.class, - "Found conflicting files", - () -> - commit( - table, - table - .newOverwrite() - .caseSensitive(false) - .addFile(FILE_DAY_2_MODIFIED) - .conflictDetectionFilter(rowFilter) - .validateNoConflictingData(), - branch)); + Assertions.assertThatThrownBy( + () -> + commit( + table, + table + .newOverwrite() + .caseSensitive(false) + .addFile(FILE_DAY_2_MODIFIED) + .conflictDetectionFilter(rowFilter) + .validateNoConflictingData(), + branch)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Found conflicting files"); } @Test diff --git a/core/src/test/java/org/apache/iceberg/TestPartitioning.java b/core/src/test/java/org/apache/iceberg/TestPartitioning.java index 21558be3da3e..4de62e3cfee3 100644 --- a/core/src/test/java/org/apache/iceberg/TestPartitioning.java +++ b/core/src/test/java/org/apache/iceberg/TestPartitioning.java @@ -27,6 +27,7 @@ import org.apache.iceberg.types.Types; import org.apache.iceberg.types.Types.NestedField; import org.apache.iceberg.types.Types.StructType; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -167,11 +168,9 @@ public void testPartitionTypeWithIncompatibleSpecEvolution() { Assert.assertEquals("Should have 2 specs", 2, table.specs().size()); - AssertHelpers.assertThrows( - "Should complain about incompatible specs", - ValidationException.class, - "Conflicting partition fields", - () -> Partitioning.partitionType(table)); + Assertions.assertThatThrownBy(() -> Partitioning.partitionType(table)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Conflicting partition fields"); } @Test @@ -380,11 +379,10 @@ public void testGroupingKeyTypeWithIncompatibleSpecEvolution() { Assert.assertEquals("Should have 2 specs", 2, table.specs().size()); - AssertHelpers.assertThrows( - "Should complain about incompatible specs", - ValidationException.class, - "Conflicting partition fields", - () -> Partitioning.groupingKeyType(table.schema(), table.specs().values())); + Assertions.assertThatThrownBy( + () -> Partitioning.groupingKeyType(table.schema(), table.specs().values())) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Conflicting partition fields"); } @Test diff --git a/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java b/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java index bad063529383..71455c571282 100644 --- a/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java +++ b/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java @@ -671,11 +671,9 @@ public void testRetainLastMultipleCalls() { @Test public void testRetainZeroSnapshots() { - AssertHelpers.assertThrows( - "Should fail retain 0 snapshots " + "because number of snapshots to retain cannot be zero", - IllegalArgumentException.class, - "Number of snapshots to retain must be at least 1, cannot be: 0", - () -> removeSnapshots(table).retainLast(0).commit()); + Assertions.assertThatThrownBy(() -> removeSnapshots(table).retainLast(0).commit()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Number of snapshots to retain must be at least 1, cannot be: 0"); } @Test @@ -1039,11 +1037,9 @@ public void testExpireSnapshotsWhenGarbageCollectionDisabled() { table.newAppend().appendFile(FILE_A).commit(); - AssertHelpers.assertThrows( - "Should complain about expiring snapshots", - ValidationException.class, - "Cannot expire snapshots: GC is disabled", - () -> table.expireSnapshots()); + Assertions.assertThatThrownBy(() -> table.expireSnapshots()) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Cannot expire snapshots: GC is disabled"); } @Test @@ -1231,16 +1227,15 @@ public void testMultipleRefsAndCleanExpiredFilesFailsForIncrementalCleanup() { waitUntilAfter(table.currentSnapshot().timestampMillis()); RemoveSnapshots removeSnapshots = (RemoveSnapshots) table.expireSnapshots(); - AssertHelpers.assertThrows( - "Should fail removing snapshots and files when there is more than 1 ref", - UnsupportedOperationException.class, - "Cannot incrementally clean files for tables with more than 1 ref", - () -> - removeSnapshots - .withIncrementalCleanup(true) - .expireOlderThan(table.currentSnapshot().timestampMillis()) - .cleanExpiredFiles(true) - .commit()); + Assertions.assertThatThrownBy( + () -> + removeSnapshots + .withIncrementalCleanup(true) + .expireOlderThan(table.currentSnapshot().timestampMillis()) + .cleanExpiredFiles(true) + .commit()) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage("Cannot incrementally clean files for tables with more than 1 ref"); } @Test @@ -1330,11 +1325,10 @@ public void testFailRemovingSnapshotWhenStillReferencedByBranch() { table.manageSnapshots().createBranch("branch", snapshotId).commit(); - AssertHelpers.assertThrows( - "Should fail removing snapshot when it is still referenced", - IllegalArgumentException.class, - "Cannot expire 2. Still referenced by refs: [branch]", - () -> removeSnapshots(table).expireSnapshotId(snapshotId).commit()); + Assertions.assertThatThrownBy( + () -> removeSnapshots(table).expireSnapshotId(snapshotId).commit()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot expire 2. Still referenced by refs: [branch]"); } @Test @@ -1348,11 +1342,10 @@ public void testFailRemovingSnapshotWhenStillReferencedByTag() { // commit another snapshot so the first one isn't referenced by main table.newAppend().appendFile(FILE_B).commit(); - AssertHelpers.assertThrows( - "Should fail removing snapshot when it is still referenced", - IllegalArgumentException.class, - "Cannot expire 1. Still referenced by refs: [tag]", - () -> removeSnapshots(table).expireSnapshotId(snapshotId).commit()); + Assertions.assertThatThrownBy( + () -> removeSnapshots(table).expireSnapshotId(snapshotId).commit()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot expire 1. Still referenced by refs: [tag]"); } @Test diff --git a/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java b/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java index 0fe760d96e16..e657e7fc43ca 100644 --- a/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java +++ b/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java @@ -24,6 +24,7 @@ import java.io.IOException; import org.apache.iceberg.ManifestEntry.Status; import org.apache.iceberg.exceptions.ValidationException; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Assume; import org.junit.Test; @@ -244,11 +245,9 @@ public void testValidationFailure() { ReplacePartitions replace = table.newReplacePartitions().addFile(FILE_F).addFile(FILE_G).validateAppendOnly(); - AssertHelpers.assertThrows( - "Should reject commit with file not matching delete expression", - ValidationException.class, - "Cannot commit file that conflicts with existing partition", - () -> commit(table, replace, branch)); + Assertions.assertThatThrownBy(() -> commit(table, replace, branch)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Cannot commit file that conflicts with existing partition"); Assert.assertEquals( "Should not create a new snapshot", @@ -332,20 +331,20 @@ public void testValidateWithDefaultSnapshotId() { // Concurrent Replace Partitions should fail with ValidationException ReplacePartitions replace = table.newReplacePartitions(); - AssertHelpers.assertThrows( - "Should reject commit with file matching partitions replaced", - ValidationException.class, - "Found conflicting files that can contain records matching partitions " - + "[data_bucket=0, data_bucket=1]: [/path/to/data-a.parquet]", - () -> - commit( - table, - replace - .addFile(FILE_A) - .addFile(FILE_B) - .validateNoConflictingData() - .validateNoConflictingDeletes(), - branch)); + Assertions.assertThatThrownBy( + () -> + commit( + table, + replace + .addFile(FILE_A) + .addFile(FILE_B) + .validateNoConflictingData() + .validateNoConflictingDeletes(), + branch)) + .isInstanceOf(ValidationException.class) + .hasMessage( + "Found conflicting files that can contain records matching partitions " + + "[data_bucket=0, data_bucket=1]: [/path/to/data-a.parquet]"); } @Test @@ -358,22 +357,22 @@ public void testConcurrentReplaceConflict() { // Concurrent Replace Partitions should fail with ValidationException commit(table, table.newReplacePartitions().addFile(FILE_A), branch); - AssertHelpers.assertThrows( - "Should reject commit with file matching partitions replaced", - ValidationException.class, - "Found conflicting files that can contain records matching partitions " - + "[data_bucket=0, data_bucket=1]: [/path/to/data-a.parquet]", - () -> - commit( - table, - table - .newReplacePartitions() - .validateFromSnapshot(baseId) - .addFile(FILE_A) - .addFile(FILE_B) - .validateNoConflictingData() - .validateNoConflictingDeletes(), - branch)); + Assertions.assertThatThrownBy( + () -> + commit( + table, + table + .newReplacePartitions() + .validateFromSnapshot(baseId) + .addFile(FILE_A) + .addFile(FILE_B) + .validateNoConflictingData() + .validateNoConflictingDeletes(), + branch)) + .isInstanceOf(ValidationException.class) + .hasMessage( + "Found conflicting files that can contain records matching partitions " + + "[data_bucket=0, data_bucket=1]: [/path/to/data-a.parquet]"); } @Test @@ -427,21 +426,21 @@ public void testConcurrentReplaceConflictNonPartitioned() { // Concurrent ReplacePartitions should fail with ValidationException commit(table, unpartitioned.newReplacePartitions().addFile(FILE_UNPARTITIONED_A), branch); - AssertHelpers.assertThrows( - "Should reject commit with file matching partitions replaced", - ValidationException.class, - "Found conflicting files that can contain records matching true: " - + "[/path/to/data-unpartitioned-a.parquet]", - () -> - commit( - table, - unpartitioned - .newReplacePartitions() - .validateFromSnapshot(replaceBaseId) - .validateNoConflictingData() - .validateNoConflictingDeletes() - .addFile(FILE_UNPARTITIONED_A), - branch)); + Assertions.assertThatThrownBy( + () -> + commit( + table, + unpartitioned + .newReplacePartitions() + .validateFromSnapshot(replaceBaseId) + .validateNoConflictingData() + .validateNoConflictingDeletes() + .addFile(FILE_UNPARTITIONED_A), + branch)) + .isInstanceOf(ValidationException.class) + .hasMessage( + "Found conflicting files that can contain records matching true: " + + "[/path/to/data-unpartitioned-a.parquet]"); } @Test @@ -454,22 +453,22 @@ public void testAppendReplaceConflict() { // Concurrent Append and ReplacePartition should fail with ValidationException commit(table, table.newFastAppend().appendFile(FILE_B), branch); - AssertHelpers.assertThrows( - "Should reject commit with file matching partitions replaced", - ValidationException.class, - "Found conflicting files that can contain records matching partitions " - + "[data_bucket=0, data_bucket=1]: [/path/to/data-b.parquet]", - () -> - commit( - table, - table - .newReplacePartitions() - .validateFromSnapshot(baseId) - .validateNoConflictingData() - .validateNoConflictingDeletes() - .addFile(FILE_A) - .addFile(FILE_B), - branch)); + Assertions.assertThatThrownBy( + () -> + commit( + table, + table + .newReplacePartitions() + .validateFromSnapshot(baseId) + .validateNoConflictingData() + .validateNoConflictingDeletes() + .addFile(FILE_A) + .addFile(FILE_B), + branch)) + .isInstanceOf(ValidationException.class) + .hasMessage( + "Found conflicting files that can contain records matching partitions " + + "[data_bucket=0, data_bucket=1]: [/path/to/data-b.parquet]"); } @Test @@ -529,21 +528,21 @@ public void testAppendReplaceConflictNonPartitioned() { // Concurrent Append and ReplacePartitions should fail with ValidationException commit(table, unpartitioned.newAppend().appendFile(FILE_UNPARTITIONED_A), branch); - AssertHelpers.assertThrows( - "Should reject commit with file matching partitions replaced", - ValidationException.class, - "Found conflicting files that can contain records matching true: " - + "[/path/to/data-unpartitioned-a.parquet]", - () -> - commit( - table, - unpartitioned - .newReplacePartitions() - .validateFromSnapshot(replaceBaseId) - .validateNoConflictingData() - .validateNoConflictingDeletes() - .addFile(FILE_UNPARTITIONED_A), - branch)); + Assertions.assertThatThrownBy( + () -> + commit( + table, + unpartitioned + .newReplacePartitions() + .validateFromSnapshot(replaceBaseId) + .validateNoConflictingData() + .validateNoConflictingDeletes() + .addFile(FILE_UNPARTITIONED_A), + branch)) + .isInstanceOf(ValidationException.class) + .hasMessage( + "Found conflicting files that can contain records matching true: " + + "[/path/to/data-unpartitioned-a.parquet]"); } @Test @@ -558,21 +557,21 @@ public void testDeleteReplaceConflict() { commit( table, table.newRowDelta().addDeletes(FILE_A_DELETES).validateFromSnapshot(baseId), branch); - AssertHelpers.assertThrows( - "Should reject commit with file matching partitions replaced", - ValidationException.class, - "Found new conflicting delete files that can apply to records matching " - + "[data_bucket=0]: [/path/to/data-a-deletes.parquet]", - () -> - commit( - table, - table - .newReplacePartitions() - .validateFromSnapshot(baseId) - .validateNoConflictingData() - .validateNoConflictingDeletes() - .addFile(FILE_A), - branch)); + Assertions.assertThatThrownBy( + () -> + commit( + table, + table + .newReplacePartitions() + .validateFromSnapshot(baseId) + .validateNoConflictingData() + .validateNoConflictingDeletes() + .addFile(FILE_A), + branch)) + .isInstanceOf(ValidationException.class) + .hasMessage( + "Found new conflicting delete files that can apply to records matching " + + "[data_bucket=0]: [/path/to/data-a-deletes.parquet]"); } @Test @@ -590,21 +589,21 @@ public void testDeleteReplaceConflictNonPartitioned() { // Concurrent Delete and ReplacePartitions should fail with ValidationException commit(table, unpartitioned.newRowDelta().addDeletes(FILE_UNPARTITIONED_A_DELETES), branch); - AssertHelpers.assertThrows( - "Should reject commit with file matching partitions replaced", - ValidationException.class, - "Found new conflicting delete files that can apply to records matching true: " - + "[/path/to/data-unpartitioned-a-deletes.parquet]", - () -> - commit( - table, - unpartitioned - .newReplacePartitions() - .validateFromSnapshot(replaceBaseId) - .validateNoConflictingData() - .validateNoConflictingDeletes() - .addFile(FILE_UNPARTITIONED_A), - branch)); + Assertions.assertThatThrownBy( + () -> + commit( + table, + unpartitioned + .newReplacePartitions() + .validateFromSnapshot(replaceBaseId) + .validateNoConflictingData() + .validateNoConflictingDeletes() + .addFile(FILE_UNPARTITIONED_A), + branch)) + .isInstanceOf(ValidationException.class) + .hasMessage( + "Found new conflicting delete files that can apply to records matching true: " + + "[/path/to/data-unpartitioned-a-deletes.parquet]"); } @Test @@ -673,21 +672,21 @@ public void testOverwriteReplaceConflict() { // Concurrent Overwrite and ReplacePartition should fail with ValidationException commit(table, table.newOverwrite().deleteFile(FILE_A), branch); - AssertHelpers.assertThrows( - "Should reject commit with file matching partitions replaced", - ValidationException.class, - "Found conflicting deleted files that can apply to records matching " - + "[data_bucket=0]: [/path/to/data-a.parquet]", - () -> - commit( - table, - table - .newReplacePartitions() - .validateFromSnapshot(baseId) - .validateNoConflictingData() - .validateNoConflictingDeletes() - .addFile(FILE_A), - branch)); + Assertions.assertThatThrownBy( + () -> + commit( + table, + table + .newReplacePartitions() + .validateFromSnapshot(baseId) + .validateNoConflictingData() + .validateNoConflictingDeletes() + .addFile(FILE_A), + branch)) + .isInstanceOf(ValidationException.class) + .hasMessage( + "Found conflicting deleted files that can apply to records matching " + + "[data_bucket=0]: [/path/to/data-a.parquet]"); } @Test @@ -746,21 +745,21 @@ public void testOverwriteReplaceConflictNonPartitioned() { // Concurrent Overwrite and ReplacePartitions should fail with ValidationException commit(table, unpartitioned.newOverwrite().deleteFile(FILE_UNPARTITIONED_A), branch); - AssertHelpers.assertThrows( - "Should reject commit with file matching partitions replaced", - ValidationException.class, - "Found conflicting deleted files that can contain records matching true: " - + "[/path/to/data-unpartitioned-a.parquet]", - () -> - commit( - table, - unpartitioned - .newReplacePartitions() - .validateFromSnapshot(replaceBaseId) - .validateNoConflictingData() - .validateNoConflictingDeletes() - .addFile(FILE_UNPARTITIONED_A), - branch)); + Assertions.assertThatThrownBy( + () -> + commit( + table, + unpartitioned + .newReplacePartitions() + .validateFromSnapshot(replaceBaseId) + .validateNoConflictingData() + .validateNoConflictingDeletes() + .addFile(FILE_UNPARTITIONED_A), + branch)) + .isInstanceOf(ValidationException.class) + .hasMessage( + "Found conflicting deleted files that can contain records matching true: " + + "[/path/to/data-unpartitioned-a.parquet]"); } @Test diff --git a/core/src/test/java/org/apache/iceberg/TestReplaceTransaction.java b/core/src/test/java/org/apache/iceberg/TestReplaceTransaction.java index fd01b5f7db5d..b338d00696dd 100644 --- a/core/src/test/java/org/apache/iceberg/TestReplaceTransaction.java +++ b/core/src/test/java/org/apache/iceberg/TestReplaceTransaction.java @@ -37,6 +37,7 @@ import org.apache.iceberg.transforms.Transforms; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Assume; import org.junit.Test; @@ -246,11 +247,9 @@ public void testReplaceDetectsUncommittedChangeOnCommit() { .appendFile(FILE_C) .appendFile(FILE_D); - AssertHelpers.assertThrows( - "Should reject commit when last operation has not committed", - IllegalStateException.class, - "Cannot commit transaction: last operation has not committed", - replace::commitTransaction); + Assertions.assertThatThrownBy(replace::commitTransaction) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Cannot commit transaction: last operation has not committed"); Assert.assertEquals("Version should be 0", 0L, (long) version()); } @@ -268,11 +267,9 @@ public void testReplaceDetectsUncommittedChangeOnTableCommit() { .appendFile(FILE_C) .appendFile(FILE_D); - AssertHelpers.assertThrows( - "Should reject commit when last operation has not committed", - IllegalStateException.class, - "Cannot commit transaction: last operation has not committed", - replace::commitTransaction); + Assertions.assertThatThrownBy(replace::commitTransaction) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Cannot commit transaction: last operation has not committed"); Assert.assertEquals("Version should be 0", 0L, (long) version()); } @@ -327,11 +324,9 @@ public void testReplaceTransactionConflict() { // keep failing to trigger eventual transaction failure ((TestTables.TestTableOperations) ((BaseTransaction) replace).ops()).failCommits(100); - AssertHelpers.assertThrows( - "Should reject commit when retries are exhausted", - CommitFailedException.class, - "Injected failure", - replace::commitTransaction); + Assertions.assertThatThrownBy(replace::commitTransaction) + .isInstanceOf(CommitFailedException.class) + .hasMessage("Injected failure"); Assert.assertEquals("Version should be 1", 1L, (long) version()); @@ -414,11 +409,9 @@ public void testReplaceTransactionWithUnknownState() { replace.newAppend().appendFile(FILE_B).commit(); - AssertHelpers.assertThrows( - "Transaction commit should fail with CommitStateUnknownException", - CommitStateUnknownException.class, - "datacenter on fire", - () -> replace.commitTransaction()); + Assertions.assertThatThrownBy(replace::commitTransaction) + .isInstanceOf(CommitStateUnknownException.class) + .hasMessageStartingWith("datacenter on fire"); table.refresh(); @@ -466,11 +459,9 @@ public void testCreateTransactionWithUnknownState() throws IOException { TestTables.readMetadata("test_append")); Assert.assertNull("Should have no metadata version", TestTables.metadataVersion("test_append")); - AssertHelpers.assertThrows( - "Transaction commit should fail with CommitStateUnknownException", - CommitStateUnknownException.class, - "datacenter on fire", - () -> replace.commitTransaction()); + Assertions.assertThatThrownBy(replace::commitTransaction) + .isInstanceOf(CommitStateUnknownException.class) + .hasMessageStartingWith("datacenter on fire"); TableMetadata meta = TestTables.readMetadata("test_append"); Assert.assertNotNull("Table metadata should be created after transaction commits", meta); diff --git a/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java b/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java index 256c0c94cced..5b868d3d3642 100644 --- a/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java +++ b/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java @@ -30,6 +30,7 @@ import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Assume; import org.junit.Test; @@ -64,119 +65,113 @@ public void testEmptyTable() { TableMetadata base = readMetadata(); Assert.assertNull("Should not have a current snapshot", base.ref(branch)); - AssertHelpers.assertThrows( - "Expected an exception", - ValidationException.class, - "Missing required files to delete: /path/to/data-a.parquet", - () -> - commit( - table, - table.newRewrite().rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_B)), - branch)); - - AssertHelpers.assertThrows( - "Expected an exception", - ValidationException.class, - "Missing required files to delete: /path/to/data-a-deletes.parquet", - () -> - commit( - table, - table - .newRewrite() - .rewriteFiles( - ImmutableSet.of(), - ImmutableSet.of(FILE_A_DELETES), - ImmutableSet.of(), - ImmutableSet.of(FILE_B_DELETES)), - branch)); + Assertions.assertThatThrownBy( + () -> + commit( + table, + table.newRewrite().rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_B)), + branch)) + .isInstanceOf(ValidationException.class) + .hasMessage("Missing required files to delete: /path/to/data-a.parquet"); + + Assertions.assertThatThrownBy( + () -> + commit( + table, + table + .newRewrite() + .rewriteFiles( + ImmutableSet.of(), + ImmutableSet.of(FILE_A_DELETES), + ImmutableSet.of(), + ImmutableSet.of(FILE_B_DELETES)), + branch)) + .isInstanceOf(ValidationException.class) + .hasMessage("Missing required files to delete: /path/to/data-a-deletes.parquet"); } @Test public void testAddOnly() { Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); - AssertHelpers.assertThrows( - "Expected an exception", - ValidationException.class, - "Missing required files to delete: /path/to/data-a.parquet", - () -> - apply( - table.newRewrite().rewriteFiles(Sets.newSet(FILE_A), Collections.emptySet()), - branch)); - - AssertHelpers.assertThrows( - "Expected an exception", - IllegalArgumentException.class, - "Delete files to add must be empty because there's no delete file to be rewritten", - () -> - apply( - table - .newRewrite() - .rewriteFiles( - ImmutableSet.of(FILE_A), - ImmutableSet.of(), - ImmutableSet.of(), - ImmutableSet.of(FILE_A_DELETES)), - branch)); - - AssertHelpers.assertThrows( - "Expected an exception", - IllegalArgumentException.class, - "Delete files to add must be empty because there's no delete file to be rewritten", - () -> - apply( - table - .newRewrite() - .rewriteFiles( - ImmutableSet.of(FILE_A), - ImmutableSet.of(), - ImmutableSet.of(FILE_B), - ImmutableSet.of(FILE_B_DELETES)), - branch)); + Assertions.assertThatThrownBy( + () -> + apply( + table.newRewrite().rewriteFiles(Sets.newSet(FILE_A), Collections.emptySet()), + branch)) + .isInstanceOf(ValidationException.class) + .hasMessage("Missing required files to delete: /path/to/data-a.parquet"); + + Assertions.assertThatThrownBy( + () -> + apply( + table + .newRewrite() + .rewriteFiles( + ImmutableSet.of(FILE_A), + ImmutableSet.of(), + ImmutableSet.of(), + ImmutableSet.of(FILE_A_DELETES)), + branch)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Delete files to add must be empty because there's no delete file to be rewritten"); + + Assertions.assertThatThrownBy( + () -> + apply( + table + .newRewrite() + .rewriteFiles( + ImmutableSet.of(FILE_A), + ImmutableSet.of(), + ImmutableSet.of(FILE_B), + ImmutableSet.of(FILE_B_DELETES)), + branch)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Delete files to add must be empty because there's no delete file to be rewritten"); } @Test public void testDeleteOnly() { Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); - AssertHelpers.assertThrows( - "Expected an exception", - IllegalArgumentException.class, - "Files to delete cannot be empty", - () -> - apply( - table.newRewrite().rewriteFiles(Collections.emptySet(), Sets.newSet(FILE_A)), - branch)); - - AssertHelpers.assertThrows( - "Expected an exception", - IllegalArgumentException.class, - "Files to delete cannot be empty", - () -> - apply( - table - .newRewrite() - .rewriteFiles( - ImmutableSet.of(), - ImmutableSet.of(), - ImmutableSet.of(), - ImmutableSet.of(FILE_A_DELETES)), - branch)); - - AssertHelpers.assertThrows( - "Expected an exception", - IllegalArgumentException.class, - "Files to delete cannot be empty", - () -> - apply( - table - .newRewrite() - .rewriteFiles( - ImmutableSet.of(), - ImmutableSet.of(), - ImmutableSet.of(FILE_A), - ImmutableSet.of(FILE_A_DELETES)), - branch)); + Assertions.assertThatThrownBy( + () -> + apply( + table.newRewrite().rewriteFiles(Collections.emptySet(), Sets.newSet(FILE_A)), + branch)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Files to delete cannot be empty"); + + Assertions.assertThatThrownBy( + () -> + apply( + table + .newRewrite() + .rewriteFiles( + ImmutableSet.of(), + ImmutableSet.of(), + ImmutableSet.of(), + ImmutableSet.of(FILE_A_DELETES)), + branch)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Files to delete cannot be empty"); + + Assertions.assertThatThrownBy( + () -> + apply( + table + .newRewrite() + .rewriteFiles( + ImmutableSet.of(), + ImmutableSet.of(), + ImmutableSet.of(FILE_A), + ImmutableSet.of(FILE_A_DELETES)), + branch)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Files to delete cannot be empty"); } @Test @@ -432,11 +427,9 @@ public void testFailure() { validateManifestEntries(manifest1, ids(pending.snapshotId()), files(FILE_B), statuses(ADDED)); validateManifestEntries(manifest2, ids(pending.snapshotId()), files(FILE_A), statuses(DELETED)); - AssertHelpers.assertThrows( - "Should retry 4 times and throw last failure", - CommitFailedException.class, - "Injected failure", - () -> commit(table, rewrite, branch)); + Assertions.assertThatThrownBy(() -> commit(table, rewrite, branch)) + .isInstanceOf(CommitFailedException.class) + .hasMessage("Injected failure"); Assert.assertFalse("Should clean up new manifest", new File(manifest1.path()).exists()); Assert.assertFalse("Should clean up new manifest", new File(manifest2.path()).exists()); @@ -500,11 +493,9 @@ public void testFailureWhenRewriteBothDataAndDeleteFiles() { files(FILE_A_DELETES, FILE_B_DELETES), statuses(DELETED, DELETED)); - AssertHelpers.assertThrows( - "Should retry 4 times and throw last failure", - CommitFailedException.class, - "Injected failure", - rewrite::commit); + Assertions.assertThatThrownBy(rewrite::commit) + .isInstanceOf(CommitFailedException.class) + .hasMessage("Injected failure"); Assert.assertFalse("Should clean up new manifest", new File(manifest1.path()).exists()); Assert.assertFalse("Should clean up new manifest", new File(manifest2.path()).exists()); @@ -731,15 +722,14 @@ public void testDeleteNonExistentFile() { 1, latestSnapshot(base, branch).allManifests(table.io()).size()); - AssertHelpers.assertThrows( - "Expected an exception", - ValidationException.class, - "Missing required files to delete: /path/to/data-c.parquet", - () -> - commit( - table, - table.newRewrite().rewriteFiles(Sets.newSet(FILE_C), Sets.newSet(FILE_D)), - branch)); + Assertions.assertThatThrownBy( + () -> + commit( + table, + table.newRewrite().rewriteFiles(Sets.newSet(FILE_C), Sets.newSet(FILE_D)), + branch)) + .isInstanceOf(ValidationException.class) + .hasMessage("Missing required files to delete: /path/to/data-c.parquet"); Assert.assertEquals("Only 1 manifests should exist", 1, listManifestFiles().size()); } @@ -775,15 +765,14 @@ public void testAlreadyDeletedFile() { commit(table, rewrite, branch); - AssertHelpers.assertThrows( - "Expected an exception", - ValidationException.class, - "Missing required files to delete: /path/to/data-a.parquet", - () -> - commit( - table, - table.newRewrite().rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_D)), - branch)); + Assertions.assertThatThrownBy( + () -> + commit( + table, + table.newRewrite().rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_D)), + branch)) + .isInstanceOf(ValidationException.class) + .hasMessage("Missing required files to delete: /path/to/data-a.parquet"); Assert.assertEquals("Only 3 manifests should exist", 3, listManifestFiles().size()); } @@ -800,17 +789,16 @@ public void testNewDeleteFile() { long snapshotAfterDeletes = latestSnapshot(table, branch).snapshotId(); - AssertHelpers.assertThrows( - "Should fail because deletes were added after the starting snapshot", - ValidationException.class, - "Cannot commit, found new delete for replaced data file", - () -> - apply( - table - .newRewrite() - .validateFromSnapshot(snapshotBeforeDeletes) - .rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_A2)), - branch)); + Assertions.assertThatThrownBy( + () -> + apply( + table + .newRewrite() + .validateFromSnapshot(snapshotBeforeDeletes) + .rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_A2)), + branch)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Cannot commit, found new delete for replaced data file"); // the rewrite should be valid when validating from the snapshot after the deletes apply( diff --git a/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java b/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java index f6717345ccb5..dbda87baf7f5 100644 --- a/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java +++ b/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java @@ -36,6 +36,7 @@ import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; @@ -822,11 +823,9 @@ public void testManifestReplacementConcurrentConflictingDelete() throws IOExcept table.newDelete().deleteFile(FILE_A).commit(); - AssertHelpers.assertThrows( - "Should reject commit", - ValidationException.class, - "Manifest is missing", - rewriteManifests::commit); + Assertions.assertThatThrownBy(rewriteManifests::commit) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Manifest is missing"); } @Test @@ -966,16 +965,15 @@ public void testInvalidUsage() throws IOException { ManifestFile invalidAddedFileManifest = writeManifest("manifest-file-2.avro", appendEntry); - AssertHelpers.assertThrows( - "Should reject commit", - IllegalArgumentException.class, - "Cannot add manifest with added files", - () -> - table - .rewriteManifests() - .deleteManifest(manifest) - .addManifest(invalidAddedFileManifest) - .commit()); + Assertions.assertThatThrownBy( + () -> + table + .rewriteManifests() + .deleteManifest(manifest) + .addManifest(invalidAddedFileManifest) + .commit()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot add manifest with added files"); ManifestEntry deleteEntry = manifestEntry(ManifestEntry.Status.DELETED, snapshot.snapshotId(), FILE_A); @@ -984,22 +982,20 @@ public void testInvalidUsage() throws IOException { ManifestFile invalidDeletedFileManifest = writeManifest("manifest-file-3.avro", deleteEntry); - AssertHelpers.assertThrows( - "Should reject commit", - IllegalArgumentException.class, - "Cannot add manifest with deleted files", - () -> - table - .rewriteManifests() - .deleteManifest(manifest) - .addManifest(invalidDeletedFileManifest) - .commit()); - - AssertHelpers.assertThrows( - "Should reject commit", - ValidationException.class, - "must have the same number of active files", - () -> table.rewriteManifests().deleteManifest(manifest).commit()); + Assertions.assertThatThrownBy( + () -> + table + .rewriteManifests() + .deleteManifest(manifest) + .addManifest(invalidDeletedFileManifest) + .commit()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot add manifest with deleted files"); + + Assertions.assertThatThrownBy(() -> table.rewriteManifests().deleteManifest(manifest).commit()) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith( + "Replaced and created manifests must have the same number of active files"); } @Test @@ -1035,11 +1031,9 @@ public void testManifestReplacementFailure() throws IOException { rewriteManifests.deleteManifest(secondSnapshotManifest); rewriteManifests.addManifest(newManifest); - AssertHelpers.assertThrows( - "Should reject commit", - CommitFailedException.class, - "Injected failure", - rewriteManifests::commit); + Assertions.assertThatThrownBy(rewriteManifests::commit) + .isInstanceOf(CommitFailedException.class) + .hasMessage("Injected failure"); Assert.assertTrue("New manifest should not be deleted", new File(newManifest.path()).exists()); } @@ -1079,11 +1073,9 @@ public void testManifestReplacementFailureWithSnapshotIdInheritance() throws IOE rewriteManifests.deleteManifest(secondSnapshotManifest); rewriteManifests.addManifest(newManifest); - AssertHelpers.assertThrows( - "Should reject commit", - CommitFailedException.class, - "Injected failure", - rewriteManifests::commit); + Assertions.assertThatThrownBy(rewriteManifests::commit) + .isInstanceOf(CommitFailedException.class) + .hasMessage("Injected failure"); Assert.assertTrue("New manifest should not be deleted", new File(newManifest.path()).exists()); } @@ -1095,11 +1087,10 @@ public void testRewriteManifestsOnBranchUnsupported() { Assert.assertEquals(1, table.currentSnapshot().allManifests(table.io()).size()); - AssertHelpers.assertThrows( - "Should reject committing rewrite manifests to branch", - UnsupportedOperationException.class, - "Cannot commit to branch someBranch: org.apache.iceberg.BaseRewriteManifests does not support branch commits", - () -> table.rewriteManifests().toBranch("someBranch").commit()); + Assertions.assertThatThrownBy(() -> table.rewriteManifests().toBranch("someBranch").commit()) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage( + "Cannot commit to branch someBranch: org.apache.iceberg.BaseRewriteManifests does not support branch commits"); } private void validateSummary( diff --git a/core/src/test/java/org/apache/iceberg/TestRowDelta.java b/core/src/test/java/org/apache/iceberg/TestRowDelta.java index ab2ec5f71363..fa04f36d367f 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowDelta.java +++ b/core/src/test/java/org/apache/iceberg/TestRowDelta.java @@ -38,6 +38,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; @@ -115,19 +116,18 @@ public void testValidateDataFilesExistDefaults() { long deleteSnapshotId = latestSnapshot(table, branch).snapshotId(); - AssertHelpers.assertThrows( - "Should fail to add FILE_A_DELETES because FILE_A is missing", - ValidationException.class, - "Cannot commit, missing data files", - () -> - commit( - table, - table - .newRowDelta() - .addDeletes(FILE_A_DELETES) - .validateFromSnapshot(validateFromSnapshotId) - .validateDataFilesExist(ImmutableList.of(FILE_A.path())), - branch)); + Assertions.assertThatThrownBy( + () -> + commit( + table, + table + .newRowDelta() + .addDeletes(FILE_A_DELETES) + .validateFromSnapshot(validateFromSnapshotId) + .validateDataFilesExist(ImmutableList.of(FILE_A.path())), + branch)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Cannot commit, missing data files"); Assert.assertEquals( "Table state should not be modified by failed RowDelta operation", @@ -174,19 +174,18 @@ public void testValidateDataFilesExistOverwrite() { long deleteSnapshotId = latestSnapshot(table, branch).snapshotId(); - AssertHelpers.assertThrows( - "Should fail to add FILE_A_DELETES because FILE_A is missing", - ValidationException.class, - "Cannot commit, missing data files", - () -> - commit( - table, - table - .newRowDelta() - .addDeletes(FILE_A_DELETES) - .validateFromSnapshot(validateFromSnapshotId) - .validateDataFilesExist(ImmutableList.of(FILE_A.path())), - branch)); + Assertions.assertThatThrownBy( + () -> + commit( + table, + table + .newRowDelta() + .addDeletes(FILE_A_DELETES) + .validateFromSnapshot(validateFromSnapshotId) + .validateDataFilesExist(ImmutableList.of(FILE_A.path())), + branch)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Cannot commit, missing data files"); Assert.assertEquals( "Table state should not be modified by failed RowDelta operation", @@ -211,19 +210,18 @@ public void testValidateDataFilesExistReplacePartitions() { long deleteSnapshotId = latestSnapshot(table, branch).snapshotId(); - AssertHelpers.assertThrows( - "Should fail to add FILE_A_DELETES because FILE_A is missing", - ValidationException.class, - "Cannot commit, missing data files", - () -> - commit( - table, - table - .newRowDelta() - .addDeletes(FILE_A_DELETES) - .validateFromSnapshot(validateFromSnapshotId) - .validateDataFilesExist(ImmutableList.of(FILE_A.path())), - branch)); + Assertions.assertThatThrownBy( + () -> + commit( + table, + table + .newRowDelta() + .addDeletes(FILE_A_DELETES) + .validateFromSnapshot(validateFromSnapshotId) + .validateDataFilesExist(ImmutableList.of(FILE_A.path())), + branch)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Cannot commit, missing data files"); Assert.assertEquals( "Table state should not be modified by failed RowDelta operation", @@ -310,19 +308,18 @@ public void testValidateDataFilesExistRewrite() { long deleteSnapshotId = latestSnapshot(table, branch).snapshotId(); - AssertHelpers.assertThrows( - "Should fail to add FILE_A_DELETES because FILE_A is missing", - ValidationException.class, - "Cannot commit, missing data files", - () -> - commit( - table, - table - .newRowDelta() - .addDeletes(FILE_A_DELETES) - .validateFromSnapshot(validateFromSnapshotId) - .validateDataFilesExist(ImmutableList.of(FILE_A.path())), - branch)); + Assertions.assertThatThrownBy( + () -> + commit( + table, + table + .newRowDelta() + .addDeletes(FILE_A_DELETES) + .validateFromSnapshot(validateFromSnapshotId) + .validateDataFilesExist(ImmutableList.of(FILE_A.path())), + branch)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Cannot commit, missing data files"); Assert.assertEquals( "Table state should not be modified by failed RowDelta operation", @@ -347,20 +344,19 @@ public void testValidateDataFilesExistValidateDeletes() { long deleteSnapshotId = latestSnapshot(table, branch).snapshotId(); - AssertHelpers.assertThrows( - "Should fail to add FILE_A_DELETES because FILE_A is missing", - ValidationException.class, - "Cannot commit, missing data files", - () -> - commit( - table, - table - .newRowDelta() - .addDeletes(FILE_A_DELETES) - .validateDeletedFiles() - .validateFromSnapshot(validateFromSnapshotId) - .validateDataFilesExist(ImmutableList.of(FILE_A.path())), - branch)); + Assertions.assertThatThrownBy( + () -> + commit( + table, + table + .newRowDelta() + .addDeletes(FILE_A_DELETES) + .validateDeletedFiles() + .validateFromSnapshot(validateFromSnapshotId) + .validateDataFilesExist(ImmutableList.of(FILE_A.path())), + branch)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Cannot commit, missing data files"); Assert.assertEquals( "Table state should not be modified by failed RowDelta operation", @@ -385,20 +381,20 @@ public void testValidateNoConflicts() { long appendSnapshotId = latestSnapshot(table, branch).snapshotId(); - AssertHelpers.assertThrows( - "Should fail to add FILE_A_DELETES because FILE_A2 was added", - ValidationException.class, - "Found conflicting files", - () -> - commit( - table, - table - .newRowDelta() - .addDeletes(FILE_A_DELETES) - .validateFromSnapshot(validateFromSnapshotId) - .conflictDetectionFilter(Expressions.equal("data", "u")) // bucket16("u") -> 0 - .validateNoConflictingDataFiles(), - branch)); + Assertions.assertThatThrownBy( + () -> + commit( + table, + table + .newRowDelta() + .addDeletes(FILE_A_DELETES) + .validateFromSnapshot(validateFromSnapshotId) + .conflictDetectionFilter( + Expressions.equal("data", "u")) // bucket16("u") -> 0 + .validateNoConflictingDataFiles(), + branch)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Found conflicting files"); Assert.assertEquals( "Table state should not be modified by failed RowDelta operation", @@ -885,11 +881,9 @@ public void testValidateDataFilesDoNotExistWithConflictDetectionFilter() { // concurrently delete the file for partition A commit(table, table.newDelete().deleteFile(dataFile1), branch); - AssertHelpers.assertThrows( - "Should fail to add deletes because data file is missing", - ValidationException.class, - "Cannot commit, missing data files", - () -> commit(table, rowDelta, branch)); + Assertions.assertThatThrownBy(() -> commit(table, rowDelta, branch)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Cannot commit, missing data files"); } @Test @@ -1133,11 +1127,9 @@ public void testAbortMultipleSpecs() { // perform a conflicting concurrent operation commit(table, table.newDelete().deleteFile(firstSnapshotDataFile), branch); - AssertHelpers.assertThrows( - "Should fail to commit row delta", - ValidationException.class, - "Cannot commit, missing data files", - () -> commit(table, rowDelta, branch)); + Assertions.assertThatThrownBy(() -> commit(table, rowDelta, branch)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Cannot commit, missing data files"); // we should clean up 1 manifest list and 2 delete manifests Assert.assertEquals("Should delete 3 files", 3, deletedFiles.size()); @@ -1172,11 +1164,9 @@ public void testConcurrentConflictingRowDelta() { .validateNoConflictingDataFiles() .commit(); - AssertHelpers.assertThrows( - "Should reject commit", - ValidationException.class, - "Found new conflicting delete files", - () -> commit(table, rowDelta, branch)); + Assertions.assertThatThrownBy(() -> commit(table, rowDelta, branch)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Found new conflicting delete files"); } @Test @@ -1205,11 +1195,9 @@ public void testConcurrentConflictingRowDeltaWithoutAppendValidation() { .validateNoConflictingDataFiles() .commit(); - AssertHelpers.assertThrows( - "Should reject commit", - ValidationException.class, - "Found new conflicting delete files", - () -> commit(table, rowDelta, branch)); + Assertions.assertThatThrownBy(() -> commit(table, rowDelta, branch)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Found new conflicting delete files"); } @Test @@ -1445,11 +1433,9 @@ public void testConcurrentConflictingRowDeltaAndRewriteFilesWithSequenceNumber() commit(table, rowDelta, branch); - AssertHelpers.assertThrows( - "Should not allow any new position delete associated with the data file", - ValidationException.class, - "Cannot commit, found new position delete for replaced data file", - () -> commit(table, rewriteFiles, branch)); + Assertions.assertThatThrownBy(() -> commit(table, rewriteFiles, branch)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Cannot commit, found new position delete for replaced data file"); } @Test @@ -1462,55 +1448,52 @@ public void testRowDeltaCaseSensitivity() { Expression conflictDetectionFilter = Expressions.equal(Expressions.bucket("dAtA", 16), 0); - AssertHelpers.assertThrows( - "Should use case sensitive binding by default", - ValidationException.class, - "Cannot find field 'dAtA'", - () -> - table - .newRowDelta() - .toBranch(branch) - .addRows(FILE_B) - .addDeletes(FILE_A2_DELETES) - .validateFromSnapshot(firstSnapshot.snapshotId()) - .conflictDetectionFilter(conflictDetectionFilter) - .validateNoConflictingDataFiles() - .validateNoConflictingDeleteFiles() - .commit()); - - AssertHelpers.assertThrows( - "Should fail with case sensitive binding", - ValidationException.class, - "Cannot find field 'dAtA'", - () -> - table - .newRowDelta() - .toBranch(branch) - .caseSensitive(true) - .addRows(FILE_B) - .addDeletes(FILE_A2_DELETES) - .validateFromSnapshot(firstSnapshot.snapshotId()) - .conflictDetectionFilter(conflictDetectionFilter) - .validateNoConflictingDataFiles() - .validateNoConflictingDeleteFiles() - .commit()); + Assertions.assertThatThrownBy( + () -> + table + .newRowDelta() + .toBranch(branch) + .addRows(FILE_B) + .addDeletes(FILE_A2_DELETES) + .validateFromSnapshot(firstSnapshot.snapshotId()) + .conflictDetectionFilter(conflictDetectionFilter) + .validateNoConflictingDataFiles() + .validateNoConflictingDeleteFiles() + .commit()) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Cannot find field 'dAtA'"); + + Assertions.assertThatThrownBy( + () -> + table + .newRowDelta() + .toBranch(branch) + .caseSensitive(true) + .addRows(FILE_B) + .addDeletes(FILE_A2_DELETES) + .validateFromSnapshot(firstSnapshot.snapshotId()) + .conflictDetectionFilter(conflictDetectionFilter) + .validateNoConflictingDataFiles() + .validateNoConflictingDeleteFiles() + .commit()) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Cannot find field 'dAtA'"); // binding should succeed and trigger the validation - AssertHelpers.assertThrows( - "Should reject case sensitive binding", - ValidationException.class, - "Found new conflicting delete files", - () -> - table - .newRowDelta() - .toBranch(branch) - .caseSensitive(false) - .addRows(FILE_B) - .addDeletes(FILE_A2_DELETES) - .validateFromSnapshot(firstSnapshot.snapshotId()) - .conflictDetectionFilter(conflictDetectionFilter) - .validateNoConflictingDataFiles() - .validateNoConflictingDeleteFiles() - .commit()); + Assertions.assertThatThrownBy( + () -> + table + .newRowDelta() + .toBranch(branch) + .caseSensitive(false) + .addRows(FILE_B) + .addDeletes(FILE_A2_DELETES) + .validateFromSnapshot(firstSnapshot.snapshotId()) + .conflictDetectionFilter(conflictDetectionFilter) + .validateNoConflictingDataFiles() + .validateNoConflictingDeleteFiles() + .commit()) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Found new conflicting delete files"); } } diff --git a/core/src/test/java/org/apache/iceberg/TestScanSummary.java b/core/src/test/java/org/apache/iceberg/TestScanSummary.java index 5f54eecc2638..59986016614a 100644 --- a/core/src/test/java/org/apache/iceberg/TestScanSummary.java +++ b/core/src/test/java/org/apache/iceberg/TestScanSummary.java @@ -29,6 +29,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.Pair; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; @@ -89,11 +90,9 @@ public void testSnapshotTimeRangeValidation() { .filter(greaterThanOrEqual("dateCreated", t0)) .filter(lessThan("dateCreated", t2)); - AssertHelpers.assertThrows( - "Should fail summary because range may include expired snapshots", - IllegalArgumentException.class, - "may include expired snapshots", - () -> new ScanSummary.Builder(scan).build()); + Assertions.assertThatThrownBy(() -> new ScanSummary.Builder(scan).build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot satisfy time filters: time range may include expired snapshots"); } @Test @@ -147,13 +146,12 @@ public void testTimestampRanges() { ImmutableList.of(greaterThanOrEqual("ts_ms", lower), lessThan("ts_ms", upper)))); // >= lower and < lower is an empty range - AssertHelpers.assertThrows( - "Should reject empty ranges", - IllegalArgumentException.class, - "No timestamps can match filters", - () -> - timestampRange( - ImmutableList.of(greaterThanOrEqual("ts_ms", lower), lessThan("ts_ms", lower)))); + Assertions.assertThatThrownBy( + () -> + timestampRange( + ImmutableList.of(greaterThanOrEqual("ts_ms", lower), lessThan("ts_ms", lower)))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("No timestamps can match filters"); } @Test diff --git a/core/src/test/java/org/apache/iceberg/TestSchemaAndMappingUpdate.java b/core/src/test/java/org/apache/iceberg/TestSchemaAndMappingUpdate.java index 7db9ec7fdc97..5b8aff686fbc 100644 --- a/core/src/test/java/org/apache/iceberg/TestSchemaAndMappingUpdate.java +++ b/core/src/test/java/org/apache/iceberg/TestSchemaAndMappingUpdate.java @@ -32,6 +32,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; @@ -170,16 +171,16 @@ public void testModificationWithMetricsMetrics() { .set("write.metadata.metrics.column.id", "full") .commit(); - AssertHelpers.assertThrows( - "Creating metrics for non-existent column fails", - ValidationException.class, - null, - () -> - table - .updateProperties() - .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson) - .set("write.metadata.metrics.column.ids", "full") - .commit()); + Assertions.assertThatThrownBy( + () -> + table + .updateProperties() + .set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson) + .set("write.metadata.metrics.column.ids", "full") + .commit()) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith( + "Invalid metrics config, could not find column ids from table prop write.metadata.metrics.column.ids in schema table"); // Re-naming a column with metrics succeeds; table.updateSchema().renameColumn("id", "bloop").commit(); diff --git a/core/src/test/java/org/apache/iceberg/TestSchemaUnionByFieldName.java b/core/src/test/java/org/apache/iceberg/TestSchemaUnionByFieldName.java index e43a19a9299b..5aedde6ce5b0 100644 --- a/core/src/test/java/org/apache/iceberg/TestSchemaUnionByFieldName.java +++ b/core/src/test/java/org/apache/iceberg/TestSchemaUnionByFieldName.java @@ -43,8 +43,7 @@ import org.apache.iceberg.types.Types.TimestampType; import org.apache.iceberg.types.Types.UUIDType; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestSchemaUnionByFieldName { @@ -83,7 +82,7 @@ private static NestedField[] primitiveFields( public void testAddTopLevelPrimitives() { Schema newSchema = new Schema(primitiveFields(0, primitiveTypes())); Schema applied = new SchemaUpdate(new Schema(), 0).unionByNameWith(newSchema).apply(); - Assert.assertEquals(newSchema.asStruct(), applied.asStruct()); + Assertions.assertThat(applied.asStruct()).isEqualTo(newSchema.asStruct()); } @Test @@ -92,7 +91,7 @@ public void testAddTopLevelListOfPrimitives() { Schema newSchema = new Schema(optional(1, "aList", Types.ListType.ofOptional(2, primitiveType))); Schema applied = new SchemaUpdate(new Schema(), 0).unionByNameWith(newSchema).apply(); - Assert.assertEquals(newSchema.asStruct(), applied.asStruct()); + Assertions.assertThat(applied.asStruct()).isEqualTo(newSchema.asStruct()); } } @@ -103,7 +102,7 @@ public void testAddTopLevelMapOfPrimitives() { new Schema( optional(1, "aMap", Types.MapType.ofOptional(2, 3, primitiveType, primitiveType))); Schema applied = new SchemaUpdate(new Schema(), 0).unionByNameWith(newSchema).apply(); - Assert.assertEquals(newSchema.asStruct(), applied.asStruct()); + Assertions.assertThat(applied.asStruct()).isEqualTo(newSchema.asStruct()); } } @@ -114,7 +113,7 @@ public void testAddTopLevelStructOfPrimitives() { new Schema( optional(1, "aStruct", Types.StructType.of(optional(2, "primitive", primitiveType)))); Schema applied = new SchemaUpdate(new Schema(), 0).unionByNameWith(currentSchema).apply(); - Assert.assertEquals(currentSchema.asStruct(), applied.asStruct()); + Assertions.assertThat(applied.asStruct()).isEqualTo(currentSchema.asStruct()); } } @@ -126,7 +125,7 @@ public void testAddNestedPrimitive() { new Schema( optional(1, "aStruct", Types.StructType.of(optional(2, "primitive", primitiveType)))); Schema applied = new SchemaUpdate(currentSchema, 1).unionByNameWith(newSchema).apply(); - Assert.assertEquals(newSchema.asStruct(), applied.asStruct()); + Assertions.assertThat(applied.asStruct()).isEqualTo(newSchema.asStruct()); } } @@ -137,7 +136,7 @@ public void testAddNestedPrimitives() { new Schema( optional(1, "aStruct", Types.StructType.of(primitiveFields(1, primitiveTypes())))); Schema applied = new SchemaUpdate(currentSchema, 1).unionByNameWith(newSchema).apply(); - Assert.assertEquals(newSchema.asStruct(), applied.asStruct()); + Assertions.assertThat(applied.asStruct()).isEqualTo(newSchema.asStruct()); } @Test @@ -166,7 +165,7 @@ public void testAddNestedLists() { Types.ListType.ofOptional( 10, DecimalType.of(11, 20)))))))))))); Schema applied = new SchemaUpdate(new Schema(), 0).unionByNameWith(newSchema).apply(); - Assert.assertEquals(newSchema.asStruct(), applied.asStruct()); + Assertions.assertThat(applied.asStruct()).isEqualTo(newSchema.asStruct()); } @Test @@ -202,7 +201,7 @@ public void testAddNestedStruct() { "aString", StringType.get())))))))))))))); Schema applied = new SchemaUpdate(new Schema(), 0).unionByNameWith(newSchema).apply(); - Assert.assertEquals(newSchema.asStruct(), applied.asStruct()); + Assertions.assertThat(applied.asStruct()).isEqualTo(newSchema.asStruct()); } @Test @@ -235,7 +234,7 @@ public void testAddNestedMaps() { Types.MapType.ofOptional( 12, 13, StringType.get(), StringType.get())))))))); Schema applied = new SchemaUpdate(new Schema(), 0).unionByNameWith(newSchema).apply(); - Assert.assertEquals(newSchema.asStruct(), applied.asStruct()); + Assertions.assertThat(applied.asStruct()).isEqualTo(newSchema.asStruct()); } @Test @@ -289,8 +288,8 @@ public void testTypePromoteIntegerToLong() { Schema newSchema = new Schema(required(1, "aCol", LongType.get())); Schema applied = new SchemaUpdate(currentSchema, 1).unionByNameWith(newSchema).apply(); - Assert.assertEquals(1, applied.asStruct().fields().size()); - Assert.assertEquals(LongType.get(), applied.asStruct().fields().get(0).type()); + Assertions.assertThat(applied.asStruct().fields()).hasSize(1); + Assertions.assertThat(applied.asStruct().fields().get(0).type()).isEqualTo(LongType.get()); } @Test @@ -300,12 +299,9 @@ public void testTypePromoteFloatToDouble() { Schema newSchema = new Schema(required(1, "aCol", DoubleType.get())); Schema applied = new SchemaUpdate(currentSchema, 1).unionByNameWith(newSchema).apply(); - Assert.assertEquals(1, applied.asStruct().fields().size()); - Assert.assertEquals(DoubleType.get(), applied.asStruct().fields().get(0).type()); - // When attempted Assert.assertEquals(newSchema.asStruct(), applied.asStruct()); - // Got java.lang.AssertionError: - // Expected :struct<1: aCol: required double> - // Actual :struct<1: aCol: required double ()> + Assertions.assertThat(applied.asStruct()).isEqualTo(newSchema.asStruct()); + Assertions.assertThat(applied.asStruct().fields()).hasSize(1); + Assertions.assertThat(applied.asStruct().fields().get(0).type()).isEqualTo(DoubleType.get()); } @Test @@ -327,7 +323,7 @@ public void testTypePromoteDecimalToFixedScaleWithWiderPrecision() { Schema newSchema = new Schema(required(1, "aCol", DecimalType.of(22, 1))); Schema applied = new SchemaUpdate(currentSchema, 1).unionByNameWith(newSchema).apply(); - Assert.assertEquals(newSchema.asStruct(), applied.asStruct()); + Assertions.assertThat(applied.asStruct()).isEqualTo(newSchema.asStruct()); } @Test @@ -388,7 +384,7 @@ public void testAddPrimitiveToNestedStruct() { optional(5, "value", StringType.get()), optional(6, "time", TimeType.get()))))))))); - Assert.assertEquals(expected.asStruct(), applied.asStruct()); + Assertions.assertThat(applied.asStruct()).isEqualTo(expected.asStruct()); } @Test @@ -427,7 +423,7 @@ public void testMirroredSchemas() { Schema union = new SchemaUpdate(aSchema, 0).unionByNameWith(mirrored).apply(); // We don't expect the original schema to have been altered. - Assert.assertEquals(aSchema.asStruct(), union.asStruct()); + Assertions.assertThat(union.asStruct()).isEqualTo(aSchema.asStruct()); } @Test @@ -463,7 +459,7 @@ public void addNewTopLevelStruct() { 7, "d1", Types.StructType.of(optional(8, "d2", Types.StringType.get())))))); Schema union = new SchemaUpdate(schema, 5).unionByNameWith(observed).apply(); - Assert.assertEquals(observed.asStruct(), union.asStruct()); + Assertions.assertThat(union.asStruct()).isEqualTo(observed.asStruct()); } @Test @@ -514,7 +510,7 @@ public void testAppendNestedStruct() { StringType.get())))))))))))))); Schema applied = new SchemaUpdate(schema, 4).unionByNameWith(observed).apply(); - Assert.assertEquals(observed.asStruct(), applied.asStruct()); + Assertions.assertThat(applied.asStruct()).isEqualTo(observed.asStruct()); } @Test @@ -579,6 +575,6 @@ public void testAppendNestedLists() { "list2", ListType.ofOptional(7, StringType.get()))))))))); - Assert.assertEquals(expected.asStruct(), union.asStruct()); + Assertions.assertThat(union.asStruct()).isEqualTo(expected.asStruct()); } } diff --git a/core/src/test/java/org/apache/iceberg/TestSchemaUpdate.java b/core/src/test/java/org/apache/iceberg/TestSchemaUpdate.java index 6c00f8b38c33..04cb64403523 100644 --- a/core/src/test/java/org/apache/iceberg/TestSchemaUpdate.java +++ b/core/src/test/java/org/apache/iceberg/TestSchemaUpdate.java @@ -30,6 +30,7 @@ import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.Pair; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Test; @@ -303,11 +304,10 @@ public void testUpdateFailure() { } String typeChange = fromType.toString() + " -> " + toType.toString(); - AssertHelpers.assertThrows( - "Should reject update: " + typeChange, - IllegalArgumentException.class, - "change column type: col: " + typeChange, - () -> new SchemaUpdate(fromSchema, 1).updateColumn("col", toType)); + Assertions.assertThatThrownBy( + () -> new SchemaUpdate(fromSchema, 1).updateColumn("col", toType)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot change column type: col: " + typeChange); } } } @@ -586,11 +586,10 @@ public void testAddRequiredColumn() { required(1, "id", Types.IntegerType.get()), required(2, "data", Types.StringType.get())); - AssertHelpers.assertThrows( - "Should reject add required column if incompatible changes are not allowed", - IllegalArgumentException.class, - "Incompatible change: cannot add required column: data", - () -> new SchemaUpdate(schema, 1).addRequiredColumn("data", Types.StringType.get())); + Assertions.assertThatThrownBy( + () -> new SchemaUpdate(schema, 1).addRequiredColumn("data", Types.StringType.get())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Incompatible change: cannot add required column: data"); Schema result = new SchemaUpdate(schema, 1) @@ -605,16 +604,15 @@ public void testAddRequiredColumn() { public void testAddRequiredColumnCaseInsensitive() { Schema schema = new Schema(required(1, "id", Types.IntegerType.get())); - AssertHelpers.assertThrows( - "Should reject add required column if same column name different case found", - IllegalArgumentException.class, - "Cannot add column, name already exists: ID", - () -> - new SchemaUpdate(schema, 1) - .caseSensitive(false) - .allowIncompatibleChanges() - .addRequiredColumn("ID", Types.StringType.get()) - .apply()); + Assertions.assertThatThrownBy( + () -> + new SchemaUpdate(schema, 1) + .caseSensitive(false) + .allowIncompatibleChanges() + .addRequiredColumn("ID", Types.StringType.get()) + .apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot add column, name already exists: ID"); } @Test @@ -633,11 +631,9 @@ public void testRequireColumn() { Schema schema = new Schema(optional(1, "id", Types.IntegerType.get())); Schema expected = new Schema(required(1, "id", Types.IntegerType.get())); - AssertHelpers.assertThrows( - "Should reject change to required if incompatible changes are not allowed", - IllegalArgumentException.class, - "Cannot change column nullability: id: optional -> required", - () -> new SchemaUpdate(schema, 1).requireColumn("id")); + Assertions.assertThatThrownBy(() -> new SchemaUpdate(schema, 1).requireColumn("id")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot change column nullability: id: optional -> required"); // required to required is not an incompatible change new SchemaUpdate(expected, 1).requireColumn("id").apply(); @@ -739,34 +735,32 @@ public void testMixedChanges() { @Test public void testAmbiguousAdd() { // preferences.booleans could be top-level or a field of preferences - AssertHelpers.assertThrows( - "Should reject ambiguous column name", - IllegalArgumentException.class, - "ambiguous name: preferences.booleans", - () -> { - UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); - update.addColumn("preferences.booleans", Types.BooleanType.get()); - }); + Assertions.assertThatThrownBy( + () -> { + UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); + update.addColumn("preferences.booleans", Types.BooleanType.get()); + }) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot add column with ambiguous name: preferences.booleans"); } @Test public void testAddAlreadyExists() { - AssertHelpers.assertThrows( - "Should reject column name that already exists", - IllegalArgumentException.class, - "already exists: preferences.feature1", - () -> { - UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); - update.addColumn("preferences", "feature1", Types.BooleanType.get()); - }); - AssertHelpers.assertThrows( - "Should reject column name that already exists", - IllegalArgumentException.class, - "already exists: preferences", - () -> { - UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); - update.addColumn("preferences", Types.BooleanType.get()); - }); + Assertions.assertThatThrownBy( + () -> { + UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); + update.addColumn("preferences", "feature1", Types.BooleanType.get()); + }) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot add column, name already exists: preferences.feature1"); + + Assertions.assertThatThrownBy( + () -> { + UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); + update.addColumn("preferences", Types.BooleanType.get()); + }) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot add column, name already exists: preferences"); } @Test @@ -839,153 +833,141 @@ public void testDeleteThenAddNested() { @Test public void testDeleteMissingColumn() { - AssertHelpers.assertThrows( - "Should reject delete missing column", - IllegalArgumentException.class, - "missing column: col", - () -> { - UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); - update.deleteColumn("col"); - }); + Assertions.assertThatThrownBy( + () -> { + UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); + update.deleteColumn("col"); + }) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot delete missing column: col"); } @Test public void testAddDeleteConflict() { - AssertHelpers.assertThrows( - "Should reject add then delete", - IllegalArgumentException.class, - "missing column: col", - () -> { - UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); - update.addColumn("col", Types.IntegerType.get()).deleteColumn("col"); - }); - AssertHelpers.assertThrows( - "Should reject add then delete", - IllegalArgumentException.class, - "column that has additions: preferences", - () -> { - UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); - update - .addColumn("preferences", "feature3", Types.IntegerType.get()) - .deleteColumn("preferences"); - }); + Assertions.assertThatThrownBy( + () -> { + UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); + update.addColumn("col", Types.IntegerType.get()).deleteColumn("col"); + }) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot delete missing column: col"); + + Assertions.assertThatThrownBy( + () -> { + UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); + update + .addColumn("preferences", "feature3", Types.IntegerType.get()) + .deleteColumn("preferences"); + }) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot delete a column that has additions: preferences"); } @Test public void testRenameMissingColumn() { - AssertHelpers.assertThrows( - "Should reject rename missing column", - IllegalArgumentException.class, - "missing column: col", - () -> { - UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); - update.renameColumn("col", "fail"); - }); + Assertions.assertThatThrownBy( + () -> { + UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); + update.renameColumn("col", "fail"); + }) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot rename missing column: col"); } @Test public void testRenameDeleteConflict() { - AssertHelpers.assertThrows( - "Should reject rename then delete", - IllegalArgumentException.class, - "column that has updates: id", - () -> { - UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); - update.renameColumn("id", "col").deleteColumn("id"); - }); - AssertHelpers.assertThrows( - "Should reject rename then delete", - IllegalArgumentException.class, - "missing column: col", - () -> { - UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); - update.renameColumn("id", "col").deleteColumn("col"); - }); + Assertions.assertThatThrownBy( + () -> { + UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); + update.renameColumn("id", "col").deleteColumn("id"); + }) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot delete a column that has updates: id"); + + Assertions.assertThatThrownBy( + () -> { + UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); + update.renameColumn("id", "col").deleteColumn("col"); + }) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot delete missing column: col"); } @Test public void testDeleteRenameConflict() { - AssertHelpers.assertThrows( - "Should reject delete then rename", - IllegalArgumentException.class, - "column that will be deleted: id", - () -> { - UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); - update.deleteColumn("id").renameColumn("id", "identifier"); - }); + Assertions.assertThatThrownBy( + () -> { + UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); + update.deleteColumn("id").renameColumn("id", "identifier"); + }) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot rename a column that will be deleted: id"); } @Test public void testUpdateMissingColumn() { - AssertHelpers.assertThrows( - "Should reject rename missing column", - IllegalArgumentException.class, - "missing column: col", - () -> { - UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); - update.updateColumn("col", Types.DateType.get()); - }); + Assertions.assertThatThrownBy( + () -> { + UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); + update.updateColumn("col", Types.DateType.get()); + }) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot update missing column: col"); } @Test public void testUpdateDeleteConflict() { - AssertHelpers.assertThrows( - "Should reject update then delete", - IllegalArgumentException.class, - "column that has updates: id", - () -> { - UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); - update.updateColumn("id", Types.LongType.get()).deleteColumn("id"); - }); + Assertions.assertThatThrownBy( + () -> { + UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); + update.updateColumn("id", Types.LongType.get()).deleteColumn("id"); + }) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot delete a column that has updates: id"); } @Test public void testDeleteUpdateConflict() { - AssertHelpers.assertThrows( - "Should reject delete then update", - IllegalArgumentException.class, - "column that will be deleted: id", - () -> { - UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); - update.deleteColumn("id").updateColumn("id", Types.LongType.get()); - }); + Assertions.assertThatThrownBy( + () -> { + UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); + update.deleteColumn("id").updateColumn("id", Types.LongType.get()); + }) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot update a column that will be deleted: id"); } @Test public void testDeleteMapKey() { - AssertHelpers.assertThrows( - "Should reject delete map key", - IllegalArgumentException.class, - "Cannot delete map keys", - () -> { - new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID).deleteColumn("locations.key").apply(); - }); + Assertions.assertThatThrownBy( + () -> + new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID) + .deleteColumn("locations.key") + .apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot delete map keys"); } @Test public void testAddFieldToMapKey() { - AssertHelpers.assertThrows( - "Should reject add sub-field to map key", - IllegalArgumentException.class, - "Cannot add fields to map keys", - () -> { - new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID) - .addColumn("locations.key", "address_line_2", Types.StringType.get()) - .apply(); - }); + Assertions.assertThatThrownBy( + () -> + new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID) + .addColumn("locations.key", "address_line_2", Types.StringType.get()) + .apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot add fields to map keys"); } @Test public void testAlterMapKey() { - AssertHelpers.assertThrows( - "Should reject alter sub-field of map key", - IllegalArgumentException.class, - "Cannot alter map keys", - () -> { - new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID) - .updateColumn("locations.key.zip", Types.LongType.get()) - .apply(); - }); + Assertions.assertThatThrownBy( + () -> + new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID) + .updateColumn("locations.key.zip", Types.LongType.get()) + .apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot alter map keys"); } @Test @@ -996,40 +978,36 @@ public void testUpdateMapKey() { 1, "m", Types.MapType.ofOptional(2, 3, Types.IntegerType.get(), Types.DoubleType.get()))); - AssertHelpers.assertThrows( - "Should reject update map key", - IllegalArgumentException.class, - "Cannot update map keys", - () -> { - new SchemaUpdate(schema, 3).updateColumn("m.key", Types.LongType.get()).apply(); - }); + Assertions.assertThatThrownBy( + () -> new SchemaUpdate(schema, 3).updateColumn("m.key", Types.LongType.get()).apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot update map keys: map"); } @Test public void testUpdateAddedColumnDoc() { Schema schema = new Schema(required(1, "i", Types.IntegerType.get())); - AssertHelpers.assertThrows( - "Should reject add and update doc", - IllegalArgumentException.class, - "Cannot update missing column", - () -> { - new SchemaUpdate(schema, 3) - .addColumn("value", Types.LongType.get()) - .updateColumnDoc("value", "a value") - .apply(); - }); + Assertions.assertThatThrownBy( + () -> + new SchemaUpdate(schema, 3) + .addColumn("value", Types.LongType.get()) + .updateColumnDoc("value", "a value") + .apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot update missing column: value"); } @Test public void testUpdateDeletedColumnDoc() { Schema schema = new Schema(required(1, "i", Types.IntegerType.get())); - AssertHelpers.assertThrows( - "Should reject add and update doc", - IllegalArgumentException.class, - "Cannot update a column that will be deleted", - () -> { - new SchemaUpdate(schema, 3).deleteColumn("i").updateColumnDoc("i", "a value").apply(); - }); + Assertions.assertThatThrownBy( + () -> + new SchemaUpdate(schema, 3) + .deleteColumn("i") + .updateColumnDoc("i", "a value") + .apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot update a column that will be deleted: i"); } @Test @@ -1454,17 +1432,13 @@ public void testMoveSelfReferenceFails() { new Schema( required(1, "id", Types.LongType.get()), required(2, "data", Types.StringType.get())); - AssertHelpers.assertThrows( - "Should fail move for a field that is not in the schema", - IllegalArgumentException.class, - "Cannot move id before itself", - () -> new SchemaUpdate(schema, 2).moveBefore("id", "id").apply()); + Assertions.assertThatThrownBy(() -> new SchemaUpdate(schema, 2).moveBefore("id", "id").apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot move id before itself"); - AssertHelpers.assertThrows( - "Should fail move for a field that is not in the schema", - IllegalArgumentException.class, - "Cannot move id after itself", - () -> new SchemaUpdate(schema, 2).moveAfter("id", "id").apply()); + Assertions.assertThatThrownBy(() -> new SchemaUpdate(schema, 2).moveAfter("id", "id").apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot move id after itself"); } @Test @@ -1473,23 +1447,19 @@ public void testMoveMissingColumnFails() { new Schema( required(1, "id", Types.LongType.get()), required(2, "data", Types.StringType.get())); - AssertHelpers.assertThrows( - "Should fail move for a field that is not in the schema", - IllegalArgumentException.class, - "Cannot move missing column", - () -> new SchemaUpdate(schema, 2).moveFirst("items").apply()); + Assertions.assertThatThrownBy(() -> new SchemaUpdate(schema, 2).moveFirst("items").apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot move missing column: items"); - AssertHelpers.assertThrows( - "Should fail move for a field that is not in the schema", - IllegalArgumentException.class, - "Cannot move missing column", - () -> new SchemaUpdate(schema, 2).moveBefore("items", "id").apply()); + Assertions.assertThatThrownBy( + () -> new SchemaUpdate(schema, 2).moveBefore("items", "id").apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot move missing column: items"); - AssertHelpers.assertThrows( - "Should fail move for a field that is not in the schema", - IllegalArgumentException.class, - "Cannot move missing column", - () -> new SchemaUpdate(schema, 2).moveAfter("items", "data").apply()); + Assertions.assertThatThrownBy( + () -> new SchemaUpdate(schema, 2).moveAfter("items", "data").apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot move missing column: items"); } @Test @@ -1498,35 +1468,32 @@ public void testMoveBeforeAddFails() { new Schema( required(1, "id", Types.LongType.get()), required(2, "data", Types.StringType.get())); - AssertHelpers.assertThrows( - "Should fail move for a field that has not been added yet", - IllegalArgumentException.class, - "Cannot move missing column", - () -> - new SchemaUpdate(schema, 2) - .moveFirst("ts") - .addColumn("ts", Types.TimestampType.withZone()) - .apply()); - - AssertHelpers.assertThrows( - "Should fail move for a field that has not been added yet", - IllegalArgumentException.class, - "Cannot move missing column", - () -> - new SchemaUpdate(schema, 2) - .moveBefore("ts", "id") - .addColumn("ts", Types.TimestampType.withZone()) - .apply()); - - AssertHelpers.assertThrows( - "Should fail move for a field that has not been added yet", - IllegalArgumentException.class, - "Cannot move missing column", - () -> - new SchemaUpdate(schema, 2) - .moveAfter("ts", "data") - .addColumn("ts", Types.TimestampType.withZone()) - .apply()); + Assertions.assertThatThrownBy( + () -> + new SchemaUpdate(schema, 2) + .moveFirst("ts") + .addColumn("ts", Types.TimestampType.withZone()) + .apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot move missing column: ts"); + + Assertions.assertThatThrownBy( + () -> + new SchemaUpdate(schema, 2) + .moveBefore("ts", "id") + .addColumn("ts", Types.TimestampType.withZone()) + .apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot move missing column: ts"); + + Assertions.assertThatThrownBy( + () -> + new SchemaUpdate(schema, 2) + .moveAfter("ts", "data") + .addColumn("ts", Types.TimestampType.withZone()) + .apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot move missing column: ts"); } @Test @@ -1535,17 +1502,15 @@ public void testMoveMissingReferenceColumnFails() { new Schema( required(1, "id", Types.LongType.get()), required(2, "data", Types.StringType.get())); - AssertHelpers.assertThrows( - "Should fail move before a field that is not in the schema", - IllegalArgumentException.class, - "Cannot move id before missing column", - () -> new SchemaUpdate(schema, 2).moveBefore("id", "items").apply()); + Assertions.assertThatThrownBy( + () -> new SchemaUpdate(schema, 2).moveBefore("id", "items").apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot move id before missing column: items"); - AssertHelpers.assertThrows( - "Should fail move after for a field that is not in the schema", - IllegalArgumentException.class, - "Cannot move data after missing column", - () -> new SchemaUpdate(schema, 2).moveAfter("data", "items").apply()); + Assertions.assertThatThrownBy( + () -> new SchemaUpdate(schema, 2).moveAfter("data", "items").apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot move data after missing column: items"); } @Test @@ -1559,11 +1524,10 @@ public void testMovePrimitiveMapKeyFails() { "map", Types.MapType.ofRequired(4, 5, Types.StringType.get(), Types.StringType.get()))); - AssertHelpers.assertThrows( - "Should fail move for map key", - IllegalArgumentException.class, - "Cannot move fields in non-struct type", - () -> new SchemaUpdate(schema, 5).moveBefore("map.key", "map.value").apply()); + Assertions.assertThatThrownBy( + () -> new SchemaUpdate(schema, 5).moveBefore("map.key", "map.value").apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot move fields in non-struct type: map"); } @Test @@ -1577,11 +1541,10 @@ public void testMovePrimitiveMapValueFails() { "map", Types.MapType.ofRequired(4, 5, Types.StringType.get(), Types.StructType.of()))); - AssertHelpers.assertThrows( - "Should fail move for map value", - IllegalArgumentException.class, - "Cannot move fields in non-struct type", - () -> new SchemaUpdate(schema, 5).moveBefore("map.value", "map.key").apply()); + Assertions.assertThatThrownBy( + () -> new SchemaUpdate(schema, 5).moveBefore("map.value", "map.key").apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot move fields in non-struct type: map>"); } @Test @@ -1592,11 +1555,10 @@ public void testMovePrimitiveListElementFails() { required(2, "data", Types.StringType.get()), optional(3, "list", Types.ListType.ofRequired(4, Types.StringType.get()))); - AssertHelpers.assertThrows( - "Should fail move for list element", - IllegalArgumentException.class, - "Cannot move fields in non-struct type", - () -> new SchemaUpdate(schema, 4).moveBefore("list.element", "list").apply()); + Assertions.assertThatThrownBy( + () -> new SchemaUpdate(schema, 4).moveBefore("list.element", "list").apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot move fields in non-struct type: list"); } @Test @@ -1612,11 +1574,10 @@ public void testMoveTopLevelBetweenStructsFails() { required(4, "x", Types.IntegerType.get()), required(5, "y", Types.IntegerType.get())))); - AssertHelpers.assertThrows( - "Should fail move between separate structs", - IllegalArgumentException.class, - "Cannot move field a to a different struct", - () -> new SchemaUpdate(schema, 5).moveBefore("a", "struct.x").apply()); + Assertions.assertThatThrownBy( + () -> new SchemaUpdate(schema, 5).moveBefore("a", "struct.x").apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot move field a to a different struct"); } @Test @@ -1636,11 +1597,10 @@ public void testMoveBetweenStructsFails() { required(5, "x", Types.IntegerType.get()), required(6, "y", Types.IntegerType.get())))); - AssertHelpers.assertThrows( - "Should fail move between separate structs", - IllegalArgumentException.class, - "Cannot move field s2.x to a different struct", - () -> new SchemaUpdate(schema, 6).moveBefore("s2.x", "s1.a").apply()); + Assertions.assertThatThrownBy( + () -> new SchemaUpdate(schema, 6).moveBefore("s2.x", "s1.a").apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot move field s2.x to a different struct"); } @Test @@ -1800,69 +1760,71 @@ public void testSetIdentifierFieldsFails() { required(2, "float", Types.FloatType.get()), required(3, "double", Types.DoubleType.get())); - AssertHelpers.assertThrows( - "Creating schema with nonexistent identifier fieldId should fail", - IllegalArgumentException.class, - "Cannot add fieldId 999 as an identifier field: field does not exist", - () -> new Schema(testSchema.asStruct().fields(), ImmutableSet.of(999))); - - AssertHelpers.assertThrows( - "Creating schema with optional identifier field should fail", - IllegalArgumentException.class, - "Cannot add field id as an identifier field: not a required field", - () -> new Schema(testSchema.asStruct().fields(), ImmutableSet.of(1))); - - AssertHelpers.assertThrows( - "Creating schema with float identifier field should fail", - IllegalArgumentException.class, - "Cannot add field float as an identifier field: must not be float or double field", - () -> new Schema(testSchema.asStruct().fields(), ImmutableSet.of(2))); - - AssertHelpers.assertThrows( - "Creating schema with double identifier field should fail", - IllegalArgumentException.class, - "Cannot add field double as an identifier field: must not be float or double field", - () -> new Schema(testSchema.asStruct().fields(), ImmutableSet.of(3))); - - AssertHelpers.assertThrows( - "add a field with name not exist should fail", - IllegalArgumentException.class, - "not found in current schema or added columns", - () -> - new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID).setIdentifierFields("unknown").apply()); - - AssertHelpers.assertThrows( - "add a field of non-primitive type should fail", - IllegalArgumentException.class, - "not a primitive type field", - () -> - new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID) - .setIdentifierFields("locations") - .apply()); - - AssertHelpers.assertThrows( - "add an optional field should fail", - IllegalArgumentException.class, - "not a required field", - () -> new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID).setIdentifierFields("data").apply()); - - AssertHelpers.assertThrows( - "add a map key nested field should fail", - IllegalArgumentException.class, - "must not be nested in " + SCHEMA.findField("locations"), - () -> - new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID) - .setIdentifierFields("locations.key.zip") - .apply()); - - AssertHelpers.assertThrows( - "add a nested field in list should fail", - IllegalArgumentException.class, - "must not be nested in " + SCHEMA.findField("points"), - () -> - new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID) - .setIdentifierFields("points.element.x") - .apply()); + Assertions.assertThatThrownBy( + () -> new Schema(testSchema.asStruct().fields(), ImmutableSet.of(999))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot add fieldId 999 as an identifier field: field does not exist"); + + Assertions.assertThatThrownBy( + () -> new Schema(testSchema.asStruct().fields(), ImmutableSet.of(1))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot add field id as an identifier field: not a required field"); + + Assertions.assertThatThrownBy( + () -> new Schema(testSchema.asStruct().fields(), ImmutableSet.of(2))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot add field float as an identifier field: must not be float or double field"); + + Assertions.assertThatThrownBy( + () -> new Schema(testSchema.asStruct().fields(), ImmutableSet.of(3))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot add field double as an identifier field: must not be float or double field"); + + Assertions.assertThatThrownBy( + () -> + new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID) + .setIdentifierFields("unknown") + .apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot add field unknown as an identifier field: not found in current schema or added columns"); + + Assertions.assertThatThrownBy( + () -> + new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID) + .setIdentifierFields("locations") + .apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot add field locations as an identifier field: not a primitive type field"); + + Assertions.assertThatThrownBy( + () -> + new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID).setIdentifierFields("data").apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot add field data as an identifier field: not a required field"); + + Assertions.assertThatThrownBy( + () -> + new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID) + .setIdentifierFields("locations.key.zip") + .apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith( + "Cannot add field zip as an identifier field: must not be nested in " + + SCHEMA.findField("locations")); + + Assertions.assertThatThrownBy( + () -> + new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID) + .setIdentifierFields("points.element.x") + .apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith( + "Cannot add field x as an identifier field: must not be nested in " + + SCHEMA.findField("points")); Schema newSchema = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID) @@ -1903,53 +1865,57 @@ public void testSetIdentifierFieldsFails() { int lastColId = SCHEMA_LAST_COLUMN_ID + 15; - AssertHelpers.assertThrows( - "add a nested field in list should fail", - IllegalArgumentException.class, - "must not be nested in " + newSchema.findField("required_list"), - () -> - new SchemaUpdate(newSchema, lastColId) - .setIdentifierFields("required_list.element.x") - .apply()); - - AssertHelpers.assertThrows( - "add a double field should fail", - IllegalArgumentException.class, - "must not be float or double field", - () -> new SchemaUpdate(newSchema, lastColId).setIdentifierFields("col_double").apply()); - - AssertHelpers.assertThrows( - "add a float field should fail", - IllegalArgumentException.class, - "must not be float or double field", - () -> new SchemaUpdate(newSchema, lastColId).setIdentifierFields("col_float").apply()); - - AssertHelpers.assertThrows( - "add a map value nested field should fail", - IllegalArgumentException.class, - "must not be nested in " + newSchema.findField("new_map"), - () -> - new SchemaUpdate(newSchema, lastColId) - .setIdentifierFields("new_map.value.val_col") - .apply()); - - AssertHelpers.assertThrows( - "add a nested field in struct of a list should fail", - IllegalArgumentException.class, - "must not be nested in " + newSchema.findField("new.fields"), - () -> - new SchemaUpdate(newSchema, lastColId) - .setIdentifierFields("new.fields.element.nested") - .apply()); - - AssertHelpers.assertThrows( - "add a nested field in an optional struct should fail", - IllegalArgumentException.class, - "must not be nested in an optional field " + newSchema.findField("preferences"), - () -> - new SchemaUpdate(newSchema, lastColId) - .setIdentifierFields("preferences.feature1") - .apply()); + Assertions.assertThatThrownBy( + () -> + new SchemaUpdate(newSchema, lastColId) + .setIdentifierFields("required_list.element.x") + .apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith( + "Cannot add field x as an identifier field: must not be nested in " + + newSchema.findField("required_list")); + + Assertions.assertThatThrownBy( + () -> new SchemaUpdate(newSchema, lastColId).setIdentifierFields("col_double").apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot add field col_double as an identifier field: must not be float or double field"); + + Assertions.assertThatThrownBy( + () -> new SchemaUpdate(newSchema, lastColId).setIdentifierFields("col_float").apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot add field col_float as an identifier field: must not be float or double field"); + + Assertions.assertThatThrownBy( + () -> + new SchemaUpdate(newSchema, lastColId) + .setIdentifierFields("new_map.value.val_col") + .apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith( + "Cannot add field val_col as an identifier field: must not be nested in " + + newSchema.findField("new_map")); + + Assertions.assertThatThrownBy( + () -> + new SchemaUpdate(newSchema, lastColId) + .setIdentifierFields("new.fields.element.nested") + .apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith( + "Cannot add field nested as an identifier field: must not be nested in " + + newSchema.findField("new.fields")); + + Assertions.assertThatThrownBy( + () -> + new SchemaUpdate(newSchema, lastColId) + .setIdentifierFields("preferences.feature1") + .apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot add field feature1 as an identifier field: must not be nested in an optional field " + + newSchema.findField("preferences")); } @Test @@ -1981,15 +1947,14 @@ public void testDeleteIdentifierFieldColumnsFails() { Schema schemaWithIdentifierFields = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID).setIdentifierFields("id").apply(); - AssertHelpers.assertThrows( - "delete an identifier column without setting identifier fields should fail", - IllegalArgumentException.class, - "Cannot delete identifier field 1: id: required int. To force deletion, " - + "also call setIdentifierFields to update identifier fields.", - () -> - new SchemaUpdate(schemaWithIdentifierFields, SCHEMA_LAST_COLUMN_ID) - .deleteColumn("id") - .apply()); + Assertions.assertThatThrownBy( + () -> + new SchemaUpdate(schemaWithIdentifierFields, SCHEMA_LAST_COLUMN_ID) + .deleteColumn("id") + .apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot delete identifier field 1: id: required int. To force deletion, also call setIdentifierFields to update identifier fields."); } @Test @@ -2005,12 +1970,13 @@ public void testDeleteContainingNestedIdentifierFieldColumnsFails() { .setIdentifierFields("out.nested") .apply(); - AssertHelpers.assertThrows( - "delete a struct with a nested identifier column without setting identifier fields should fail", - IllegalArgumentException.class, - "Cannot delete field 24: out: required struct<25: nested: required string> " - + "as it will delete nested identifier field 25: nested: required string", - () -> new SchemaUpdate(newSchema, SCHEMA_LAST_COLUMN_ID + 2).deleteColumn("out").apply()); + Assertions.assertThatThrownBy( + () -> + new SchemaUpdate(newSchema, SCHEMA_LAST_COLUMN_ID + 2).deleteColumn("out").apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot delete field 24: out: required struct<25: nested: required string> " + + "as it will delete nested identifier field 25: nested: required string"); } @Test diff --git a/core/src/test/java/org/apache/iceberg/TestSequenceNumberForV2Table.java b/core/src/test/java/org/apache/iceberg/TestSequenceNumberForV2Table.java index ec6de6230bb7..08bdc64a0853 100644 --- a/core/src/test/java/org/apache/iceberg/TestSequenceNumberForV2Table.java +++ b/core/src/test/java/org/apache/iceberg/TestSequenceNumberForV2Table.java @@ -23,6 +23,7 @@ import org.apache.iceberg.ManifestEntry.Status; import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.assertj.core.api.Assertions; import org.junit.Test; public class TestSequenceNumberForV2Table extends TableTestBase { @@ -104,11 +105,9 @@ public void testCommitConflict() { table.ops().failCommits(1); - AssertHelpers.assertThrows( - "Should reject commit", - CommitFailedException.class, - "Injected failure", - () -> table.newFastAppend().appendFile(FILE_B).commit()); + Assertions.assertThatThrownBy(() -> table.newFastAppend().appendFile(FILE_B).commit()) + .isInstanceOf(CommitFailedException.class) + .hasMessage("Injected failure"); table.updateProperties().set(TableProperties.COMMIT_NUM_RETRIES, "5").commit(); @@ -352,11 +351,9 @@ public void testTransactionFailure() { Transaction txn = table.newTransaction(); txn.newAppend().appendFile(FILE_C).commit(); - AssertHelpers.assertThrows( - "Transaction commit should fail", - CommitFailedException.class, - "Injected failure", - txn::commitTransaction); + Assertions.assertThatThrownBy(txn::commitTransaction) + .isInstanceOf(CommitFailedException.class) + .hasMessage("Injected failure"); V2Assert.assertEquals( "Last sequence number should be 1", 1, readMetadata().lastSequenceNumber()); diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotLoading.java b/core/src/test/java/org/apache/iceberg/TestSnapshotLoading.java index d80ae4b2e669..25a48ce0622b 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotLoading.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotLoading.java @@ -31,6 +31,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.SerializableSupplier; +import org.assertj.core.api.Assumptions; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -131,6 +132,10 @@ public void testCurrentTableScanDoesNotLoad() { @Test public void testFutureSnapshotsAreRemoved() { + Assumptions.assumeThat(formatVersion) + .as("Future snapshots are only removed for V2 tables") + .isGreaterThan(1); + table.newFastAppend().appendFile(FILE_C).commit(); TableMetadata futureTableMetadata = diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotManager.java b/core/src/test/java/org/apache/iceberg/TestSnapshotManager.java index 5c248bb59f2d..d497dbd360a7 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotManager.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotManager.java @@ -20,6 +20,7 @@ import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; @@ -115,11 +116,10 @@ public void testCherryPickDynamicOverwriteConflict() { long lastSnapshotId = table.currentSnapshot().snapshotId(); // pick the snapshot into the current state - AssertHelpers.assertThrows( - "Should reject partition replacement when a partition has been modified", - ValidationException.class, - "Cannot cherry-pick replace partitions with changed partition", - () -> table.manageSnapshots().cherrypick(staged.snapshotId()).commit()); + Assertions.assertThatThrownBy( + () -> table.manageSnapshots().cherrypick(staged.snapshotId()).commit()) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Cannot cherry-pick replace partitions with changed partition"); Assert.assertEquals( "Failed cherry-pick should not change the table state", @@ -147,11 +147,10 @@ public void testCherryPickDynamicOverwriteDeleteConflict() { long lastSnapshotId = table.currentSnapshot().snapshotId(); // pick the snapshot into the current state - AssertHelpers.assertThrows( - "Should reject partition replacement when a partition has been modified", - ValidationException.class, - "Missing required files to delete", - () -> table.manageSnapshots().cherrypick(staged.snapshotId()).commit()); + Assertions.assertThatThrownBy( + () -> table.manageSnapshots().cherrypick(staged.snapshotId()).commit()) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Missing required files to delete"); Assert.assertEquals( "Failed cherry-pick should not change the table state", @@ -178,11 +177,11 @@ public void testCherryPickFromBranch() { long lastSnapshotId = table.currentSnapshot().snapshotId(); // pick the snapshot into the current state - AssertHelpers.assertThrows( - "Should reject partition replacement when a partition has been modified", - ValidationException.class, - "Cannot cherry-pick overwrite not based on an ancestor of the current state", - () -> table.manageSnapshots().cherrypick(replaceSnapshotId).commit()); + Assertions.assertThatThrownBy( + () -> table.manageSnapshots().cherrypick(replaceSnapshotId).commit()) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith( + "Cannot cherry-pick overwrite not based on an ancestor of the current state"); Assert.assertEquals( "Failed cherry-pick should not change the table state", @@ -207,11 +206,10 @@ public void testCherryPickOverwrite() { long lastSnapshotId = table.currentSnapshot().snapshotId(); // pick the snapshot into the current state - AssertHelpers.assertThrows( - "Should reject partition replacement when a partition has been modified", - ValidationException.class, - "not append, dynamic overwrite, or fast-forward", - () -> table.manageSnapshots().cherrypick(staged.snapshotId()).commit()); + Assertions.assertThatThrownBy( + () -> table.manageSnapshots().cherrypick(staged.snapshotId()).commit()) + .isInstanceOf(ValidationException.class) + .hasMessageEndingWith("not append, dynamic overwrite, or fast-forward"); Assert.assertEquals( "Failed cherry-pick should not change the table state", @@ -232,29 +230,75 @@ public void testCreateBranch() { && expectedBranch.equals(SnapshotRef.branchBuilder(snapshotId).build())); } + @Test + public void testCreateBranchWithoutSnapshotId() { + table.newAppend().appendFile(FILE_A).commit(); + long snapshotId = table.currentSnapshot().snapshotId(); + // Test a basic case of creating a branch + table.manageSnapshots().createBranch("branch1").commit(); + SnapshotRef actualBranch = table.ops().refresh().ref("branch1"); + Assertions.assertThat(actualBranch).isNotNull(); + Assertions.assertThat(actualBranch).isEqualTo(SnapshotRef.branchBuilder(snapshotId).build()); + } + + @Test + public void testCreateBranchOnEmptyTable() { + table.manageSnapshots().createBranch("branch1").commit(); + + SnapshotRef mainSnapshotRef = table.ops().refresh().ref(SnapshotRef.MAIN_BRANCH); + Assertions.assertThat(mainSnapshotRef).isNull(); + + SnapshotRef branch1SnapshotRef = table.ops().refresh().ref("branch1"); + Assertions.assertThat(branch1SnapshotRef).isNotNull(); + Assertions.assertThat(branch1SnapshotRef.minSnapshotsToKeep()).isNull(); + Assertions.assertThat(branch1SnapshotRef.maxSnapshotAgeMs()).isNull(); + Assertions.assertThat(branch1SnapshotRef.maxRefAgeMs()).isNull(); + + Snapshot snapshot = table.snapshot(branch1SnapshotRef.snapshotId()); + Assertions.assertThat(snapshot.parentId()).isNull(); + Assertions.assertThat(snapshot.addedDataFiles(table.io())).isEmpty(); + Assertions.assertThat(snapshot.removedDataFiles(table.io())).isEmpty(); + Assertions.assertThat(snapshot.addedDeleteFiles(table.io())).isEmpty(); + Assertions.assertThat(snapshot.removedDeleteFiles(table.io())).isEmpty(); + } + + @Test + public void testCreateBranchOnEmptyTableFailsWhenRefAlreadyExists() { + table.manageSnapshots().createBranch("branch1").commit(); + + // Trying to create a branch with an existing name should fail + Assertions.assertThatThrownBy(() -> table.manageSnapshots().createBranch("branch1").commit()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Ref branch1 already exists"); + + // Trying to create another branch within the same chain + Assertions.assertThatThrownBy( + () -> table.manageSnapshots().createBranch("branch2").createBranch("branch2").commit()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Ref branch2 already exists"); + } + @Test public void testCreateBranchFailsWhenRefAlreadyExists() { table.newAppend().appendFile(FILE_A).commit(); long snapshotId = table.currentSnapshot().snapshotId(); table.manageSnapshots().createBranch("branch1", snapshotId).commit(); // Trying to create a branch with an existing name should fail - AssertHelpers.assertThrows( - "Creating branch which already exists should fail", - IllegalArgumentException.class, - "Ref branch1 already exists", - () -> table.manageSnapshots().createBranch("branch1", snapshotId).commit()); + Assertions.assertThatThrownBy( + () -> table.manageSnapshots().createBranch("branch1", snapshotId).commit()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Ref branch1 already exists"); // Trying to create another branch within the same chain - AssertHelpers.assertThrows( - "Creating branch which already exists should fail", - IllegalArgumentException.class, - "Ref branch2 already exists", - () -> - table - .manageSnapshots() - .createBranch("branch2", snapshotId) - .createBranch("branch2", snapshotId) - .commit()); + Assertions.assertThatThrownBy( + () -> + table + .manageSnapshots() + .createBranch("branch2", snapshotId) + .createBranch("branch2", snapshotId) + .commit()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Ref branch2 already exists"); } @Test @@ -276,23 +320,21 @@ public void testCreateTagFailsWhenRefAlreadyExists() { table.manageSnapshots().createTag("tag1", snapshotId).commit(); // Trying to create a tag with an existing name should fail - AssertHelpers.assertThrows( - "Creating tag which already exists should fail", - IllegalArgumentException.class, - "Ref tag1 already exists", - () -> table.manageSnapshots().createTag("tag1", snapshotId).commit()); + Assertions.assertThatThrownBy( + () -> table.manageSnapshots().createTag("tag1", snapshotId).commit()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Ref tag1 already exists"); // Trying to create another tag within the same chain - AssertHelpers.assertThrows( - "Creating branch which already exists should fail", - IllegalArgumentException.class, - "Ref tag2 already exists", - () -> - table - .manageSnapshots() - .createTag("tag2", snapshotId) - .createTag("tag2", snapshotId) - .commit()); + Assertions.assertThatThrownBy( + () -> + table + .manageSnapshots() + .createTag("tag2", snapshotId) + .createTag("tag2", snapshotId) + .commit()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Ref tag2 already exists"); } @Test @@ -315,20 +357,18 @@ public void testRemoveBranch() { @Test public void testRemovingNonExistingBranchFails() { - AssertHelpers.assertThrows( - "Trying to remove non-existent branch should fail", - IllegalArgumentException.class, - "Branch does not exist: non-existing", - () -> table.manageSnapshots().removeBranch("non-existing").commit()); + Assertions.assertThatThrownBy( + () -> table.manageSnapshots().removeBranch("non-existing").commit()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Branch does not exist: non-existing"); } @Test public void testRemovingMainBranchFails() { - AssertHelpers.assertThrows( - "Removing main should fail", - IllegalArgumentException.class, - "Cannot remove main branch", - () -> table.manageSnapshots().removeBranch(SnapshotRef.MAIN_BRANCH).commit()); + Assertions.assertThatThrownBy( + () -> table.manageSnapshots().removeBranch(SnapshotRef.MAIN_BRANCH).commit()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot remove main branch"); } @Test @@ -350,11 +390,9 @@ public void testRemoveTag() { @Test public void testRemovingNonExistingTagFails() { - AssertHelpers.assertThrows( - "Removing a non-existing tag should fail", - IllegalArgumentException.class, - "Tag does not exist: non-existing", - () -> table.manageSnapshots().removeTag("non-existing").commit()); + Assertions.assertThatThrownBy(() -> table.manageSnapshots().removeTag("non-existing").commit()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Tag does not exist: non-existing"); } @Test @@ -372,11 +410,10 @@ public void testReplaceBranch() { @Test public void testReplaceBranchNonExistingTargetBranchFails() { - AssertHelpers.assertThrows( - "Replacing a non-existing branch should fail", - IllegalArgumentException.class, - "Target branch does not exist: non-existing", - () -> table.manageSnapshots().replaceBranch("non-existing", "other-branch").commit()); + Assertions.assertThatThrownBy( + () -> table.manageSnapshots().replaceBranch("non-existing", "other-branch").commit()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Target branch does not exist: non-existing"); } @Test @@ -384,11 +421,10 @@ public void testReplaceBranchNonExistingSourceFails() { table.newAppend().appendFile(FILE_A).commit(); long snapshotId = table.currentSnapshot().snapshotId(); table.manageSnapshots().createBranch("branch1", snapshotId).commit(); - AssertHelpers.assertThrows( - "Replacing where the source ref does not exist should fail", - IllegalArgumentException.class, - "Ref does not exist: non-existing", - () -> table.manageSnapshots().replaceBranch("branch1", "non-existing").commit()); + Assertions.assertThatThrownBy( + () -> table.manageSnapshots().replaceBranch("branch1", "non-existing").commit()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Ref does not exist: non-existing"); } @Test @@ -422,12 +458,15 @@ public void testFastForwardWhenTargetIsNotAncestorFails() { final String newBranch = "new-branch-at-staged-snapshot"; table.manageSnapshots().createBranch(newBranch, snapshot).commit(); - AssertHelpers.assertThrows( - "Fast-forward should fail if target is not an ancestor of the source", - IllegalArgumentException.class, - "Cannot fast-forward: main is not an ancestor of new-branch-at-staged-snapshot", - () -> - table.manageSnapshots().fastForwardBranch(SnapshotRef.MAIN_BRANCH, newBranch).commit()); + Assertions.assertThatThrownBy( + () -> + table + .manageSnapshots() + .fastForwardBranch(SnapshotRef.MAIN_BRANCH, newBranch) + .commit()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot fast-forward: main is not an ancestor of new-branch-at-staged-snapshot"); } @Test @@ -473,26 +512,25 @@ public void testSettingBranchRetentionOnTagFails() { table.newAppend().appendFile(FILE_A).commit(); long snapshotId = table.currentSnapshot().snapshotId(); - AssertHelpers.assertThrows( - "Setting minSnapshotsToKeep should fail for tags", - IllegalArgumentException.class, - "Tags do not support setting minSnapshotsToKeep", - () -> - table - .manageSnapshots() - .createTag("tag1", snapshotId) - .setMinSnapshotsToKeep("tag1", 10) - .commit()); - AssertHelpers.assertThrows( - "Setting maxSnapshotAgeMs should fail for tags", - IllegalArgumentException.class, - "Tags do not support setting maxSnapshotAgeMs", - () -> - table - .manageSnapshots() - .createTag("tag1", snapshotId) - .setMaxSnapshotAgeMs("tag1", 10) - .commit()); + Assertions.assertThatThrownBy( + () -> + table + .manageSnapshots() + .createTag("tag1", snapshotId) + .setMinSnapshotsToKeep("tag1", 10) + .commit()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Tags do not support setting minSnapshotsToKeep"); + + Assertions.assertThatThrownBy( + () -> + table + .manageSnapshots() + .createTag("tag1", snapshotId) + .setMaxSnapshotAgeMs("tag1", 10) + .commit()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Tags do not support setting maxSnapshotAgeMs"); } @Test @@ -558,21 +596,23 @@ public void testRenameBranch() { @Test public void testFailRenamingMainBranch() { - AssertHelpers.assertThrows( - "Renaming main branch should fail", - IllegalArgumentException.class, - "Cannot rename main branch", - () -> - table.manageSnapshots().renameBranch(SnapshotRef.MAIN_BRANCH, "some-branch").commit()); + Assertions.assertThatThrownBy( + () -> + table + .manageSnapshots() + .renameBranch(SnapshotRef.MAIN_BRANCH, "some-branch") + .commit()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot rename main branch"); } @Test public void testRenamingNonExistingBranchFails() { - AssertHelpers.assertThrows( - "Renaming non-existent branch should fail", - IllegalArgumentException.class, - "Branch does not exist: some-missing-branch", - () -> table.manageSnapshots().renameBranch("some-missing-branch", "some-branch").commit()); + Assertions.assertThatThrownBy( + () -> + table.manageSnapshots().renameBranch("some-missing-branch", "some-branch").commit()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Branch does not exist: some-missing-branch"); } @Test diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotRefParser.java b/core/src/test/java/org/apache/iceberg/TestSnapshotRefParser.java index d193671d6151..d8940367e1a5 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotRefParser.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotRefParser.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Test; @@ -106,77 +107,59 @@ public void testBranchFromJsonAllFields() { @Test public void testFailParsingWhenNullOrEmptyJson() { String nullJson = null; - AssertHelpers.assertThrows( - "SnapshotRefParser should fail to deserialize null JSON string", - IllegalArgumentException.class, - "Cannot parse snapshot ref from invalid JSON", - () -> SnapshotRefParser.fromJson(nullJson)); + Assertions.assertThatThrownBy(() -> SnapshotRefParser.fromJson(nullJson)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot parse snapshot ref from invalid JSON"); String emptyJson = ""; - AssertHelpers.assertThrows( - "SnapshotRefParser should fail to deserialize empty JSON string", - IllegalArgumentException.class, - "Cannot parse snapshot ref from invalid JSON", - () -> SnapshotRefParser.fromJson(emptyJson)); + Assertions.assertThatThrownBy(() -> SnapshotRefParser.fromJson(emptyJson)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot parse snapshot ref from invalid JSON"); } @Test public void testFailParsingWhenMissingRequiredFields() { String refMissingType = "{\"snapshot-id\":1}"; - AssertHelpers.assertThrows( - "SnapshotRefParser should fail to deserialize ref with missing type", - IllegalArgumentException.class, - "Cannot parse missing string", - () -> SnapshotRefParser.fromJson(refMissingType)); + Assertions.assertThatThrownBy(() -> SnapshotRefParser.fromJson(refMissingType)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot parse missing string"); String refMissingSnapshotId = "{\"type\":\"branch\"}"; - AssertHelpers.assertThrows( - "SnapshotRefParser should fail to deserialize ref with missing snapshot id", - IllegalArgumentException.class, - "Cannot parse missing long", - () -> SnapshotRefParser.fromJson(refMissingSnapshotId)); + Assertions.assertThatThrownBy(() -> SnapshotRefParser.fromJson(refMissingSnapshotId)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot parse missing long"); } @Test public void testFailWhenFieldsHaveInvalidValues() { String invalidSnapshotId = "{\"snapshot-id\":\"invalid-snapshot-id\",\"type\":\"not-a-valid-tag-type\"}"; - AssertHelpers.assertThrows( - "SnapshotRefParser should fail to deserialize ref with invalid snapshot id", - IllegalArgumentException.class, - "Cannot parse to a long value: snapshot-id: \"invalid-snapshot-id\"", - () -> SnapshotRefParser.fromJson(invalidSnapshotId)); + Assertions.assertThatThrownBy(() -> SnapshotRefParser.fromJson(invalidSnapshotId)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse to a long value: snapshot-id: \"invalid-snapshot-id\""); String invalidTagType = "{\"snapshot-id\":1,\"type\":\"not-a-valid-tag-type\"}"; - AssertHelpers.assertThrows( - "SnapshotRefParser should fail to deserialize ref with invalid tag", - IllegalArgumentException.class, - "Invalid snapshot ref type: not-a-valid-tag-type", - () -> SnapshotRefParser.fromJson(invalidTagType)); + Assertions.assertThatThrownBy(() -> SnapshotRefParser.fromJson(invalidTagType)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid snapshot ref type: not-a-valid-tag-type"); String invalidRefAge = "{\"snapshot-id\":1,\"type\":\"tag\",\"max-ref-age-ms\":\"not-a-valid-value\"}"; - AssertHelpers.assertThrows( - "SnapshotRefParser should fail to deserialize ref with invalid ref age", - IllegalArgumentException.class, - "Cannot parse to a long value: max-ref-age-ms: \"not-a-valid-value\"", - () -> SnapshotRefParser.fromJson(invalidRefAge)); + Assertions.assertThatThrownBy(() -> SnapshotRefParser.fromJson(invalidRefAge)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse to a long value: max-ref-age-ms: \"not-a-valid-value\""); String invalidSnapshotsToKeep = "{\"snapshot-id\":1,\"type\":\"branch\", " + "\"min-snapshots-to-keep\":\"invalid-number\"}"; - AssertHelpers.assertThrows( - "SnapshotRefParser should fail to deserialize ref with missing snapshot id", - IllegalArgumentException.class, - "Cannot parse to an integer value: min-snapshots-to-keep: \"invalid-number\"", - () -> SnapshotRefParser.fromJson(invalidSnapshotsToKeep)); + Assertions.assertThatThrownBy(() -> SnapshotRefParser.fromJson(invalidSnapshotsToKeep)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse to an integer value: min-snapshots-to-keep: \"invalid-number\""); String invalidMaxSnapshotAge = "{\"snapshot-id\":1,\"type\":\"branch\", " + "\"max-snapshot-age-ms\":\"invalid-age\"}"; - AssertHelpers.assertThrows( - "SnapshotRefParser should fail to deserialize ref with missing snapshot id", - IllegalArgumentException.class, - "Cannot parse to a long value: max-snapshot-age-ms: \"invalid-age\"", - () -> SnapshotRefParser.fromJson(invalidMaxSnapshotAge)); + Assertions.assertThatThrownBy(() -> SnapshotRefParser.fromJson(invalidMaxSnapshotAge)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse to a long value: max-snapshot-age-ms: \"invalid-age\""); } } diff --git a/core/src/test/java/org/apache/iceberg/TestSortOrder.java b/core/src/test/java/org/apache/iceberg/TestSortOrder.java index 597727c4f95c..8fbc4e11fbc3 100644 --- a/core/src/test/java/org/apache/iceberg/TestSortOrder.java +++ b/core/src/test/java/org/apache/iceberg/TestSortOrder.java @@ -100,16 +100,14 @@ public void testSortOrderBuilder() { SortOrder.unsorted(), SortOrder.builderFor(SCHEMA).withOrderId(0).build()); - AssertHelpers.assertThrows( - "Should not allow sort orders ID 0", - IllegalArgumentException.class, - "order ID 0 is reserved for unsorted order", - () -> SortOrder.builderFor(SCHEMA).asc("data").withOrderId(0).build()); - AssertHelpers.assertThrows( - "Should not allow unsorted orders with arbitrary IDs", - IllegalArgumentException.class, - "order ID must be 0", - () -> SortOrder.builderFor(SCHEMA).withOrderId(1).build()); + Assertions.assertThatThrownBy( + () -> SortOrder.builderFor(SCHEMA).asc("data").withOrderId(0).build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Sort order ID 0 is reserved for unsorted order"); + + Assertions.assertThatThrownBy(() -> SortOrder.builderFor(SCHEMA).withOrderId(1).build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Unsorted order ID must be 0"); } @Test @@ -335,11 +333,9 @@ public void testIncompatibleSchemaEvolutionWithSortOrder() { TestTables.TestTable table = TestTables.create(tableDir, "test", SCHEMA, spec, order, formatVersion); - AssertHelpers.assertThrows( - "Should reject deletion of sort columns", - ValidationException.class, - "Cannot find source column", - () -> table.updateSchema().deleteColumn("s.id").commit()); + Assertions.assertThatThrownBy(() -> table.updateSchema().deleteColumn("s.id").commit()) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Cannot find source column for sort field"); } @Test diff --git a/core/src/test/java/org/apache/iceberg/TestSplitPlanning.java b/core/src/test/java/org/apache/iceberg/TestSplitPlanning.java index d5d6cd4e41f4..ab83b277509c 100644 --- a/core/src/test/java/org/apache/iceberg/TestSplitPlanning.java +++ b/core/src/test/java/org/apache/iceberg/TestSplitPlanning.java @@ -33,6 +33,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.types.Types; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -177,32 +178,29 @@ public void testSplitPlanningWithOverridenOpenCostSize() { @Test public void testSplitPlanningWithNegativeValues() { - AssertHelpers.assertThrows( - "User provided split size should be validated", - IllegalArgumentException.class, - "Split size must be > 0: -10", - () -> { - table.newScan().option(TableProperties.SPLIT_SIZE, String.valueOf(-10)).planTasks(); - }); - - AssertHelpers.assertThrows( - "User provided split planning lookback should be validated", - IllegalArgumentException.class, - "Split planning lookback must be > 0: -10", - () -> { - table.newScan().option(TableProperties.SPLIT_LOOKBACK, String.valueOf(-10)).planTasks(); - }); - - AssertHelpers.assertThrows( - "User provided split open file cost should be validated", - IllegalArgumentException.class, - "File open cost must be >= 0: -10", - () -> { - table - .newScan() - .option(TableProperties.SPLIT_OPEN_FILE_COST, String.valueOf(-10)) - .planTasks(); - }); + Assertions.assertThatThrownBy( + () -> + table.newScan().option(TableProperties.SPLIT_SIZE, String.valueOf(-10)).planTasks()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Split size must be > 0: -10"); + + Assertions.assertThatThrownBy( + () -> + table + .newScan() + .option(TableProperties.SPLIT_LOOKBACK, String.valueOf(-10)) + .planTasks()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Split planning lookback must be > 0: -10"); + + Assertions.assertThatThrownBy( + () -> + table + .newScan() + .option(TableProperties.SPLIT_OPEN_FILE_COST, String.valueOf(-10)) + .planTasks()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("File open cost must be >= 0: -10"); } @Test diff --git a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java index aad46bd56cd8..3e4ff1504263 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java @@ -406,35 +406,34 @@ public void testInvalidMainBranch() throws IOException { Map refs = ImmutableMap.of("main", SnapshotRef.branchBuilder(previousSnapshotId).build()); - AssertHelpers.assertThrows( - "Should fail if main branch snapshot ID does not match currentSnapshotId", - IllegalArgumentException.class, - "Current snapshot ID does not match main branch", - () -> - new TableMetadata( - null, - 2, - UUID.randomUUID().toString(), - TEST_LOCATION, - SEQ_NO, - System.currentTimeMillis(), - 3, - 7, - ImmutableList.of(TEST_SCHEMA, schema), - 5, - ImmutableList.of(SPEC_5), - SPEC_5.lastAssignedFieldId(), - 3, - ImmutableList.of(SORT_ORDER_3), - ImmutableMap.of("property", "value"), - currentSnapshotId, - Arrays.asList(previousSnapshot, currentSnapshot), - null, - snapshotLog, - ImmutableList.of(), - refs, - ImmutableList.of(), - ImmutableList.of())); + Assertions.assertThatThrownBy( + () -> + new TableMetadata( + null, + 2, + UUID.randomUUID().toString(), + TEST_LOCATION, + SEQ_NO, + System.currentTimeMillis(), + 3, + 7, + ImmutableList.of(TEST_SCHEMA, schema), + 5, + ImmutableList.of(SPEC_5), + SPEC_5.lastAssignedFieldId(), + 3, + ImmutableList.of(SORT_ORDER_3), + ImmutableMap.of("property", "value"), + currentSnapshotId, + Arrays.asList(previousSnapshot, currentSnapshot), + null, + snapshotLog, + ImmutableList.of(), + refs, + ImmutableList.of(), + ImmutableList.of())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Current snapshot ID does not match main branch"); } @Test @@ -451,35 +450,34 @@ public void testMainWithoutCurrent() throws IOException { Map refs = ImmutableMap.of("main", SnapshotRef.branchBuilder(snapshotId).build()); - AssertHelpers.assertThrows( - "Should fail if main branch snapshot ID does not match currentSnapshotId", - IllegalArgumentException.class, - "Current snapshot is not set, but main branch exists", - () -> - new TableMetadata( - null, - 2, - UUID.randomUUID().toString(), - TEST_LOCATION, - SEQ_NO, - System.currentTimeMillis(), - 3, - 7, - ImmutableList.of(TEST_SCHEMA, schema), - 5, - ImmutableList.of(SPEC_5), - SPEC_5.lastAssignedFieldId(), - 3, - ImmutableList.of(SORT_ORDER_3), - ImmutableMap.of("property", "value"), - -1, - ImmutableList.of(snapshot), - null, - ImmutableList.of(), - ImmutableList.of(), - refs, - ImmutableList.of(), - ImmutableList.of())); + Assertions.assertThatThrownBy( + () -> + new TableMetadata( + null, + 2, + UUID.randomUUID().toString(), + TEST_LOCATION, + SEQ_NO, + System.currentTimeMillis(), + 3, + 7, + ImmutableList.of(TEST_SCHEMA, schema), + 5, + ImmutableList.of(SPEC_5), + SPEC_5.lastAssignedFieldId(), + 3, + ImmutableList.of(SORT_ORDER_3), + ImmutableMap.of("property", "value"), + -1, + ImmutableList.of(snapshot), + null, + ImmutableList.of(), + ImmutableList.of(), + refs, + ImmutableList.of(), + ImmutableList.of())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Current snapshot is not set, but main branch exists"); } @Test @@ -491,35 +489,34 @@ public void testBranchSnapshotMissing() { Map refs = ImmutableMap.of("main", SnapshotRef.branchBuilder(snapshotId).build()); - AssertHelpers.assertThrows( - "Should fail if main branch snapshot ID does not match currentSnapshotId", - IllegalArgumentException.class, - "does not exist in the existing snapshots list", - () -> - new TableMetadata( - null, - 2, - UUID.randomUUID().toString(), - TEST_LOCATION, - SEQ_NO, - System.currentTimeMillis(), - 3, - 7, - ImmutableList.of(TEST_SCHEMA, schema), - 5, - ImmutableList.of(SPEC_5), - SPEC_5.lastAssignedFieldId(), - 3, - ImmutableList.of(SORT_ORDER_3), - ImmutableMap.of("property", "value"), - -1, - ImmutableList.of(), - null, - ImmutableList.of(), - ImmutableList.of(), - refs, - ImmutableList.of(), - ImmutableList.of())); + Assertions.assertThatThrownBy( + () -> + new TableMetadata( + null, + 2, + UUID.randomUUID().toString(), + TEST_LOCATION, + SEQ_NO, + System.currentTimeMillis(), + 3, + 7, + ImmutableList.of(TEST_SCHEMA, schema), + 5, + ImmutableList.of(SPEC_5), + SPEC_5.lastAssignedFieldId(), + 3, + ImmutableList.of(SORT_ORDER_3), + ImmutableMap.of("property", "value"), + -1, + ImmutableList.of(), + null, + ImmutableList.of(), + ImmutableList.of(), + refs, + ImmutableList.of(), + ImmutableList.of())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageEndingWith("does not exist in the existing snapshots list"); } private static String toJsonWithoutSpecAndSchemaList(TableMetadata metadata) { @@ -922,69 +919,67 @@ public void testAddPreviousMetadataRemoveMultiple() throws IOException { @Test public void testV2UUIDValidation() { - AssertHelpers.assertThrows( - "Should reject v2 metadata without a UUID", - IllegalArgumentException.class, - "UUID is required in format v2", - () -> - new TableMetadata( - null, - 2, - null, - TEST_LOCATION, - SEQ_NO, - System.currentTimeMillis(), - LAST_ASSIGNED_COLUMN_ID, - 7, - ImmutableList.of(TEST_SCHEMA), - SPEC_5.specId(), - ImmutableList.of(SPEC_5), - SPEC_5.lastAssignedFieldId(), - 3, - ImmutableList.of(SORT_ORDER_3), - ImmutableMap.of(), - -1L, - ImmutableList.of(), - null, - ImmutableList.of(), - ImmutableList.of(), - ImmutableMap.of(), - ImmutableList.of(), - ImmutableList.of())); + Assertions.assertThatThrownBy( + () -> + new TableMetadata( + null, + 2, + null, + TEST_LOCATION, + SEQ_NO, + System.currentTimeMillis(), + LAST_ASSIGNED_COLUMN_ID, + 7, + ImmutableList.of(TEST_SCHEMA), + SPEC_5.specId(), + ImmutableList.of(SPEC_5), + SPEC_5.lastAssignedFieldId(), + 3, + ImmutableList.of(SORT_ORDER_3), + ImmutableMap.of(), + -1L, + ImmutableList.of(), + null, + ImmutableList.of(), + ImmutableList.of(), + ImmutableMap.of(), + ImmutableList.of(), + ImmutableList.of())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("UUID is required in format v2"); } @Test public void testVersionValidation() { int unsupportedVersion = TableMetadata.SUPPORTED_TABLE_FORMAT_VERSION + 1; - AssertHelpers.assertThrows( - "Should reject unsupported metadata", - IllegalArgumentException.class, - "Unsupported format version: v" + unsupportedVersion, - () -> - new TableMetadata( - null, - unsupportedVersion, - null, - TEST_LOCATION, - SEQ_NO, - System.currentTimeMillis(), - LAST_ASSIGNED_COLUMN_ID, - 7, - ImmutableList.of(TEST_SCHEMA), - SPEC_5.specId(), - ImmutableList.of(SPEC_5), - SPEC_5.lastAssignedFieldId(), - 3, - ImmutableList.of(SORT_ORDER_3), - ImmutableMap.of(), - -1L, - ImmutableList.of(), - null, - ImmutableList.of(), - ImmutableList.of(), - ImmutableMap.of(), - ImmutableList.of(), - ImmutableList.of())); + Assertions.assertThatThrownBy( + () -> + new TableMetadata( + null, + unsupportedVersion, + null, + TEST_LOCATION, + SEQ_NO, + System.currentTimeMillis(), + LAST_ASSIGNED_COLUMN_ID, + 7, + ImmutableList.of(TEST_SCHEMA), + SPEC_5.specId(), + ImmutableList.of(SPEC_5), + SPEC_5.lastAssignedFieldId(), + 3, + ImmutableList.of(SORT_ORDER_3), + ImmutableMap.of(), + -1L, + ImmutableList.of(), + null, + ImmutableList.of(), + ImmutableList.of(), + ImmutableMap.of(), + ImmutableList.of(), + ImmutableList.of())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Unsupported format version: v" + unsupportedVersion); } @Test @@ -998,63 +993,51 @@ public void testParserVersionValidation() throws Exception { Assert.assertNotNull("Should successfully read supported metadata version", parsed2); String unsupportedVersion = readTableMetadataInputFile("TableMetadataUnsupportedVersion.json"); - AssertHelpers.assertThrows( - "Should not read unsupported metadata", - IllegalArgumentException.class, - "Cannot read unsupported version", - () -> TableMetadataParser.fromJson(unsupportedVersion)); + Assertions.assertThatThrownBy(() -> TableMetadataParser.fromJson(unsupportedVersion)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot read unsupported version"); } @Test public void testParserV2PartitionSpecsValidation() throws Exception { String unsupportedVersion = readTableMetadataInputFile("TableMetadataV2MissingPartitionSpecs.json"); - AssertHelpers.assertThrows( - "Should reject v2 metadata without partition specs", - IllegalArgumentException.class, - "partition-specs must exist in format v2", - () -> TableMetadataParser.fromJson(unsupportedVersion)); + Assertions.assertThatThrownBy(() -> TableMetadataParser.fromJson(unsupportedVersion)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("partition-specs must exist in format v2"); } @Test public void testParserV2LastAssignedFieldIdValidation() throws Exception { String unsupportedVersion = readTableMetadataInputFile("TableMetadataV2MissingLastPartitionId.json"); - AssertHelpers.assertThrows( - "Should reject v2 metadata without last assigned partition field id", - IllegalArgumentException.class, - "last-partition-id must exist in format v2", - () -> TableMetadataParser.fromJson(unsupportedVersion)); + Assertions.assertThatThrownBy(() -> TableMetadataParser.fromJson(unsupportedVersion)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("last-partition-id must exist in format v2"); } @Test public void testParserV2SortOrderValidation() throws Exception { String unsupportedVersion = readTableMetadataInputFile("TableMetadataV2MissingSortOrder.json"); - AssertHelpers.assertThrows( - "Should reject v2 metadata without sort order", - IllegalArgumentException.class, - "sort-orders must exist in format v2", - () -> TableMetadataParser.fromJson(unsupportedVersion)); + Assertions.assertThatThrownBy(() -> TableMetadataParser.fromJson(unsupportedVersion)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("sort-orders must exist in format v2"); } @Test public void testParserV2CurrentSchemaIdValidation() throws Exception { String unsupported = readTableMetadataInputFile("TableMetadataV2CurrentSchemaNotFound.json"); - AssertHelpers.assertThrows( - "Should reject v2 metadata without valid schema id", - IllegalArgumentException.class, - "Cannot find schema with current-schema-id=2 from schemas", - () -> TableMetadataParser.fromJson(unsupported)); + Assertions.assertThatThrownBy(() -> TableMetadataParser.fromJson(unsupported)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot find schema with current-schema-id=2 from schemas"); } @Test public void testParserV2SchemasValidation() throws Exception { String unsupported = readTableMetadataInputFile("TableMetadataV2MissingSchemas.json"); - AssertHelpers.assertThrows( - "Should reject v2 metadata without schemas", - IllegalArgumentException.class, - "schemas must exist in format v2", - () -> TableMetadataParser.fromJson(unsupported)); + Assertions.assertThatThrownBy(() -> TableMetadataParser.fromJson(unsupported)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("schemas must exist in format v2"); } private String readTableMetadataInputFile(String fileName) throws Exception { @@ -1105,11 +1088,9 @@ public void testInvalidUpdatePartitionSpecForV1Table() throws Exception { TableMetadata.newTableMetadata( schema, PartitionSpec.unpartitioned(), location, ImmutableMap.of()); - AssertHelpers.assertThrows( - "Should fail to update an invalid partition spec", - ValidationException.class, - "Spec does not use sequential IDs that are required in v1", - () -> metadata.updatePartitionSpec(spec)); + Assertions.assertThatThrownBy(() -> metadata.updatePartitionSpec(spec)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Spec does not use sequential IDs that are required in v1"); } @Test @@ -1343,6 +1324,16 @@ public void testParseSchemaIdentifierFields() throws Exception { Assert.assertEquals(Sets.newHashSet(1, 2), parsed.schemasById().get(1).identifierFieldIds()); } + @Test + public void testParseMinimal() throws Exception { + String data = readTableMetadataInputFile("TableMetadataV2ValidMinimal.json"); + TableMetadata parsed = TableMetadataParser.fromJson(data); + Assertions.assertThat(parsed.snapshots()).isEmpty(); + Assertions.assertThat(parsed.snapshotLog()).isEmpty(); + Assertions.assertThat(parsed.properties()).isEmpty(); + Assertions.assertThat(parsed.previousFiles()).isEmpty(); + } + @Test public void testUpdateSchemaIdentifierFields() { Schema schema = new Schema(Types.NestedField.required(10, "x", Types.StringType.get())); @@ -1547,31 +1538,30 @@ public void testParseStatisticsFiles() throws Exception { public void testNoReservedPropertyForTableMetadataCreation() { Schema schema = new Schema(Types.NestedField.required(10, "x", Types.StringType.get())); - AssertHelpers.assertThrows( - "should not allow reserved table property when creating table metadata", - IllegalArgumentException.class, - "Table properties should not contain reserved properties, but got {format-version=1}", - () -> - TableMetadata.newTableMetadata( - schema, - PartitionSpec.unpartitioned(), - null, - "/tmp", - ImmutableMap.of(TableProperties.FORMAT_VERSION, "1"), - 1)); - - AssertHelpers.assertThrows( - "should not allow reserved table property when creating table metadata", - IllegalArgumentException.class, - "Table properties should not contain reserved properties, but got {uuid=uuid}", - () -> - TableMetadata.newTableMetadata( - schema, - PartitionSpec.unpartitioned(), - null, - "/tmp", - ImmutableMap.of(TableProperties.UUID, "uuid"), - 1)); + Assertions.assertThatThrownBy( + () -> + TableMetadata.newTableMetadata( + schema, + PartitionSpec.unpartitioned(), + null, + "/tmp", + ImmutableMap.of(TableProperties.FORMAT_VERSION, "1"), + 1)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Table properties should not contain reserved properties, but got {format-version=1}"); + + Assertions.assertThatThrownBy( + () -> + TableMetadata.newTableMetadata( + schema, + PartitionSpec.unpartitioned(), + null, + "/tmp", + ImmutableMap.of(TableProperties.UUID, "uuid"), + 1)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Table properties should not contain reserved properties, but got {uuid=uuid}"); } @Test diff --git a/core/src/test/java/org/apache/iceberg/TestTransaction.java b/core/src/test/java/org/apache/iceberg/TestTransaction.java index f1c133633d2d..b76974e2440d 100644 --- a/core/src/test/java/org/apache/iceberg/TestTransaction.java +++ b/core/src/test/java/org/apache/iceberg/TestTransaction.java @@ -216,11 +216,9 @@ public void testDetectsUncommittedChange() { "Base metadata should not change when commit is created", base, readMetadata()); Assert.assertEquals("Table should be on version 0 after txn create", 0, (int) version()); - AssertHelpers.assertThrows( - "Should reject commit when last operation has not committed", - IllegalStateException.class, - "Cannot create new DeleteFiles: last operation has not committed", - txn::newDelete); + Assertions.assertThatThrownBy(txn::newDelete) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Cannot create new DeleteFiles: last operation has not committed"); } @Test @@ -241,11 +239,9 @@ public void testDetectsUncommittedChangeOnCommit() { "Base metadata should not change when commit is created", base, readMetadata()); Assert.assertEquals("Table should be on version 0 after txn create", 0, (int) version()); - AssertHelpers.assertThrows( - "Should reject commit when last operation has not committed", - IllegalStateException.class, - "Cannot commit transaction: last operation has not committed", - txn::commitTransaction); + Assertions.assertThatThrownBy(txn::commitTransaction) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Cannot commit transaction: last operation has not committed"); } @Test @@ -272,11 +268,9 @@ public void testTransactionConflict() { // cause the transaction commit to fail table.ops().failCommits(1); - AssertHelpers.assertThrows( - "Transaction commit should fail", - CommitFailedException.class, - "Injected failure", - txn::commitTransaction); + Assertions.assertThatThrownBy(txn::commitTransaction) + .isInstanceOf(CommitFailedException.class) + .hasMessage("Injected failure"); } @Test @@ -445,11 +439,9 @@ public void testTransactionRetrySchemaUpdate() { schemaId); // commit the transaction for adding "new-column" - AssertHelpers.assertThrows( - "Should fail due to conflicting transaction even after retry", - CommitFailedException.class, - "Table metadata refresh is required", - txn::commitTransaction); + Assertions.assertThatThrownBy(txn::commitTransaction) + .isInstanceOf(CommitFailedException.class) + .hasMessage("Table metadata refresh is required"); } @Test @@ -664,17 +656,16 @@ public void testTransactionRetryAndAppendManifestsWithSnapshotIdInheritance() th @Test public void testTransactionNoCustomDeleteFunc() { - AssertHelpers.assertThrows( - "Should fail setting a custom delete function with a transaction", - IllegalArgumentException.class, - "Cannot set delete callback more than once", - () -> - table - .newTransaction() - .newAppend() - .appendFile(FILE_A) - .appendFile(FILE_B) - .deleteWith(file -> {})); + Assertions.assertThatThrownBy( + () -> + table + .newTransaction() + .newAppend() + .appendFile(FILE_A) + .appendFile(FILE_B) + .deleteWith(file -> {})) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot set delete callback more than once"); } @Test @@ -758,11 +749,9 @@ public void testSimpleTransactionNotDeletingMetadataOnUnknownSate() throws IOExc Transaction transaction = table.newTransaction(); transaction.newAppend().appendFile(FILE_A).commit(); - AssertHelpers.assertThrows( - "Transaction commit should fail with CommitStateUnknownException", - CommitStateUnknownException.class, - "datacenter on fire", - () -> transaction.commitTransaction()); + Assertions.assertThatThrownBy(transaction::commitTransaction) + .isInstanceOf(CommitStateUnknownException.class) + .hasMessageStartingWith("datacenter on fire"); // Make sure metadata files still exist Snapshot current = table.currentSnapshot(); diff --git a/core/src/test/java/org/apache/iceberg/TestUpdatePartitionSpec.java b/core/src/test/java/org/apache/iceberg/TestUpdatePartitionSpec.java index 97210e51b5ba..926272e60f98 100644 --- a/core/src/test/java/org/apache/iceberg/TestUpdatePartitionSpec.java +++ b/core/src/test/java/org/apache/iceberg/TestUpdatePartitionSpec.java @@ -29,6 +29,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.transforms.Transforms; import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; @@ -419,71 +420,65 @@ public void testAddDeletedName() { @Test public void testRemoveNewlyAddedFieldByName() { - AssertHelpers.assertThrows( - "Should fail trying to remove unknown field", - IllegalArgumentException.class, - "Cannot delete newly added field", - () -> - new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) - .addField("prefix", truncate("data", 4)) - .removeField("prefix")); + Assertions.assertThatThrownBy( + () -> + new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) + .addField("prefix", truncate("data", 4)) + .removeField("prefix")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot delete newly added field"); } @Test public void testRemoveNewlyAddedFieldByTransform() { - AssertHelpers.assertThrows( - "Should fail adding a duplicate field", - IllegalArgumentException.class, - "Cannot delete newly added field", - () -> - new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) - .addField("prefix", truncate("data", 4)) - .removeField(truncate("data", 4))); + Assertions.assertThatThrownBy( + () -> + new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) + .addField("prefix", truncate("data", 4)) + .removeField(truncate("data", 4))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot delete newly added field"); } @Test public void testAddAlreadyAddedFieldByTransform() { - AssertHelpers.assertThrows( - "Should fail adding a duplicate field", - IllegalArgumentException.class, - "Cannot add duplicate partition field", - () -> - new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) - .addField("prefix", truncate("data", 4)) - .addField(truncate("data", 4))); + Assertions.assertThatThrownBy( + () -> + new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) + .addField("prefix", truncate("data", 4)) + .addField(truncate("data", 4))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot add duplicate partition field"); } @Test public void testAddAlreadyAddedFieldByName() { - AssertHelpers.assertThrows( - "Should fail adding a duplicate field", - IllegalArgumentException.class, - "Cannot add duplicate partition field", - () -> - new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) - .addField("prefix", truncate("data", 4)) - .addField("prefix", truncate("data", 6))); + Assertions.assertThatThrownBy( + () -> + new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) + .addField("prefix", truncate("data", 4)) + .addField("prefix", truncate("data", 6))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot add duplicate partition field"); } @Test public void testAddRedundantTimePartition() { - AssertHelpers.assertThrows( - "Should fail adding a duplicate field", - IllegalArgumentException.class, - "Cannot add redundant partition field", - () -> - new BaseUpdatePartitionSpec(formatVersion, UNPARTITIONED) - .addField(day("ts")) - .addField(hour("ts"))); // conflicts with hour - - AssertHelpers.assertThrows( - "Should fail adding a duplicate field", - IllegalArgumentException.class, - "Cannot add redundant partition", - () -> - new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) - .addField(hour("ts")) // does not conflict with day because day already exists - .addField(month("ts"))); // conflicts with hour + Assertions.assertThatThrownBy( + () -> + new BaseUpdatePartitionSpec(formatVersion, UNPARTITIONED) + .addField(day("ts")) + .addField(hour("ts"))) // conflicts with hour + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot add redundant partition field"); + + Assertions.assertThatThrownBy( + () -> + new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) + .addField(hour("ts")) // does not conflict with day because day already exists + .addField(month("ts"))) // conflicts with hour + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot add redundant partition"); } @Test @@ -532,106 +527,99 @@ public void testGenerateNewSpecAddDeletedSameFieldWithDifferentName() { @Test public void testAddDuplicateByName() { - AssertHelpers.assertThrows( - "Should fail adding a duplicate field", - IllegalArgumentException.class, - "Cannot add duplicate partition field", - () -> new BaseUpdatePartitionSpec(formatVersion, PARTITIONED).addField("category")); + Assertions.assertThatThrownBy( + () -> new BaseUpdatePartitionSpec(formatVersion, PARTITIONED).addField("category")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot add duplicate partition field"); } @Test public void testAddDuplicateByRef() { - AssertHelpers.assertThrows( - "Should fail adding a duplicate field", - IllegalArgumentException.class, - "Cannot add duplicate partition field", - () -> new BaseUpdatePartitionSpec(formatVersion, PARTITIONED).addField(ref("category"))); + Assertions.assertThatThrownBy( + () -> new BaseUpdatePartitionSpec(formatVersion, PARTITIONED).addField(ref("category"))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot add duplicate partition field"); } @Test public void testAddDuplicateTransform() { - AssertHelpers.assertThrows( - "Should fail adding a duplicate field", - IllegalArgumentException.class, - "Cannot add duplicate partition field", - () -> new BaseUpdatePartitionSpec(formatVersion, PARTITIONED).addField(bucket("id", 16))); + Assertions.assertThatThrownBy( + () -> + new BaseUpdatePartitionSpec(formatVersion, PARTITIONED).addField(bucket("id", 16))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot add duplicate partition field"); } @Test public void testAddNamedDuplicate() { - AssertHelpers.assertThrows( - "Should fail adding a duplicate field", - IllegalArgumentException.class, - "Cannot add duplicate partition field", - () -> - new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) - .addField("b16", bucket("id", 16))); + Assertions.assertThatThrownBy( + () -> + new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) + .addField("b16", bucket("id", 16))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot add duplicate partition field"); } @Test public void testRemoveUnknownFieldByName() { - AssertHelpers.assertThrows( - "Should fail trying to remove unknown field", - IllegalArgumentException.class, - "Cannot find partition field to remove", - () -> new BaseUpdatePartitionSpec(formatVersion, PARTITIONED).removeField("moon")); + Assertions.assertThatThrownBy( + () -> new BaseUpdatePartitionSpec(formatVersion, PARTITIONED).removeField("moon")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot find partition field to remove"); } @Test public void testRemoveUnknownFieldByEquivalent() { - AssertHelpers.assertThrows( - "Should fail trying to remove unknown field", - IllegalArgumentException.class, - "Cannot find partition field to remove", - () -> - new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) - .removeField(hour("ts")) // day(ts) exists - ); + Assertions.assertThatThrownBy( + () -> + new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) + .removeField(hour("ts")) // day(ts) exists + ) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot find partition field to remove"); } @Test public void testRenameUnknownField() { - AssertHelpers.assertThrows( - "Should fail trying to rename an unknown field", - IllegalArgumentException.class, - "Cannot find partition field to rename", - () -> new BaseUpdatePartitionSpec(formatVersion, PARTITIONED).renameField("shake", "seal")); + Assertions.assertThatThrownBy( + () -> + new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) + .renameField("shake", "seal")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot find partition field to rename: shake"); } @Test public void testRenameAfterAdd() { - AssertHelpers.assertThrows( - "Should fail trying to rename an added field", - IllegalArgumentException.class, - "Cannot rename newly added partition field", - () -> - new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) - .addField("data_trunc", truncate("data", 4)) - .renameField("data_trunc", "prefix")); + Assertions.assertThatThrownBy( + () -> + new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) + .addField("data_trunc", truncate("data", 4)) + .renameField("data_trunc", "prefix")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot rename newly added partition field: data_trunc"); } @Test public void testDeleteAndRename() { - AssertHelpers.assertThrows( - "Should fail trying to rename a deleted field", - IllegalArgumentException.class, - "Cannot rename and delete partition field", - () -> - new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) - .renameField("shard", "id_bucket") - .removeField(bucket("id", 16))); + Assertions.assertThatThrownBy( + () -> + new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) + .renameField("shard", "id_bucket") + .removeField(bucket("id", 16))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot rename and delete partition field: shard"); } @Test public void testRenameAndDelete() { - AssertHelpers.assertThrows( - "Should fail trying to delete a renamed field", - IllegalArgumentException.class, - "Cannot delete and rename partition field", - () -> - new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) - .removeField(bucket("id", 16)) - .renameField("shard", "id_bucket")); + Assertions.assertThatThrownBy( + () -> + new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) + .removeField(bucket("id", 16)) + .renameField("shard", "id_bucket")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot delete and rename partition field: shard"); } @Test diff --git a/core/src/test/java/org/apache/iceberg/rest/requests/TestUpdateRequirementParser.java b/core/src/test/java/org/apache/iceberg/TestUpdateRequirementParser.java similarity index 89% rename from core/src/test/java/org/apache/iceberg/rest/requests/TestUpdateRequirementParser.java rename to core/src/test/java/org/apache/iceberg/TestUpdateRequirementParser.java index 9af2b1d6588f..92401b9d9ef5 100644 --- a/core/src/test/java/org/apache/iceberg/rest/requests/TestUpdateRequirementParser.java +++ b/core/src/test/java/org/apache/iceberg/TestUpdateRequirementParser.java @@ -16,14 +16,12 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.rest.requests; +package org.apache.iceberg; import java.util.List; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.rest.requests.UpdateTableRequest.UpdateRequirement; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestUpdateRequirementParser { @@ -55,10 +53,9 @@ public void testAssertUUIDToJson() { String uuid = "2cc52516-5e73-41f2-b139-545d41a4e151"; String expected = String.format("{\"type\":\"assert-table-uuid\",\"uuid\":\"%s\"}", uuid); UpdateRequirement actual = new UpdateRequirement.AssertTableUUID(uuid); - Assert.assertEquals( - "AssertTableUUID should convert to the correct JSON value", - expected, - UpdateRequirementParser.toJson(actual)); + Assertions.assertThat(UpdateRequirementParser.toJson(actual)) + .as("AssertTableUUID should convert to the correct JSON value") + .isEqualTo(expected); } @Test @@ -73,10 +70,9 @@ public void testAssertTableDoesNotExistFromJson() { public void testAssertTableDoesNotExistToJson() { String expected = "{\"type\":\"assert-create\"}"; UpdateRequirement actual = new UpdateRequirement.AssertTableDoesNotExist(); - Assert.assertEquals( - "AssertTableDoesNotExist should convert to the correct JSON value", - expected, - UpdateRequirementParser.toJson(actual)); + Assertions.assertThat(UpdateRequirementParser.toJson(actual)) + .as("AssertTableDoesNotExist should convert to the correct JSON value") + .isEqualTo(expected); } @Test @@ -115,10 +111,9 @@ public void testAssertRefSnapshotIdFromJson() { "{\"type\":\"%s\",\"ref\":\"%s\",\"snapshot-id\":%d}", requirementType, refName, snapshotId); UpdateRequirement actual = new UpdateRequirement.AssertRefSnapshotID(refName, snapshotId); - Assert.assertEquals( - "AssertRefSnapshotId should convert to the correct JSON value", - expected, - UpdateRequirementParser.toJson(actual)); + Assertions.assertThat(UpdateRequirementParser.toJson(actual)) + .as("AssertRefSnapshotId should convert to the correct JSON value") + .isEqualTo(expected); } @Test @@ -131,10 +126,9 @@ public void testAssertRefSnapshotIdFromJsonWithNullSnapshotId() { "{\"type\":\"%s\",\"ref\":\"%s\",\"snapshot-id\":%d}", requirementType, refName, snapshotId); UpdateRequirement actual = new UpdateRequirement.AssertRefSnapshotID(refName, snapshotId); - Assert.assertEquals( - "AssertRefSnapshotId should convert to the correct JSON value", - expected, - UpdateRequirementParser.toJson(actual)); + Assertions.assertThat(UpdateRequirementParser.toJson(actual)) + .as("AssertRefSnapshotId should convert to the correct JSON value") + .isEqualTo(expected); } @Test @@ -159,10 +153,9 @@ public void testAssertLastAssignedFieldIdToJson() { "{\"type\":\"%s\",\"last-assigned-field-id\":%d}", requirementType, lastAssignedFieldId); UpdateRequirement actual = new UpdateRequirement.AssertLastAssignedFieldId(lastAssignedFieldId); - Assert.assertEquals( - "AssertLastAssignedFieldId should convert to the correct JSON value", - expected, - UpdateRequirementParser.toJson(actual)); + Assertions.assertThat(UpdateRequirementParser.toJson(actual)) + .as("AssertLastAssignedFieldId should convert to the correct JSON value") + .isEqualTo(expected); } @Test @@ -182,10 +175,9 @@ public void testAssertCurrentSchemaIdToJson() { String expected = String.format("{\"type\":\"%s\",\"current-schema-id\":%d}", requirementType, schemaId); UpdateRequirement actual = new UpdateRequirement.AssertCurrentSchemaID(schemaId); - Assert.assertEquals( - "AssertCurrentSchemaId should convert to the correct JSON value", - expected, - UpdateRequirementParser.toJson(actual)); + Assertions.assertThat(UpdateRequirementParser.toJson(actual)) + .as("AssertCurrentSchemaId should convert to the correct JSON value") + .isEqualTo(expected); } @Test @@ -211,10 +203,9 @@ public void testAssertLastAssignedPartitionIdToJson() { requirementType, lastAssignedPartitionId); UpdateRequirement actual = new UpdateRequirement.AssertLastAssignedPartitionId(lastAssignedPartitionId); - Assert.assertEquals( - "AssertLastAssignedPartitionId should convert to the correct JSON value", - expected, - UpdateRequirementParser.toJson(actual)); + Assertions.assertThat(UpdateRequirementParser.toJson(actual)) + .as("AssertLastAssignedPartitionId should convert to the correct JSON value") + .isEqualTo(expected); } @Test @@ -234,10 +225,9 @@ public void testAssertDefaultSpecIdToJson() { String expected = String.format("{\"type\":\"%s\",\"default-spec-id\":%d}", requirementType, specId); UpdateRequirement actual = new UpdateRequirement.AssertDefaultSpecID(specId); - Assert.assertEquals( - "AssertDefaultSpecId should convert to the correct JSON value", - expected, - UpdateRequirementParser.toJson(actual)); + Assertions.assertThat(UpdateRequirementParser.toJson(actual)) + .as("AssertDefaultSpecId should convert to the correct JSON value") + .isEqualTo(expected); } @Test @@ -259,10 +249,9 @@ public void testAssertDefaultSortOrderIdToJson() { String.format( "{\"type\":\"%s\",\"default-sort-order-id\":%d}", requirementType, sortOrderId); UpdateRequirement actual = new UpdateRequirement.AssertDefaultSortOrderID(sortOrderId); - Assert.assertEquals( - "AssertDefaultSortOrderId should convert to the correct JSON value", - expected, - UpdateRequirementParser.toJson(actual)); + Assertions.assertThat(UpdateRequirementParser.toJson(actual)) + .as("AssertDefaultSortOrderId should convert to the correct JSON value") + .isEqualTo(expected); } public void assertEquals( @@ -310,7 +299,7 @@ public void assertEquals( (UpdateRequirement.AssertDefaultSortOrderID) actual); break; default: - Assert.fail("Unrecognized update requirement type: " + requirementType); + Assertions.fail("Unrecognized update requirement type: " + requirementType); } } diff --git a/core/src/test/java/org/apache/iceberg/TestUpdateRequirements.java b/core/src/test/java/org/apache/iceberg/TestUpdateRequirements.java new file mode 100644 index 000000000000..bf3c32628ab0 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestUpdateRequirements.java @@ -0,0 +1,577 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.util.List; +import java.util.UUID; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.Types; +import org.assertj.core.api.InstanceOfAssertFactories; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +public class TestUpdateRequirements { + private final TableMetadata metadata = mock(TableMetadata.class); + private final TableMetadata updated = mock(TableMetadata.class); + + @BeforeEach + public void before() { + String uuid = UUID.randomUUID().toString(); + when(metadata.uuid()).thenReturn(uuid); + when(updated.uuid()).thenReturn(uuid); + } + + @Test + public void nullCheck() { + assertThatThrownBy(() -> UpdateRequirements.forCreateTable(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid metadata updates: null"); + + assertThatThrownBy(() -> UpdateRequirements.forUpdateTable(null, null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid table metadata: null"); + + assertThatThrownBy(() -> UpdateRequirements.forUpdateTable(metadata, null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid metadata updates: null"); + + assertThatThrownBy(() -> UpdateRequirements.forReplaceTable(null, null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid table metadata: null"); + + assertThatThrownBy(() -> UpdateRequirements.forReplaceTable(metadata, null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid metadata updates: null"); + } + + @Test + public void emptyUpdatesForCreateTable() { + assertThat(UpdateRequirements.forCreateTable(ImmutableList.of())) + .hasSize(1) + .hasOnlyElementsOfType(UpdateRequirement.AssertTableDoesNotExist.class); + } + + @Test + public void emptyUpdatesForUpdateAndReplaceTable() { + assertThat(UpdateRequirements.forReplaceTable(metadata, ImmutableList.of())) + .hasSize(1) + .hasOnlyElementsOfType(UpdateRequirement.AssertTableUUID.class); + + assertThat(UpdateRequirements.forUpdateTable(metadata, ImmutableList.of())) + .hasSize(1) + .hasOnlyElementsOfType(UpdateRequirement.AssertTableUUID.class); + } + + @Test + public void tableAlreadyExists() { + List requirements = UpdateRequirements.forCreateTable(ImmutableList.of()); + + assertThatThrownBy(() -> requirements.forEach(req -> req.validate(metadata))) + .isInstanceOf(CommitFailedException.class) + .hasMessage("Requirement failed: table already exists"); + } + + @Test + public void assignUUID() { + List requirements = + UpdateRequirements.forUpdateTable( + metadata, + ImmutableList.of( + new MetadataUpdate.AssignUUID(metadata.uuid()), + new MetadataUpdate.AssignUUID(UUID.randomUUID().toString()), + new MetadataUpdate.AssignUUID(UUID.randomUUID().toString()))); + requirements.forEach(req -> req.validate(metadata)); + + assertThat(requirements) + .hasSize(1) + .hasOnlyElementsOfType(UpdateRequirement.AssertTableUUID.class); + + assertTableUUID(requirements); + } + + @Test + public void assignUUIDFailure() { + List requirements = + UpdateRequirements.forUpdateTable( + metadata, ImmutableList.of(new MetadataUpdate.AssignUUID(metadata.uuid()))); + + when(updated.uuid()).thenReturn(UUID.randomUUID().toString()); + assertThatThrownBy(() -> requirements.forEach(req -> req.validate(updated))) + .isInstanceOf(CommitFailedException.class) + .hasMessage( + String.format( + "Requirement failed: UUID does not match: expected %s != %s", + updated.uuid(), metadata.uuid())); + } + + @Test + public void upgradeFormatVersion() { + List requirements = + UpdateRequirements.forUpdateTable( + metadata, ImmutableList.of(new MetadataUpdate.UpgradeFormatVersion(2))); + requirements.forEach(req -> req.validate(metadata)); + + assertThat(requirements) + .hasSize(1) + .hasOnlyElementsOfType(UpdateRequirement.AssertTableUUID.class); + + assertTableUUID(requirements); + } + + @Test + public void addSchema() { + int lastColumnId = 1; + when(metadata.lastColumnId()).thenReturn(lastColumnId); + List requirements = + UpdateRequirements.forUpdateTable( + metadata, + ImmutableList.of( + new MetadataUpdate.AddSchema(new Schema(), lastColumnId), + new MetadataUpdate.AddSchema(new Schema(), lastColumnId + 1), + new MetadataUpdate.AddSchema(new Schema(), lastColumnId + 2))); + requirements.forEach(req -> req.validate(metadata)); + + assertThat(requirements) + .hasSize(2) + .hasOnlyElementsOfTypes( + UpdateRequirement.AssertTableUUID.class, + UpdateRequirement.AssertLastAssignedFieldId.class); + + assertTableUUID(requirements); + + assertThat(requirements) + .element(1) + .asInstanceOf( + InstanceOfAssertFactories.type(UpdateRequirement.AssertLastAssignedFieldId.class)) + .extracting(UpdateRequirement.AssertLastAssignedFieldId::lastAssignedFieldId) + .isEqualTo(lastColumnId); + } + + @Test + public void addSchemaFailure() { + when(metadata.lastColumnId()).thenReturn(2); + when(updated.lastColumnId()).thenReturn(3); + + List requirements = + UpdateRequirements.forUpdateTable( + metadata, + ImmutableList.of( + new MetadataUpdate.AddSchema(new Schema(), 1), + new MetadataUpdate.AddSchema(new Schema(), 2), + new MetadataUpdate.AddSchema(new Schema(), 3))); + + assertThatThrownBy(() -> requirements.forEach(req -> req.validate(updated))) + .isInstanceOf(CommitFailedException.class) + .hasMessage("Requirement failed: last assigned field id changed: expected id 2 != 3"); + } + + @Test + public void setCurrentSchema() { + int schemaId = 3; + when(metadata.currentSchemaId()).thenReturn(schemaId); + List requirements = + UpdateRequirements.forUpdateTable( + metadata, + ImmutableList.of( + new MetadataUpdate.SetCurrentSchema(schemaId), + new MetadataUpdate.SetCurrentSchema(schemaId + 1), + new MetadataUpdate.SetCurrentSchema(schemaId + 2))); + requirements.forEach(req -> req.validate(metadata)); + + assertThat(requirements) + .hasSize(2) + .hasOnlyElementsOfTypes( + UpdateRequirement.AssertTableUUID.class, UpdateRequirement.AssertCurrentSchemaID.class); + + assertTableUUID(requirements); + + assertThat(requirements) + .element(1) + .asInstanceOf(InstanceOfAssertFactories.type(UpdateRequirement.AssertCurrentSchemaID.class)) + .extracting(UpdateRequirement.AssertCurrentSchemaID::schemaId) + .isEqualTo(schemaId); + } + + @Test + public void setCurrentSchemaFailure() { + int schemaId = 3; + when(metadata.currentSchemaId()).thenReturn(schemaId); + when(updated.currentSchemaId()).thenReturn(schemaId + 1); + + List requirements = + UpdateRequirements.forUpdateTable( + metadata, + ImmutableList.of( + new MetadataUpdate.SetCurrentSchema(schemaId), + new MetadataUpdate.SetCurrentSchema(schemaId + 1), + new MetadataUpdate.SetCurrentSchema(schemaId + 2))); + + assertThatThrownBy(() -> requirements.forEach(req -> req.validate(updated))) + .isInstanceOf(CommitFailedException.class) + .hasMessage("Requirement failed: current schema changed: expected id 3 != 4"); + } + + @Test + public void addPartitionSpec() { + int lastAssignedPartitionId = 3; + when(metadata.lastAssignedPartitionId()).thenReturn(lastAssignedPartitionId); + Schema schema = new Schema(required(3, "id", Types.IntegerType.get())); + List requirements = + UpdateRequirements.forUpdateTable( + metadata, + ImmutableList.of( + new MetadataUpdate.AddPartitionSpec( + PartitionSpec.builderFor(schema).withSpecId(lastAssignedPartitionId).build()), + new MetadataUpdate.AddPartitionSpec( + PartitionSpec.builderFor(schema) + .withSpecId(lastAssignedPartitionId + 1) + .build()), + new MetadataUpdate.AddPartitionSpec( + PartitionSpec.builderFor(schema) + .withSpecId(lastAssignedPartitionId + 2) + .build()))); + requirements.forEach(req -> req.validate(metadata)); + + assertThat(requirements) + .hasSize(2) + .hasOnlyElementsOfTypes( + UpdateRequirement.AssertTableUUID.class, + UpdateRequirement.AssertLastAssignedPartitionId.class); + + assertTableUUID(requirements); + + assertThat(requirements) + .element(1) + .asInstanceOf( + InstanceOfAssertFactories.type(UpdateRequirement.AssertLastAssignedPartitionId.class)) + .extracting(UpdateRequirement.AssertLastAssignedPartitionId::lastAssignedPartitionId) + .isEqualTo(lastAssignedPartitionId); + } + + @Test + public void addPartitionSpecFailure() { + when(metadata.lastAssignedPartitionId()).thenReturn(3); + when(updated.lastAssignedPartitionId()).thenReturn(4); + + List requirements = + UpdateRequirements.forUpdateTable( + metadata, + ImmutableList.of(new MetadataUpdate.AddPartitionSpec(PartitionSpec.unpartitioned()))); + + assertThatThrownBy(() -> requirements.forEach(req -> req.validate(updated))) + .isInstanceOf(CommitFailedException.class) + .hasMessage("Requirement failed: last assigned partition id changed: expected id 3 != 4"); + } + + @Test + public void setDefaultPartitionSpec() { + int specId = 3; + when(metadata.defaultSpecId()).thenReturn(specId); + List requirements = + UpdateRequirements.forUpdateTable( + metadata, + ImmutableList.of( + new MetadataUpdate.SetDefaultPartitionSpec(specId), + new MetadataUpdate.SetDefaultPartitionSpec(specId + 1), + new MetadataUpdate.SetDefaultPartitionSpec(specId + 2))); + requirements.forEach(req -> req.validate(metadata)); + + assertThat(requirements) + .hasSize(2) + .hasOnlyElementsOfTypes( + UpdateRequirement.AssertTableUUID.class, UpdateRequirement.AssertDefaultSpecID.class); + + assertTableUUID(requirements); + + assertThat(requirements) + .element(1) + .asInstanceOf(InstanceOfAssertFactories.type(UpdateRequirement.AssertDefaultSpecID.class)) + .extracting(UpdateRequirement.AssertDefaultSpecID::specId) + .isEqualTo(specId); + } + + @Test + public void setDefaultPartitionSpecFailure() { + int specId = PartitionSpec.unpartitioned().specId(); + when(updated.defaultSpecId()).thenReturn(specId + 1); + + List requirements = + UpdateRequirements.forUpdateTable( + metadata, + ImmutableList.of( + new MetadataUpdate.SetDefaultPartitionSpec(specId), + new MetadataUpdate.SetDefaultPartitionSpec(specId + 1), + new MetadataUpdate.SetDefaultPartitionSpec(specId + 2))); + + assertThatThrownBy(() -> requirements.forEach(req -> req.validate(updated))) + .isInstanceOf(CommitFailedException.class) + .hasMessage("Requirement failed: default partition spec changed: expected id 0 != 1"); + } + + @Test + public void addSortOrder() { + List requirements = + UpdateRequirements.forUpdateTable( + metadata, ImmutableList.of(new MetadataUpdate.AddSortOrder(SortOrder.unsorted()))); + requirements.forEach(req -> req.validate(metadata)); + + assertThat(requirements) + .hasSize(1) + .hasOnlyElementsOfTypes(UpdateRequirement.AssertTableUUID.class); + + assertTableUUID(requirements); + } + + @Test + public void setDefaultSortOrder() { + int sortOrderId = 3; + when(metadata.defaultSortOrderId()).thenReturn(sortOrderId); + List requirements = + UpdateRequirements.forUpdateTable( + metadata, + ImmutableList.of( + new MetadataUpdate.SetDefaultSortOrder(sortOrderId), + new MetadataUpdate.SetDefaultSortOrder(sortOrderId + 1), + new MetadataUpdate.SetDefaultSortOrder(sortOrderId + 2))); + requirements.forEach(req -> req.validate(metadata)); + + assertThat(requirements) + .hasSize(2) + .hasOnlyElementsOfTypes( + UpdateRequirement.AssertTableUUID.class, + UpdateRequirement.AssertDefaultSortOrderID.class); + + assertTableUUID(requirements); + + assertThat(requirements) + .element(1) + .asInstanceOf( + InstanceOfAssertFactories.type(UpdateRequirement.AssertDefaultSortOrderID.class)) + .extracting(UpdateRequirement.AssertDefaultSortOrderID::sortOrderId) + .isEqualTo(sortOrderId); + } + + @Test + public void setDefaultSortOrderFailure() { + int sortOrderId = SortOrder.unsorted().orderId(); + when(updated.defaultSortOrderId()).thenReturn(sortOrderId + 1); + List requirements = + UpdateRequirements.forUpdateTable( + metadata, ImmutableList.of(new MetadataUpdate.SetDefaultSortOrder(sortOrderId))); + + assertThatThrownBy(() -> requirements.forEach(req -> req.validate(updated))) + .isInstanceOf(CommitFailedException.class) + .hasMessage("Requirement failed: default sort order changed: expected id 0 != 1"); + } + + @Test + public void setAndRemoveStatistics() { + List requirements = + UpdateRequirements.forUpdateTable( + metadata, + ImmutableList.of(new MetadataUpdate.SetStatistics(0L, mock(StatisticsFile.class)))); + requirements.forEach(req -> req.validate(metadata)); + + assertThat(requirements) + .hasSize(1) + .hasOnlyElementsOfTypes(UpdateRequirement.AssertTableUUID.class); + + assertTableUUID(requirements); + + requirements = + UpdateRequirements.forUpdateTable( + metadata, ImmutableList.of(new MetadataUpdate.RemoveStatistics(0L))); + requirements.forEach(req -> req.validate(metadata)); + + assertThat(requirements) + .hasSize(1) + .hasOnlyElementsOfTypes(UpdateRequirement.AssertTableUUID.class); + + assertTableUUID(requirements); + } + + @Test + public void addAndRemoveSnapshot() { + List requirements = + UpdateRequirements.forUpdateTable( + metadata, ImmutableList.of(new MetadataUpdate.AddSnapshot(mock(Snapshot.class)))); + requirements.forEach(req -> req.validate(metadata)); + + assertThat(requirements) + .hasSize(1) + .hasOnlyElementsOfTypes(UpdateRequirement.AssertTableUUID.class); + + assertTableUUID(requirements); + + requirements = + UpdateRequirements.forUpdateTable( + metadata, ImmutableList.of(new MetadataUpdate.RemoveSnapshot(0L))); + + assertThat(requirements) + .hasSize(1) + .hasOnlyElementsOfTypes(UpdateRequirement.AssertTableUUID.class); + + assertTableUUID(requirements); + } + + @Test + public void setAndRemoveSnapshotRef() { + long snapshotId = 14L; + String refName = "branch"; + SnapshotRef snapshotRef = mock(SnapshotRef.class); + when(snapshotRef.snapshotId()).thenReturn(snapshotId); + when(metadata.ref(refName)).thenReturn(snapshotRef); + + List requirements = + UpdateRequirements.forUpdateTable( + metadata, + ImmutableList.of( + new MetadataUpdate.SetSnapshotRef( + refName, snapshotId, SnapshotRefType.BRANCH, 0, 0L, 0L), + new MetadataUpdate.SetSnapshotRef( + refName, snapshotId + 1, SnapshotRefType.BRANCH, 0, 0L, 0L), + new MetadataUpdate.SetSnapshotRef( + refName, snapshotId + 2, SnapshotRefType.BRANCH, 0, 0L, 0L))); + requirements.forEach(req -> req.validate(metadata)); + + assertThat(requirements) + .hasSize(2) + .hasOnlyElementsOfTypes( + UpdateRequirement.AssertTableUUID.class, UpdateRequirement.AssertRefSnapshotID.class); + + assertTableUUID(requirements); + + UpdateRequirement.AssertRefSnapshotID assertRefSnapshotID = + (UpdateRequirement.AssertRefSnapshotID) requirements.get(1); + assertThat(assertRefSnapshotID.snapshotId()).isEqualTo(snapshotId); + assertThat(assertRefSnapshotID.refName()).isEqualTo(refName); + + requirements = + UpdateRequirements.forUpdateTable( + metadata, ImmutableList.of(new MetadataUpdate.RemoveSnapshot(0L))); + requirements.forEach(req -> req.validate(metadata)); + + assertThat(requirements) + .hasSize(1) + .hasOnlyElementsOfTypes(UpdateRequirement.AssertTableUUID.class); + + assertTableUUID(requirements); + } + + @Test + public void setSnapshotRefFailure() { + long snapshotId = 14L; + String refName = "random_branch"; + SnapshotRef snapshotRef = mock(SnapshotRef.class); + when(snapshotRef.isBranch()).thenReturn(true); + when(snapshotRef.snapshotId()).thenReturn(snapshotId); + + ImmutableList metadataUpdates = + ImmutableList.of( + new MetadataUpdate.SetSnapshotRef( + refName, snapshotId, SnapshotRefType.BRANCH, 0, 0L, 0L)); + + when(metadata.ref(refName)).thenReturn(null); + when(updated.ref(refName)).thenReturn(snapshotRef); + assertThatThrownBy( + () -> + UpdateRequirements.forUpdateTable(metadata, metadataUpdates) + .forEach(req -> req.validate(updated))) + .isInstanceOf(CommitFailedException.class) + .hasMessage("Requirement failed: branch random_branch was created concurrently"); + + when(metadata.ref(refName)).thenReturn(snapshotRef); + when(updated.ref(refName)).thenReturn(null); + assertThatThrownBy( + () -> + UpdateRequirements.forUpdateTable(metadata, metadataUpdates) + .forEach(req -> req.validate(updated))) + .isInstanceOf(CommitFailedException.class) + .hasMessage("Requirement failed: branch or tag random_branch is missing, expected 14"); + + SnapshotRef snapshotRefUpdated = mock(SnapshotRef.class); + when(snapshotRefUpdated.isBranch()).thenReturn(true); + when(snapshotRefUpdated.snapshotId()).thenReturn(snapshotId + 1); + when(updated.ref(refName)).thenReturn(snapshotRefUpdated); + assertThatThrownBy( + () -> + UpdateRequirements.forUpdateTable(metadata, metadataUpdates) + .forEach(req -> req.validate(updated))) + .isInstanceOf(CommitFailedException.class) + .hasMessage("Requirement failed: branch random_branch has changed: expected id 14 != 15"); + } + + @Test + public void setAndRemoveProperties() { + List requirements = + UpdateRequirements.forUpdateTable( + metadata, + ImmutableList.of(new MetadataUpdate.SetProperties(ImmutableMap.of("test", "test")))); + requirements.forEach(req -> req.validate(metadata)); + + assertThat(requirements) + .hasSize(1) + .hasOnlyElementsOfTypes(UpdateRequirement.AssertTableUUID.class); + + assertTableUUID(requirements); + + requirements = + UpdateRequirements.forUpdateTable( + metadata, + ImmutableList.of(new MetadataUpdate.RemoveProperties(Sets.newHashSet("test")))); + requirements.forEach(req -> req.validate(metadata)); + + assertThat(requirements) + .hasSize(1) + .hasOnlyElementsOfTypes(UpdateRequirement.AssertTableUUID.class); + + assertTableUUID(requirements); + } + + @Test + public void setLocation() { + List requirements = + UpdateRequirements.forUpdateTable( + metadata, ImmutableList.of(new MetadataUpdate.SetLocation("location"))); + requirements.forEach(req -> req.validate(metadata)); + + assertThat(requirements) + .hasSize(1) + .hasOnlyElementsOfTypes(UpdateRequirement.AssertTableUUID.class); + + assertTableUUID(requirements); + } + + private void assertTableUUID(List requirements) { + assertThat(requirements) + .element(0) + .asInstanceOf(InstanceOfAssertFactories.type(UpdateRequirement.AssertTableUUID.class)) + .extracting(UpdateRequirement.AssertTableUUID::uuid) + .isEqualTo(metadata.uuid()); + } +} diff --git a/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java b/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java index 978027b81434..c65d469e3262 100644 --- a/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java +++ b/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java @@ -23,6 +23,7 @@ import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Streams; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -86,11 +87,11 @@ public void testCherryPickOverwriteFailsIfCurrentHasChanged() { .get(); // try to cherry-pick, which should fail because the overwrite's parent is no longer current - AssertHelpers.assertThrows( - "Should reject overwrite that is not a fast-forward commit", - ValidationException.class, - "not append, dynamic overwrite, or fast-forward", - () -> table.manageSnapshots().cherrypick(overwrite.snapshotId()).commit()); + Assertions.assertThatThrownBy( + () -> table.manageSnapshots().cherrypick(overwrite.snapshotId()).commit()) + .isInstanceOf(ValidationException.class) + .hasMessage( + "Cannot cherry-pick snapshot 2: not append, dynamic overwrite, or fast-forward"); // the table state should not have changed validateTableFiles(table, FILE_A, FILE_C); @@ -193,14 +194,11 @@ public void testRollbackOnInvalidNonAncestor() { "Snapshot log should indicate number of snapshots committed", 1, base.snapshotLog().size()); // do rollback - AssertHelpers.assertThrows( - "should fail on invalid snapshot", - ValidationException.class, - "Cannot roll back to snapshot, not an ancestor of the current state: 2", - () -> { - // rollback to snapshot that is not an ancestor - table.manageSnapshots().rollbackTo(wapSnapshot.snapshotId()).commit(); - }); + Assertions.assertThatThrownBy( + // rollback to snapshot that is not an ancestor + () -> table.manageSnapshots().rollbackTo(wapSnapshot.snapshotId()).commit()) + .isInstanceOf(ValidationException.class) + .hasMessage("Cannot roll back to snapshot, not an ancestor of the current state: 2"); base = readMetadata(); Assert.assertEquals( @@ -652,14 +650,11 @@ public void testCherrypickingAncestor() { Assert.assertEquals( "Snapshot log should indicate number of snapshots committed", 2, base.snapshotLog().size()); - AssertHelpers.assertThrows( - "should throw exception", - CherrypickAncestorCommitException.class, - String.format("Cannot cherrypick snapshot %s: already an ancestor", 1), - () -> { - // duplicate cherry-pick snapshot - table.manageSnapshots().cherrypick(firstSnapshotId).commit(); - }); + Assertions.assertThatThrownBy( + // duplicate cherry-pick snapshot + () -> table.manageSnapshots().cherrypick(firstSnapshotId).commit()) + .isInstanceOf(CherrypickAncestorCommitException.class) + .hasMessage("Cannot cherrypick snapshot 1: already an ancestor"); } @Test @@ -710,15 +705,12 @@ public void testDuplicateCherrypick() { Assert.assertEquals( "Snapshot log should indicate number of snapshots committed", 2, base.snapshotLog().size()); - AssertHelpers.assertThrows( - "should throw exception", - DuplicateWAPCommitException.class, - String.format( - "Duplicate request to cherry pick wap id that was published already: %s", 123456789), - () -> { - // duplicate cherry-pick snapshot - table.manageSnapshots().cherrypick(wapSnapshot2.snapshotId()).commit(); - }); + Assertions.assertThatThrownBy( + // duplicate cherry-pick snapshot + () -> table.manageSnapshots().cherrypick(wapSnapshot2.snapshotId()).commit()) + .isInstanceOf(DuplicateWAPCommitException.class) + .hasMessage( + "Duplicate request to cherry pick wap id that was published already: 123456789"); } @Test @@ -762,23 +754,18 @@ public void testNonWapCherrypick() { table.currentSnapshot().snapshotId()); // try double cherrypicking of the third snapshot - AssertHelpers.assertThrows( - "should not allow cherrypicking ancestor", - CherrypickAncestorCommitException.class, - String.format("Cannot cherrypick snapshot %s: already an ancestor", 3), - () -> { - // double cherrypicking of second snapshot - table.manageSnapshots().cherrypick(thirdSnapshotId).commit(); - }); + Assertions.assertThatThrownBy( + // double cherrypicking of second snapshot + () -> table.manageSnapshots().cherrypick(thirdSnapshotId).commit()) + .isInstanceOf(CherrypickAncestorCommitException.class) + .hasMessage("Cannot cherrypick snapshot 3: already an ancestor"); // try cherrypicking an ancestor - AssertHelpers.assertThrows( - "should not allow double cherrypick", - CherrypickAncestorCommitException.class, - String.format("Cannot cherrypick snapshot %s: already an ancestor", firstSnapshotId), - () -> { - // double cherrypicking of second snapshot - table.manageSnapshots().cherrypick(firstSnapshotId).commit(); - }); + Assertions.assertThatThrownBy( + // double cherrypicking of second snapshot + () -> table.manageSnapshots().cherrypick(firstSnapshotId).commit()) + .isInstanceOf(CherrypickAncestorCommitException.class) + .hasMessage( + String.format("Cannot cherrypick snapshot %s: already an ancestor", firstSnapshotId)); } } diff --git a/core/src/test/java/org/apache/iceberg/actions/TestCommitService.java b/core/src/test/java/org/apache/iceberg/actions/TestCommitService.java new file mode 100644 index 000000000000..1aae6483337f --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/actions/TestCommitService.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.actions; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.spy; + +import java.util.Set; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.stream.IntStream; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.util.Tasks; +import org.assertj.core.api.Assertions; +import org.awaitility.Awaitility; +import org.junit.Test; + +public class TestCommitService extends TableTestBase { + + public TestCommitService() { + super(1); + } + + @Test + public void testCommittedResultsCorrectly() { + CustomCommitService commitService = new CustomCommitService(table, 5, 10000); + commitService.start(); + + ExecutorService executorService = Executors.newFixedThreadPool(10); + int numberOfFileGroups = 100; + Tasks.range(numberOfFileGroups).executeWith(executorService).run(commitService::offer); + commitService.close(); + + Set expected = Sets.newHashSet(IntStream.range(0, 100).iterator()); + Set actual = Sets.newHashSet(commitService.results()); + Assertions.assertThat(actual).isEqualTo(expected); + } + + @Test + public void testAbortFileGroupsAfterTimeout() { + CustomCommitService commitService = new CustomCommitService(table, 5, 200); + commitService.start(); + + // Add file groups [0-3] for commit. + // There are less than the rewritesPerCommit, and thus will not trigger a commit action. Those + // file groups will be added to the completedRewrites queue. + // Now the queue has 4 file groups that need to commit. + for (int i = 0; i < 4; i++) { + commitService.offer(i); + } + + // Add file groups [4-7] for commit + // These are gated to not be able to commit, so all those 4 file groups will be added to the + // queue as well. + // Now the queue has 8 file groups that need to commit. + CustomCommitService spyCommitService = spy(commitService); + doReturn(false).when(spyCommitService).canCreateCommitGroup(); + for (int i = 4; i < 8; i++) { + spyCommitService.offer(i); + } + + // close commitService. + // This allows committerService thread to start to commit the remaining file groups [0-7] in the + // completedRewrites queue. And also the main thread waits for the committerService thread to + // finish within a timeout. + + // The committerService thread commits file groups [0-4]. These will wait a fixed duration to + // simulate timeout on the main thread, which then tries to abort file groups [5-7]. + // This tests the race conditions, as the committerService is also trying to commit groups + // [5-7]. + Assertions.assertThatThrownBy(commitService::close) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Timeout occurred when waiting for commits"); + + // Wait for the commitService to finish. Committed all file groups or aborted remaining file + // groups. + Awaitility.await() + .atMost(5, TimeUnit.SECONDS) + .pollInSameThread() + .untilAsserted(() -> assertThat(commitService.completedRewritesAllCommitted()).isTrue()); + if (commitService.aborted.isEmpty()) { + // All file groups are committed + Assertions.assertThat(commitService.results()) + .isEqualTo(ImmutableList.of(0, 1, 2, 3, 4, 5, 6, 7)); + } else { + // File groups [5-7] are aborted + Assertions.assertThat(commitService.results()) + .doesNotContainAnyElementsOf(commitService.aborted); + Assertions.assertThat(commitService.results()).isEqualTo(ImmutableList.of(0, 1, 2, 3, 4)); + Assertions.assertThat(commitService.aborted).isEqualTo(ImmutableSet.of(5, 6, 7)); + } + } + + private static class CustomCommitService extends BaseCommitService { + private final Set aborted = Sets.newConcurrentHashSet(); + + CustomCommitService(Table table, int rewritesPerCommit, int timeoutInSeconds) { + super(table, rewritesPerCommit, timeoutInSeconds); + } + + @Override + protected void commitOrClean(Set batch) { + try { + // Slightly longer than timeout + Thread.sleep(210); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + @Override + protected void abortFileGroup(Integer group) { + aborted.add(group); + } + } +} diff --git a/core/src/test/java/org/apache/iceberg/avro/AvroTestHelpers.java b/core/src/test/java/org/apache/iceberg/avro/AvroTestHelpers.java index 974c29c14f86..af35e27f5b22 100644 --- a/core/src/test/java/org/apache/iceberg/avro/AvroTestHelpers.java +++ b/core/src/test/java/org/apache/iceberg/avro/AvroTestHelpers.java @@ -29,7 +29,6 @@ import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.assertj.core.api.Assertions; -import org.junit.Assert; class AvroTestHelpers { @@ -82,7 +81,7 @@ static void assertEquals(Types.StructType struct, Record expected, Record actual static void assertEquals(Types.ListType list, List expected, List actual) { Type elementType = list.elementType(); - Assert.assertEquals("List size should match", expected.size(), actual.size()); + Assertions.assertThat(actual).as("List size should match").hasSameSizeAs(expected); for (int i = 0; i < expected.size(); i += 1) { Object expectedValue = expected.get(i); @@ -95,7 +94,7 @@ static void assertEquals(Types.ListType list, List expected, List actual) static void assertEquals(Types.MapType map, Map expected, Map actual) { Type valueType = map.valueType(); - Assert.assertEquals("Map size should match", expected.size(), actual.size()); + Assertions.assertThat(actual).as("Map keys should match").hasSameSizeAs(expected); for (Object expectedKey : expected.keySet()) { Object expectedValue = expected.get(expectedKey); @@ -124,7 +123,9 @@ private static void assertEquals(Type type, Object expected, Object actual) { case FIXED: case BINARY: case DECIMAL: - Assert.assertEquals("Primitive value should be equal to expected", expected, actual); + Assertions.assertThat(actual) + .as("Primitive value should be equal to expected") + .isEqualTo(expected); break; case STRUCT: Assertions.assertThat(expected) diff --git a/core/src/test/java/org/apache/iceberg/avro/TestAvroNameMapping.java b/core/src/test/java/org/apache/iceberg/avro/TestAvroNameMapping.java index c6bdb88b7d97..2efb5c8f9af3 100644 --- a/core/src/test/java/org/apache/iceberg/avro/TestAvroNameMapping.java +++ b/core/src/test/java/org/apache/iceberg/avro/TestAvroNameMapping.java @@ -28,7 +28,6 @@ import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericDatumWriter; import org.apache.avro.io.DatumWriter; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.Files; import org.apache.iceberg.Schema; import org.apache.iceberg.mapping.MappedField; @@ -41,6 +40,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Comparators; import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Test; @@ -202,12 +202,11 @@ public void testMissingRequiredFields() { new Schema(Types.NestedField.optional(18, "y", Types.IntegerType.get()))); Schema readSchema = writeSchema; - AssertHelpers.assertThrows( - "Missing required field in nameMapping", - IllegalArgumentException.class, - "Missing required field: x", - // In this case, pruneColumns result is an empty record - () -> writeAndRead(writeSchema, readSchema, record, nameMapping)); + Assertions.assertThatThrownBy( + // In this case, pruneColumns result is an empty record + () -> writeAndRead(writeSchema, readSchema, record, nameMapping)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Missing required field: x"); } @Test diff --git a/core/src/test/java/org/apache/iceberg/avro/TestReadProjection.java b/core/src/test/java/org/apache/iceberg/avro/TestReadProjection.java index 8c8e7b24d5ba..b6b6c2c967d8 100644 --- a/core/src/test/java/org/apache/iceberg/avro/TestReadProjection.java +++ b/core/src/test/java/org/apache/iceberg/avro/TestReadProjection.java @@ -21,8 +21,9 @@ import java.io.IOException; import java.util.List; import java.util.Map; +import org.apache.avro.AvroRuntimeException; import org.apache.avro.generic.GenericData.Record; -import org.apache.iceberg.AssertHelpers; +import org.apache.avro.generic.GenericRecord; import org.apache.iceberg.Schema; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -140,14 +141,14 @@ public void testBasicProjection() throws Exception { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); Record projected = writeAndRead("basic_projection_id", writeSchema, idOnly, record); - AssertHelpers.assertEmptyAvroField(projected, "data"); + assertEmptyAvroField(projected, "data"); Assert.assertEquals("Should contain the correct id value", 34L, (long) projected.get("id")); Schema dataOnly = new Schema(Types.NestedField.optional(1, "data", Types.StringType.get())); projected = writeAndRead("basic_projection_data", writeSchema, dataOnly, record); - AssertHelpers.assertEmptyAvroField(projected, "id"); + assertEmptyAvroField(projected, "id"); int cmp = Comparators.charSequences().compare("test", (CharSequence) projected.get("data")); Assert.assertEquals("Should contain the correct data value", 0, cmp); } @@ -198,7 +199,7 @@ public void testNestedStructProjection() throws Exception { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); Record projected = writeAndRead("id_only", writeSchema, idOnly, record); - AssertHelpers.assertEmptyAvroField(projected, "location"); + assertEmptyAvroField(projected, "location"); Assert.assertEquals("Should contain the correct id value", 34L, (long) projected.get("id")); Schema latOnly = @@ -210,9 +211,9 @@ public void testNestedStructProjection() throws Exception { projected = writeAndRead("latitude_only", writeSchema, latOnly, record); Record projectedLocation = (Record) projected.get("location"); - AssertHelpers.assertEmptyAvroField(projected, "id"); + assertEmptyAvroField(projected, "id"); Assert.assertNotNull("Should project location", projected.get("location")); - AssertHelpers.assertEmptyAvroField(projectedLocation, "long"); + assertEmptyAvroField(projectedLocation, "long"); Assert.assertEquals( "Should project latitude", 52.995143f, (float) projectedLocation.get("lat"), 0.000001f); @@ -225,16 +226,16 @@ public void testNestedStructProjection() throws Exception { projected = writeAndRead("longitude_only", writeSchema, longOnly, record); projectedLocation = (Record) projected.get("location"); - AssertHelpers.assertEmptyAvroField(projected, "id"); + assertEmptyAvroField(projected, "id"); Assert.assertNotNull("Should project location", projected.get("location")); - AssertHelpers.assertEmptyAvroField(projectedLocation, "lat"); + assertEmptyAvroField(projectedLocation, "lat"); Assert.assertEquals( "Should project longitude", -1.539054f, (float) projectedLocation.get("long"), 0.000001f); Schema locationOnly = writeSchema.select("location"); projected = writeAndRead("location_only", writeSchema, locationOnly, record); projectedLocation = (Record) projected.get("location"); - AssertHelpers.assertEmptyAvroField(projected, "id"); + assertEmptyAvroField(projected, "id"); Assert.assertNotNull("Should project location", projected.get("location")); Assert.assertEquals( "Should project latitude", 52.995143f, (float) projectedLocation.get("lat"), 0.000001f); @@ -262,23 +263,23 @@ public void testMapProjection() throws IOException { Record projected = writeAndRead("id_only", writeSchema, idOnly, record); Assert.assertEquals("Should contain the correct id value", 34L, (long) projected.get("id")); - AssertHelpers.assertEmptyAvroField(projected, "properties"); + assertEmptyAvroField(projected, "properties"); Schema keyOnly = writeSchema.select("properties.key"); projected = writeAndRead("key_only", writeSchema, keyOnly, record); - AssertHelpers.assertEmptyAvroField(projected, "id"); + assertEmptyAvroField(projected, "id"); Assert.assertEquals( "Should project entire map", properties, toStringMap((Map) projected.get("properties"))); Schema valueOnly = writeSchema.select("properties.value"); projected = writeAndRead("value_only", writeSchema, valueOnly, record); - AssertHelpers.assertEmptyAvroField(projected, "id"); + assertEmptyAvroField(projected, "id"); Assert.assertEquals( "Should project entire map", properties, toStringMap((Map) projected.get("properties"))); Schema mapOnly = writeSchema.select("properties"); projected = writeAndRead("map_only", writeSchema, mapOnly, record); - AssertHelpers.assertEmptyAvroField(projected, "id"); + assertEmptyAvroField(projected, "id"); Assert.assertEquals( "Should project entire map", properties, toStringMap((Map) projected.get("properties"))); } @@ -329,17 +330,17 @@ public void testMapOfStructsProjection() throws IOException { Record projected = writeAndRead("id_only", writeSchema, idOnly, record); Assert.assertEquals("Should contain the correct id value", 34L, (long) projected.get("id")); - AssertHelpers.assertEmptyAvroField(projected, "locations"); + assertEmptyAvroField(projected, "locations"); projected = writeAndRead("all_locations", writeSchema, writeSchema.select("locations"), record); - AssertHelpers.assertEmptyAvroField(projected, "id"); + assertEmptyAvroField(projected, "id"); Assert.assertEquals( "Should project locations map", record.get("locations"), toStringMap((Map) projected.get("locations"))); projected = writeAndRead("lat_only", writeSchema, writeSchema.select("locations.lat"), record); - AssertHelpers.assertEmptyAvroField(projected, "id"); + assertEmptyAvroField(projected, "id"); Map locations = toStringMap((Map) projected.get("locations")); Assert.assertNotNull("Should project locations map", locations); Assert.assertEquals( @@ -348,28 +349,28 @@ public void testMapOfStructsProjection() throws IOException { Assert.assertNotNull("L1 should not be null", projectedL1); Assert.assertEquals( "L1 should contain lat", 53.992811f, (float) projectedL1.get("lat"), 0.000001); - AssertHelpers.assertEmptyAvroField(projectedL1, "long"); + assertEmptyAvroField(projectedL1, "long"); Record projectedL2 = (Record) locations.get("L2"); Assert.assertNotNull("L2 should not be null", projectedL2); Assert.assertEquals( "L2 should contain lat", 52.995143f, (float) projectedL2.get("lat"), 0.000001); - AssertHelpers.assertEmptyAvroField(projectedL2, "y"); + assertEmptyAvroField(projectedL2, "y"); projected = writeAndRead("long_only", writeSchema, writeSchema.select("locations.long"), record); - AssertHelpers.assertEmptyAvroField(projected, "id"); + assertEmptyAvroField(projected, "id"); locations = toStringMap((Map) projected.get("locations")); Assert.assertNotNull("Should project locations map", locations); Assert.assertEquals( "Should contain L1 and L2", Sets.newHashSet("L1", "L2"), locations.keySet()); projectedL1 = (Record) locations.get("L1"); Assert.assertNotNull("L1 should not be null", projectedL1); - AssertHelpers.assertEmptyAvroField(projectedL1, "lat"); + assertEmptyAvroField(projectedL1, "lat"); Assert.assertEquals( "L1 should contain long", -1.542616f, (float) projectedL1.get("long"), 0.000001); projectedL2 = (Record) locations.get("L2"); Assert.assertNotNull("L2 should not be null", projectedL2); - AssertHelpers.assertEmptyAvroField(projectedL2, "lat"); + assertEmptyAvroField(projectedL2, "lat"); Assert.assertEquals( "L2 should contain long", -1.539054f, (float) projectedL2.get("long"), 0.000001); @@ -386,7 +387,7 @@ public void testMapOfStructsProjection() throws IOException { Types.NestedField.required(1, "latitude", Types.FloatType.get()))))); projected = writeAndRead("latitude_renamed", writeSchema, latitiudeRenamed, record); - AssertHelpers.assertEmptyAvroField(projected, "id"); + assertEmptyAvroField(projected, "id"); locations = toStringMap((Map) projected.get("locations")); Assert.assertNotNull("Should project locations map", locations); Assert.assertEquals( @@ -395,14 +396,14 @@ public void testMapOfStructsProjection() throws IOException { Assert.assertNotNull("L1 should not be null", projectedL1); Assert.assertEquals( "L1 should contain latitude", 53.992811f, (float) projectedL1.get("latitude"), 0.000001); - AssertHelpers.assertEmptyAvroField(projectedL1, "lat"); - AssertHelpers.assertEmptyAvroField(projectedL1, "long"); + assertEmptyAvroField(projectedL1, "lat"); + assertEmptyAvroField(projectedL1, "long"); projectedL2 = (Record) locations.get("L2"); Assert.assertNotNull("L2 should not be null", projectedL2); Assert.assertEquals( "L2 should contain latitude", 52.995143f, (float) projectedL2.get("latitude"), 0.000001); - AssertHelpers.assertEmptyAvroField(projectedL2, "lat"); - AssertHelpers.assertEmptyAvroField(projectedL2, "long"); + assertEmptyAvroField(projectedL2, "lat"); + assertEmptyAvroField(projectedL2, "long"); } @Test @@ -423,16 +424,16 @@ public void testListProjection() throws IOException { Record projected = writeAndRead("id_only", writeSchema, idOnly, record); Assert.assertEquals("Should contain the correct id value", 34L, (long) projected.get("id")); - AssertHelpers.assertEmptyAvroField(projected, "values"); + assertEmptyAvroField(projected, "values"); Schema elementOnly = writeSchema.select("values.element"); projected = writeAndRead("element_only", writeSchema, elementOnly, record); - AssertHelpers.assertEmptyAvroField(projected, "id"); + assertEmptyAvroField(projected, "id"); Assert.assertEquals("Should project entire list", values, projected.get("values")); Schema listOnly = writeSchema.select("values"); projected = writeAndRead("list_only", writeSchema, listOnly, record); - AssertHelpers.assertEmptyAvroField(projected, "id"); + assertEmptyAvroField(projected, "id"); Assert.assertEquals("Should project entire list", values, projected.get("values")); } @@ -469,35 +470,35 @@ public void testListOfStructsProjection() throws IOException { Record projected = writeAndRead("id_only", writeSchema, idOnly, record); Assert.assertEquals("Should contain the correct id value", 34L, (long) projected.get("id")); - AssertHelpers.assertEmptyAvroField(projected, "points"); + assertEmptyAvroField(projected, "points"); projected = writeAndRead("all_points", writeSchema, writeSchema.select("points"), record); - AssertHelpers.assertEmptyAvroField(projected, "id"); + assertEmptyAvroField(projected, "id"); Assert.assertEquals( "Should project points list", record.get("points"), projected.get("points")); projected = writeAndRead("x_only", writeSchema, writeSchema.select("points.x"), record); - AssertHelpers.assertEmptyAvroField(projected, "id"); + assertEmptyAvroField(projected, "id"); Assert.assertNotNull("Should project points list", projected.get("points")); List points = (List) projected.get("points"); Assert.assertEquals("Should read 2 points", 2, points.size()); Record projectedP1 = points.get(0); Assert.assertEquals("Should project x", 1, (int) projectedP1.get("x")); - AssertHelpers.assertEmptyAvroField(projectedP1, "y"); + assertEmptyAvroField(projectedP1, "y"); Record projectedP2 = points.get(1); Assert.assertEquals("Should project x", 3, (int) projectedP2.get("x")); - AssertHelpers.assertEmptyAvroField(projectedP2, "y"); + assertEmptyAvroField(projectedP2, "y"); projected = writeAndRead("y_only", writeSchema, writeSchema.select("points.y"), record); - AssertHelpers.assertEmptyAvroField(projected, "id"); + assertEmptyAvroField(projected, "id"); Assert.assertNotNull("Should project points list", projected.get("points")); points = (List) projected.get("points"); Assert.assertEquals("Should read 2 points", 2, points.size()); projectedP1 = points.get(0); - AssertHelpers.assertEmptyAvroField(projectedP1, "x"); + assertEmptyAvroField(projectedP1, "x"); Assert.assertEquals("Should project y", 2, (int) projectedP1.get("y")); projectedP2 = points.get(1); - AssertHelpers.assertEmptyAvroField(projectedP2, "x"); + assertEmptyAvroField(projectedP2, "x"); Assert.assertEquals("Should project null y", null, projectedP2.get("y")); Schema yRenamed = @@ -511,17 +512,17 @@ public void testListOfStructsProjection() throws IOException { Types.NestedField.optional(18, "z", Types.IntegerType.get()))))); projected = writeAndRead("y_renamed", writeSchema, yRenamed, record); - AssertHelpers.assertEmptyAvroField(projected, "id"); + assertEmptyAvroField(projected, "id"); Assert.assertNotNull("Should project points list", projected.get("points")); points = (List) projected.get("points"); Assert.assertEquals("Should read 2 points", 2, points.size()); projectedP1 = points.get(0); - AssertHelpers.assertEmptyAvroField(projectedP1, "x"); - AssertHelpers.assertEmptyAvroField(projectedP1, "y"); + assertEmptyAvroField(projectedP1, "x"); + assertEmptyAvroField(projectedP1, "y"); Assert.assertEquals("Should project z", 2, (int) projectedP1.get("z")); projectedP2 = points.get(1); - AssertHelpers.assertEmptyAvroField(projectedP2, "x"); - AssertHelpers.assertEmptyAvroField(projectedP2, "y"); + assertEmptyAvroField(projectedP2, "x"); + assertEmptyAvroField(projectedP2, "y"); Assert.assertNull("Should project null z", projectedP2.get("z")); } @@ -549,13 +550,13 @@ public void testEmptyStructProjection() throws Exception { new Schema(Types.NestedField.required(3, "location", Types.StructType.of())); Record projected = writeAndRead("empty_proj", writeSchema, emptyStruct, record); - AssertHelpers.assertEmptyAvroField(projected, "id"); + assertEmptyAvroField(projected, "id"); Record result = (Record) projected.get("location"); Assert.assertEquals("location should be in the 0th position", result, projected.get(0)); Assert.assertNotNull("Should contain an empty record", result); - AssertHelpers.assertEmptyAvroField(result, "lat"); - AssertHelpers.assertEmptyAvroField(result, "long"); + assertEmptyAvroField(result, "lat"); + assertEmptyAvroField(result, "long"); } @Test @@ -581,12 +582,12 @@ public void testEmptyStructRequiredProjection() throws Exception { new Schema(Types.NestedField.required(3, "location", Types.StructType.of())); Record projected = writeAndRead("empty_req_proj", writeSchema, emptyStruct, record); - AssertHelpers.assertEmptyAvroField(projected, "id"); + assertEmptyAvroField(projected, "id"); Record result = (Record) projected.get("location"); Assert.assertEquals("location should be in the 0th position", result, projected.get(0)); Assert.assertNotNull("Should contain an empty record", result); - AssertHelpers.assertEmptyAvroField(result, "lat"); - AssertHelpers.assertEmptyAvroField(result, "long"); + assertEmptyAvroField(result, "lat"); + assertEmptyAvroField(result, "long"); } @Test @@ -623,8 +624,8 @@ public void testRequiredEmptyStructInRequiredStruct() throws Exception { Record result = (Record) projected.get("location"); Assert.assertEquals("location should be in the 1st position", result, projected.get(1)); Assert.assertNotNull("Should contain an empty record", result); - AssertHelpers.assertEmptyAvroField(result, "lat"); - AssertHelpers.assertEmptyAvroField(result, "long"); + assertEmptyAvroField(result, "lat"); + assertEmptyAvroField(result, "long"); Assert.assertNotNull("Should project empty", result.getSchema().getField("empty")); Assert.assertNotNull("Empty should not be null", result.get("empty")); Assert.assertEquals( @@ -667,15 +668,15 @@ public void testEmptyNestedStructProjection() throws Exception { Types.NestedField.required(2, "inner", Types.StructType.of())))); Record projected = writeAndRead("nested_empty_proj", writeSchema, emptyStruct, record); - AssertHelpers.assertEmptyAvroField(projected, "id"); + assertEmptyAvroField(projected, "id"); Record outerResult = (Record) projected.get("outer"); Assert.assertEquals("Outer should be in the 0th position", outerResult, projected.get(0)); Assert.assertNotNull("Should contain the outer record", outerResult); - AssertHelpers.assertEmptyAvroField(outerResult, "lat"); + assertEmptyAvroField(outerResult, "lat"); Record innerResult = (Record) outerResult.get("inner"); Assert.assertEquals("Inner should be in the 0th position", innerResult, outerResult.get(0)); Assert.assertNotNull("Should contain the inner record", innerResult); - AssertHelpers.assertEmptyAvroField(innerResult, "lon"); + assertEmptyAvroField(innerResult, "lon"); } @Test @@ -712,14 +713,20 @@ public void testEmptyNestedStructRequiredProjection() throws Exception { Types.NestedField.required(2, "inner", Types.StructType.of())))); Record projected = writeAndRead("nested_empty_req_proj", writeSchema, emptyStruct, record); - AssertHelpers.assertEmptyAvroField(projected, "id"); + assertEmptyAvroField(projected, "id"); Record outerResult = (Record) projected.get("outer"); Assert.assertEquals("Outer should be in the 0th position", outerResult, projected.get(0)); Assert.assertNotNull("Should contain the outer record", outerResult); - AssertHelpers.assertEmptyAvroField(outerResult, "lat"); + assertEmptyAvroField(outerResult, "lat"); Record innerResult = (Record) outerResult.get("inner"); Assert.assertEquals("Inner should be in the 0th position", innerResult, outerResult.get(0)); Assert.assertNotNull("Should contain the inner record", innerResult); - AssertHelpers.assertEmptyAvroField(innerResult, "lon"); + assertEmptyAvroField(innerResult, "lon"); + } + + private void assertEmptyAvroField(GenericRecord record, String field) { + Assertions.assertThatThrownBy(() -> record.get(field)) + .isInstanceOf(AvroRuntimeException.class) + .hasMessage("Not a valid schema field: " + field); } } diff --git a/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java b/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java index 6e9a175183b2..203b5fc38170 100644 --- a/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java +++ b/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java @@ -42,6 +42,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.TableOperations; import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TestHelpers; import org.apache.iceberg.Transaction; import org.apache.iceberg.UpdatePartitionSpec; import org.apache.iceberg.UpdateSchema; @@ -62,8 +63,6 @@ import org.apache.iceberg.types.Types; import org.apache.iceberg.util.CharSequenceSet; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Assume; import org.junit.jupiter.api.Assumptions; import org.junit.jupiter.api.Test; @@ -75,19 +74,19 @@ public abstract class CatalogTests { // Schema passed to create tables protected static final Schema SCHEMA = new Schema( - required(3, "id", Types.IntegerType.get(), "unique ID"), + required(3, "id", Types.IntegerType.get(), "unique ID 🤪"), required(4, "data", Types.StringType.get())); // This is the actual schema for the table, with column IDs reassigned private static final Schema TABLE_SCHEMA = new Schema( - required(1, "id", Types.IntegerType.get(), "unique ID"), + required(1, "id", Types.IntegerType.get(), "unique ID 🤪"), required(2, "data", Types.StringType.get())); // This is the actual schema for the table, with column IDs reassigned private static final Schema REPLACE_SCHEMA = new Schema( - required(2, "id", Types.IntegerType.get(), "unique ID"), + required(2, "id", Types.IntegerType.get(), "unique ID 🤪"), required(3, "data", Types.StringType.get())); // another schema that is not the same @@ -168,70 +167,70 @@ protected boolean supportsNamesWithSlashes() { public void testCreateNamespace() { C catalog = catalog(); - Assert.assertFalse("Namespace should not exist", catalog.namespaceExists(NS)); + Assertions.assertThat(catalog.namespaceExists(NS)).as("Namespace should not exist").isFalse(); catalog.createNamespace(NS); - Assert.assertTrue( - "Catalog should have the created namespace", catalog.listNamespaces().contains(NS)); - Assert.assertTrue("Namespace should exist", catalog.namespaceExists(NS)); + Assertions.assertThat(catalog.listNamespaces()) + .as("Catalog should have the created namespace") + .contains(NS); + Assertions.assertThat(catalog.namespaceExists(NS)).as("Namespace should exist").isTrue(); } @Test public void testCreateExistingNamespace() { C catalog = catalog(); - Assert.assertFalse("Namespace should not exist", catalog.namespaceExists(NS)); + Assertions.assertThat(catalog.namespaceExists(NS)).as("Namespace should not exist").isFalse(); catalog.createNamespace(NS); - Assert.assertTrue("Namespace should exist", catalog.namespaceExists(NS)); + Assertions.assertThat(catalog.namespaceExists(NS)).as("Namespace should exist").isTrue(); Assertions.assertThatThrownBy(() -> catalog.createNamespace(NS)) .isInstanceOf(AlreadyExistsException.class) .hasMessageContaining("Namespace already exists"); - Assert.assertTrue("Namespace should still exist", catalog.namespaceExists(NS)); + Assertions.assertThat(catalog.namespaceExists(NS)).as("Namespace should still exist").isTrue(); } @Test public void testCreateNamespaceWithProperties() { - Assume.assumeTrue(supportsNamespaceProperties()); + Assumptions.assumeTrue(supportsNamespaceProperties()); C catalog = catalog(); - Assert.assertFalse("Namespace should not exist", catalog.namespaceExists(NS)); + Assertions.assertThat(catalog.namespaceExists(NS)).as("Namespace should not exist").isFalse(); Map createProps = ImmutableMap.of("prop", "val"); catalog.createNamespace(NS, createProps); - Assert.assertTrue("Namespace should exist", catalog.namespaceExists(NS)); + Assertions.assertThat(catalog.namespaceExists(NS)).as("Namespace should exist").isTrue(); Map props = catalog.loadNamespaceMetadata(NS); - Assert.assertEquals( - "Create properties should be a subset of returned properties", - createProps.entrySet(), - Sets.intersection(createProps.entrySet(), props.entrySet())); + + Assertions.assertThat(Sets.intersection(createProps.entrySet(), props.entrySet())) + .as("Create properties should be a subset of returned properties") + .containsExactlyInAnyOrderElementsOf(createProps.entrySet()); } @Test public void testLoadNamespaceMetadata() { C catalog = catalog(); - Assert.assertFalse("Namespace should not exist", catalog.namespaceExists(NS)); + Assertions.assertThat(catalog.namespaceExists(NS)).as("Namespace should not exist").isFalse(); Assertions.assertThatThrownBy(() -> catalog.loadNamespaceMetadata(NS)) .isInstanceOf(NoSuchNamespaceException.class) - .hasMessage("Namespace does not exist: newdb"); + .hasMessageStartingWith("Namespace does not exist: newdb"); catalog.createNamespace(NS); - Assert.assertTrue("Namespace should exist", catalog.namespaceExists(NS)); - + Assertions.assertThat(catalog.namespaceExists(NS)).as("Namespace should exist").isTrue(); Map props = catalog.loadNamespaceMetadata(NS); - Assert.assertNotNull("Should return non-null property map", props); + Assertions.assertThat(props).as("Should return non-null property map").isNotNull(); // note that there are no requirements for the properties returned by the catalog } @Test public void testSetNamespaceProperties() { - Assume.assumeTrue(supportsNamespaceProperties()); + Assumptions.assumeTrue(supportsNamespaceProperties()); C catalog = catalog(); @@ -241,15 +240,14 @@ public void testSetNamespaceProperties() { catalog.setProperties(NS, properties); Map actualProperties = catalog.loadNamespaceMetadata(NS); - Assert.assertEquals( - "Set properties should be a subset of returned properties", - properties.entrySet(), - Sets.intersection(properties.entrySet(), actualProperties.entrySet())); + Assertions.assertThat(actualProperties.entrySet()) + .as("Set properties should be a subset of returned properties") + .containsAll(properties.entrySet()); } @Test public void testUpdateNamespaceProperties() { - Assume.assumeTrue(supportsNamespaceProperties()); + Assumptions.assumeTrue(supportsNamespaceProperties()); C catalog = catalog(); @@ -259,25 +257,23 @@ public void testUpdateNamespaceProperties() { catalog.setProperties(NS, initialProperties); Map actualProperties = catalog.loadNamespaceMetadata(NS); - Assert.assertEquals( - "Set properties should be a subset of returned properties", - initialProperties.entrySet(), - Sets.intersection(initialProperties.entrySet(), actualProperties.entrySet())); + Assertions.assertThat(actualProperties.entrySet()) + .as("Set properties should be a subset of returned properties") + .containsAll(initialProperties.entrySet()); Map updatedProperties = ImmutableMap.of("owner", "newuser"); catalog.setProperties(NS, updatedProperties); Map finalProperties = catalog.loadNamespaceMetadata(NS); - Assert.assertEquals( - "Updated properties should be a subset of returned properties", - updatedProperties.entrySet(), - Sets.intersection(updatedProperties.entrySet(), finalProperties.entrySet())); + Assertions.assertThat(finalProperties.entrySet()) + .as("Updated properties should be a subset of returned properties") + .containsAll(updatedProperties.entrySet()); } @Test public void testUpdateAndSetNamespaceProperties() { - Assume.assumeTrue(supportsNamespaceProperties()); + Assumptions.assumeTrue(supportsNamespaceProperties()); C catalog = catalog(); @@ -287,10 +283,9 @@ public void testUpdateAndSetNamespaceProperties() { catalog.setProperties(NS, initialProperties); Map actualProperties = catalog.loadNamespaceMetadata(NS); - Assert.assertEquals( - "Set properties should be a subset of returned properties", - initialProperties.entrySet(), - Sets.intersection(initialProperties.entrySet(), actualProperties.entrySet())); + Assertions.assertThat(actualProperties.entrySet()) + .as("Set properties should be a subset of returned properties") + .containsAll(initialProperties.entrySet()); Map updatedProperties = ImmutableMap.of("owner", "newuser", "last-modified-at", "now"); @@ -298,26 +293,25 @@ public void testUpdateAndSetNamespaceProperties() { catalog.setProperties(NS, updatedProperties); Map finalProperties = catalog.loadNamespaceMetadata(NS); - Assert.assertEquals( - "Updated properties should be a subset of returned properties", - updatedProperties.entrySet(), - Sets.intersection(updatedProperties.entrySet(), finalProperties.entrySet())); + Assertions.assertThat(finalProperties.entrySet()) + .as("Updated properties should be a subset of returned properties") + .containsAll(updatedProperties.entrySet()); } @Test public void testSetNamespacePropertiesNamespaceDoesNotExist() { - Assume.assumeTrue(supportsNamespaceProperties()); + Assumptions.assumeTrue(supportsNamespaceProperties()); C catalog = catalog(); Assertions.assertThatThrownBy(() -> catalog.setProperties(NS, ImmutableMap.of("test", "value"))) .isInstanceOf(NoSuchNamespaceException.class) - .hasMessage("Namespace does not exist: newdb"); + .hasMessageStartingWith("Namespace does not exist: newdb"); } @Test public void testRemoveNamespaceProperties() { - Assume.assumeTrue(supportsNamespaceProperties()); + Assumptions.assumeTrue(supportsNamespaceProperties()); C catalog = catalog(); @@ -328,45 +322,46 @@ public void testRemoveNamespaceProperties() { catalog.removeProperties(NS, ImmutableSet.of("created-at")); Map actualProperties = catalog.loadNamespaceMetadata(NS); - Assert.assertFalse( - "Should not contain deleted property key", actualProperties.containsKey("created-at")); - Assert.assertEquals( - "Expected properties should be a subset of returned properties", - ImmutableMap.of("owner", "user").entrySet(), - Sets.intersection(properties.entrySet(), actualProperties.entrySet())); + Assertions.assertThat(actualProperties.containsKey("created-at")) + .as("Should not contain deleted property key") + .isFalse(); + Assertions.assertThat(Sets.intersection(properties.entrySet(), actualProperties.entrySet())) + .as("Expected properties should be a subset of returned properties") + .containsExactlyInAnyOrderElementsOf(ImmutableMap.of("owner", "user").entrySet()); } @Test public void testRemoveNamespacePropertiesNamespaceDoesNotExist() { - Assume.assumeTrue(supportsNamespaceProperties()); + Assumptions.assumeTrue(supportsNamespaceProperties()); C catalog = catalog(); Assertions.assertThatThrownBy(() -> catalog.removeProperties(NS, ImmutableSet.of("a", "b"))) .isInstanceOf(NoSuchNamespaceException.class) - .hasMessage("Namespace does not exist: newdb"); + .hasMessageStartingWith("Namespace does not exist: newdb"); } @Test public void testDropNamespace() { C catalog = catalog(); - Assert.assertFalse("Namespace should not exist", catalog.namespaceExists(NS)); + Assertions.assertThat(catalog.namespaceExists(NS)).as("Namespace should not exist").isFalse(); catalog.createNamespace(NS); - Assert.assertTrue("Namespace should exist", catalog.namespaceExists(NS)); - - Assert.assertTrue( - "Dropping an existing namespace should return true", catalog.dropNamespace(NS)); - Assert.assertFalse("Namespace should not exist", catalog.namespaceExists(NS)); + Assertions.assertThat(catalog.namespaceExists(NS)).as("Namespace should exist").isTrue(); + Assertions.assertThat(catalog.dropNamespace(NS)) + .as("Dropping an existing namespace should return true") + .isTrue(); + Assertions.assertThat(catalog.namespaceExists(NS)).as("Namespace should not exist").isFalse(); } @Test public void testDropNonexistentNamespace() { C catalog = catalog(); - Assert.assertFalse( - "Dropping a nonexistent namespace should return false", catalog.dropNamespace(NS)); + Assertions.assertThat(catalog.dropNamespace(NS)) + .as("Dropping a nonexistent namespace should return false") + .isFalse(); } @Test @@ -394,14 +389,15 @@ public void testListNamespaces() { .hasSameElementsAs(concat(starting, ns2)); catalog.dropNamespace(ns2); - Assert.assertTrue( - "Should include only starting namespaces", catalog.listNamespaces().containsAll(starting)); + Assertions.assertThat(catalog.listNamespaces().containsAll(starting)) + .as("Should include only starting namespaces") + .isTrue(); } @Test public void testListNestedNamespaces() { - Assume.assumeTrue( - "Only valid when the catalog supports nested namespaces", supportsNestedNamespaces()); + Assumptions.assumeTrue( + supportsNestedNamespaces(), "Only valid when the catalog supports nested namespaces"); C catalog = catalog(); @@ -448,22 +444,27 @@ public void testListNestedNamespaces() { @Test public void testNamespaceWithSlash() { - Assume.assumeTrue(supportsNamesWithSlashes()); + Assumptions.assumeTrue(supportsNamesWithSlashes()); C catalog = catalog(); Namespace withSlash = Namespace.of("new/db"); - Assert.assertFalse("Namespace should not exist", catalog.namespaceExists(withSlash)); + Assertions.assertThat(catalog.namespaceExists(withSlash)) + .as("Namespace should not exist") + .isFalse(); catalog.createNamespace(withSlash); - Assert.assertTrue("Namespace should exist", catalog.namespaceExists(withSlash)); + Assertions.assertThat(catalog.namespaceExists(withSlash)).as("Namespace should exist").isTrue(); Map properties = catalog.loadNamespaceMetadata(withSlash); - Assert.assertNotNull("Properties should be accessible", properties); - - Assert.assertTrue("Dropping the namespace should succeed", catalog.dropNamespace(withSlash)); - Assert.assertFalse("Namespace should not exist", catalog.namespaceExists(withSlash)); + Assertions.assertThat(properties).as("Properties should be accessible").isNotNull(); + Assertions.assertThat(catalog.dropNamespace(withSlash)) + .as("Dropping the namespace should succeed") + .isTrue(); + Assertions.assertThat(catalog.namespaceExists(withSlash)) + .as("Namespace should not exist") + .isFalse(); } @Test @@ -472,16 +473,21 @@ public void testNamespaceWithDot() { Namespace withDot = Namespace.of("new.db"); - Assert.assertFalse("Namespace should not exist", catalog.namespaceExists(withDot)); + Assertions.assertThat(catalog.namespaceExists(withDot)) + .as("Namespace should not exist") + .isFalse(); catalog.createNamespace(withDot); - Assert.assertTrue("Namespace should exist", catalog.namespaceExists(withDot)); + Assertions.assertThat(catalog.namespaceExists(withDot)).as("Namespace should exist").isTrue(); Map properties = catalog.loadNamespaceMetadata(withDot); - Assert.assertNotNull("Properties should be accessible", properties); - - Assert.assertTrue("Dropping the namespace should succeed", catalog.dropNamespace(withDot)); - Assert.assertFalse("Namespace should not exist", catalog.namespaceExists(withDot)); + Assertions.assertThat(properties).as("Properties should be accessible").isNotNull(); + Assertions.assertThat(catalog.dropNamespace(withDot)) + .as("Dropping the namespace should succeed") + .isTrue(); + Assertions.assertThat(catalog.namespaceExists(withDot)) + .as("Namespace should not exist") + .isFalse(); } @Test @@ -490,31 +496,31 @@ public void testBasicCreateTable() { TableIdentifier ident = TableIdentifier.of("ns", "table"); - Assert.assertFalse("Table should not exist", catalog.tableExists(ident)); + Assertions.assertThat(catalog.tableExists(ident)).as("Table should not exist").isFalse(); if (requiresNamespaceCreate()) { catalog.createNamespace(ident.namespace()); } Table table = catalog.buildTable(ident, SCHEMA).create(); - Assert.assertTrue("Table should exist", catalog.tableExists(ident)); + Assertions.assertThat(catalog.tableExists(ident)).as("Table should exist").isTrue(); // validate table settings - Assert.assertEquals( - "Table name should report its full name", catalog.name() + "." + ident, table.name()); - Assert.assertEquals( - "Schema should match expected ID assignment", - TABLE_SCHEMA.asStruct(), - table.schema().asStruct()); - Assert.assertNotNull("Should have a location", table.location()); - Assert.assertTrue("Should be unpartitioned", table.spec().isUnpartitioned()); - Assert.assertTrue("Should be unsorted", table.sortOrder().isUnsorted()); - Assert.assertNotNull("Should have table properties", table.properties()); + Assertions.assertThat(table.name()) + .as("Table name should report its full name") + .isEqualTo(catalog.name() + "." + ident); + Assertions.assertThat(table.schema().asStruct()) + .as("Schema should match expected ID assignment") + .isEqualTo(TABLE_SCHEMA.asStruct()); + Assertions.assertThat(table.location()).as("Should have a location").isNotNull(); + Assertions.assertThat(table.spec().isUnpartitioned()).as("Should be unpartitioned").isTrue(); + Assertions.assertThat(table.sortOrder().isUnsorted()).as("Should be unsorted").isTrue(); + Assertions.assertThat(table.properties()).as("Should have table properties").isNotNull(); } @Test public void testTableNameWithSlash() { - Assume.assumeTrue(supportsNamesWithSlashes()); + Assumptions.assumeTrue(supportsNamesWithSlashes()); C catalog = catalog(); @@ -523,20 +529,19 @@ public void testTableNameWithSlash() { catalog.createNamespace(Namespace.of("ns")); } - Assert.assertFalse("Table should not exist", catalog.tableExists(ident)); + Assertions.assertThat(catalog.tableExists(ident)).as("Table should not exist").isFalse(); catalog.buildTable(ident, SCHEMA).create(); - Assert.assertTrue("Table should exist", catalog.tableExists(ident)); + Assertions.assertThat(catalog.tableExists(ident)).as("Table should exist").isTrue(); Table loaded = catalog.loadTable(ident); - Assert.assertEquals( - "Schema should match expected ID assignment", - TABLE_SCHEMA.asStruct(), - loaded.schema().asStruct()); + Assertions.assertThat(loaded.schema().asStruct()) + .as("Schema should match expected ID assignment") + .isEqualTo(TABLE_SCHEMA.asStruct()); catalog.dropTable(ident); - Assert.assertFalse("Table should not exist", catalog.tableExists(ident)); + Assertions.assertThat(catalog.tableExists(ident)).as("Table should not exist").isFalse(); } @Test @@ -548,20 +553,19 @@ public void testTableNameWithDot() { catalog.createNamespace(Namespace.of("ns")); } - Assert.assertFalse("Table should not exist", catalog.tableExists(ident)); + Assertions.assertThat(catalog.tableExists(ident)).as("Table should not exist").isFalse(); catalog.buildTable(ident, SCHEMA).create(); - Assert.assertTrue("Table should exist", catalog.tableExists(ident)); + Assertions.assertThat(catalog.tableExists(ident)).as("Table should exist").isTrue(); Table loaded = catalog.loadTable(ident); - Assert.assertEquals( - "Schema should match expected ID assignment", - TABLE_SCHEMA.asStruct(), - loaded.schema().asStruct()); + Assertions.assertThat(loaded.schema().asStruct()) + .as("Schema should match expected ID assignment") + .isEqualTo(TABLE_SCHEMA.asStruct()); catalog.dropTable(ident); - Assert.assertFalse("Table should not exist", catalog.tableExists(ident)); + Assertions.assertThat(catalog.tableExists(ident)).as("Table should not exist").isFalse(); } @Test @@ -574,20 +578,19 @@ public void testBasicCreateTableThatAlreadyExists() { catalog.createNamespace(ident.namespace()); } - Assert.assertFalse("Table should not exist", catalog.tableExists(ident)); + Assertions.assertThat(catalog.tableExists(ident)).as("Table should not exist").isFalse(); catalog.buildTable(ident, SCHEMA).create(); - Assert.assertTrue("Table should exist", catalog.tableExists(ident)); + Assertions.assertThat(catalog.tableExists(ident)).as("Table should exist").isTrue(); Assertions.assertThatThrownBy(() -> catalog.buildTable(ident, OTHER_SCHEMA).create()) .isInstanceOf(AlreadyExistsException.class) - .hasMessage("Table already exists: ns.table"); + .hasMessageStartingWith("Table already exists: ns.table"); Table table = catalog.loadTable(ident); - Assert.assertEquals( - "Schema should match original table schema", - TABLE_SCHEMA.asStruct(), - table.schema().asStruct()); + Assertions.assertThat(table.schema().asStruct()) + .as("Schema should match original table schema") + .isEqualTo(TABLE_SCHEMA.asStruct()); } @Test @@ -600,7 +603,7 @@ public void testCompleteCreateTable() { catalog.createNamespace(ident.namespace()); } - Assert.assertFalse("Table should not exist", catalog.tableExists(ident)); + Assertions.assertThat(catalog.tableExists(ident)).as("Table should not exist").isFalse(); Map properties = ImmutableMap.of("user", "someone", "created-at", "2022-02-25T00:38:19"); @@ -614,20 +617,23 @@ public void testCompleteCreateTable() { .create(); // validate table settings - Assert.assertEquals( - "Table name should report its full name", catalog.name() + "." + ident, table.name()); - Assert.assertTrue("Table should exist", catalog.tableExists(ident)); - Assert.assertEquals( - "Schema should match expected ID assignment", - TABLE_SCHEMA.asStruct(), - table.schema().asStruct()); - Assert.assertNotNull("Should have a location", table.location()); - Assert.assertEquals("Should use requested partition spec", TABLE_SPEC, table.spec()); - Assert.assertEquals("Should use requested write order", TABLE_WRITE_ORDER, table.sortOrder()); - Assert.assertEquals( - "Table properties should be a superset of the requested properties", - properties.entrySet(), - Sets.intersection(properties.entrySet(), table.properties().entrySet())); + Assertions.assertThat(table.name()) + .as("Table name should report its full name") + .isEqualTo(catalog.name() + "." + ident); + Assertions.assertThat(catalog.tableExists(ident)).as("Table should exist").isTrue(); + Assertions.assertThat(table.schema().asStruct()) + .as("Schema should match expected ID assignment") + .isEqualTo(TABLE_SCHEMA.asStruct()); + Assertions.assertThat(table.location()).as("Should have a location").isNotNull(); + Assertions.assertThat(table.spec()) + .as("Should use requested partition spec") + .isEqualTo(TABLE_SPEC); + Assertions.assertThat(table.sortOrder()) + .as("Should use requested write order") + .isEqualTo(TABLE_WRITE_ORDER); + Assertions.assertThat(table.properties().entrySet()) + .as("Table properties should be a superset of the requested properties") + .containsAll(properties.entrySet()); } @Test @@ -640,7 +646,7 @@ public void testLoadTable() { catalog.createNamespace(ident.namespace()); } - Assert.assertFalse("Table should not exist", catalog.tableExists(ident)); + Assertions.assertThat(catalog.tableExists(ident)).as("Table should not exist").isFalse(); Map properties = ImmutableMap.of("user", "someone", "created-at", "2022-02-25T00:38:19"); @@ -651,24 +657,27 @@ public void testLoadTable() { .withSortOrder(WRITE_ORDER) .withProperties(properties) .create(); - Assert.assertTrue("Table should exist", catalog.tableExists(ident)); + Assertions.assertThat(catalog.tableExists(ident)).as("Table should exist").isTrue(); Table table = catalog.loadTable(ident); // validate table settings - Assert.assertEquals( - "Table name should report its full name", catalog.name() + "." + ident, table.name()); - Assert.assertTrue("Table should exist", catalog.tableExists(ident)); - Assert.assertEquals( - "Schema should match expected ID assignment", - TABLE_SCHEMA.asStruct(), - table.schema().asStruct()); - Assert.assertNotNull("Should have a location", table.location()); - Assert.assertEquals("Should use requested partition spec", TABLE_SPEC, table.spec()); - Assert.assertEquals("Should use requested write order", TABLE_WRITE_ORDER, table.sortOrder()); - Assert.assertEquals( - "Table properties should be a superset of the requested properties", - properties.entrySet(), - Sets.intersection(properties.entrySet(), table.properties().entrySet())); + Assertions.assertThat(table.name()) + .as("Table name should report its full name") + .isEqualTo(catalog.name() + "." + ident); + Assertions.assertThat(catalog.tableExists(ident)).as("Table should exist").isTrue(); + Assertions.assertThat(table.schema().asStruct()) + .as("Schema should match expected ID assignment") + .isEqualTo(TABLE_SCHEMA.asStruct()); + Assertions.assertThat(table.location()).as("Should have a location").isNotNull(); + Assertions.assertThat(table.spec()) + .as("Should use requested partition spec") + .isEqualTo(TABLE_SPEC); + Assertions.assertThat(table.sortOrder()) + .as("Should use requested write order") + .isEqualTo(TABLE_WRITE_ORDER); + Assertions.assertThat(table.properties().entrySet()) + .as("Table properties should be a superset of the requested properties") + .containsAll(properties.entrySet()); } @Test @@ -700,10 +709,10 @@ public void testLoadMissingTable() { TableIdentifier ident = TableIdentifier.of("ns", "table"); - Assert.assertFalse("Table should not exist", catalog.tableExists(ident)); + Assertions.assertThat(catalog.tableExists(ident)).as("Table should not exist").isFalse(); Assertions.assertThatThrownBy(() -> catalog.loadTable(ident)) .isInstanceOf(NoSuchTableException.class) - .hasMessage("Table does not exist: ns.table"); + .hasMessageStartingWith("Table does not exist: ns.table"); } @Test @@ -714,17 +723,26 @@ public void testRenameTable() { catalog.createNamespace(NS); } - Assert.assertFalse("Source table should not exist before create", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Source table should not exist before create") + .isFalse(); catalog.buildTable(TABLE, SCHEMA).create(); - Assert.assertTrue("Table should exist after create", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should exist after create") + .isTrue(); - Assert.assertFalse( - "Destination table should not exist before rename", catalog.tableExists(RENAMED_TABLE)); + Assertions.assertThat(catalog.tableExists(RENAMED_TABLE)) + .as("Destination table should not exist before rename") + .isFalse(); catalog.renameTable(TABLE, RENAMED_TABLE); - Assert.assertTrue("Table should exist with new name", catalog.tableExists(RENAMED_TABLE)); - Assert.assertFalse("Original table should no longer exist", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(RENAMED_TABLE)) + .as("Table should exist with new name") + .isTrue(); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Original table should no longer exist") + .isFalse(); catalog.dropTable(RENAMED_TABLE); assertEmpty("Should not contain table after drop", catalog, NS); @@ -738,17 +756,20 @@ public void testRenameTableMissingSourceTable() { catalog.createNamespace(NS); } - Assert.assertFalse("Source table should not exist before rename", catalog.tableExists(TABLE)); - Assert.assertFalse( - "Destination table should not exist before rename", catalog.tableExists(RENAMED_TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Source table should not exist before rename") + .isFalse(); + Assertions.assertThat(catalog.tableExists(RENAMED_TABLE)) + .as("Destination table should not exist before rename") + .isFalse(); Assertions.assertThatThrownBy(() -> catalog.renameTable(TABLE, RENAMED_TABLE)) .isInstanceOf(NoSuchTableException.class) .hasMessageContaining("Table does not exist"); - Assert.assertFalse( - "Destination table should not exist after failed rename", - catalog.tableExists(RENAMED_TABLE)); + Assertions.assertThat(catalog.tableExists(RENAMED_TABLE)) + .as("Destination table should not exist after failed rename") + .isFalse(); } @Test @@ -759,34 +780,40 @@ public void testRenameTableDestinationTableAlreadyExists() { catalog.createNamespace(NS); } - Assert.assertFalse("Source table should not exist before create", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Source table should not exist before create") + .isFalse(); + catalog.buildTable(TABLE, SCHEMA).create(); - Assert.assertTrue("Source table should exist after create", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Source table should exist after create") + .isTrue(); - Assert.assertFalse( - "Destination table should not exist before create", catalog.tableExists(RENAMED_TABLE)); - catalog.buildTable(RENAMED_TABLE, SCHEMA).create(); - Assert.assertTrue( - "Destination table should exist after create", catalog.tableExists(RENAMED_TABLE)); + Assertions.assertThat(catalog.tableExists(RENAMED_TABLE)) + .as("Destination table should not exist before create") + .isFalse(); + catalog.buildTable(RENAMED_TABLE, SCHEMA).create(); + Assertions.assertThat(catalog.tableExists(RENAMED_TABLE)) + .as("Destination table should exist after create") + .isTrue(); Assertions.assertThatThrownBy(() -> catalog.renameTable(TABLE, RENAMED_TABLE)) .isInstanceOf(AlreadyExistsException.class) .hasMessageContaining("Table already exists"); - - Assert.assertTrue( - "Source table should still exist after failed rename", catalog.tableExists(TABLE)); - Assert.assertTrue( - "Destination table should still exist after failed rename", - catalog.tableExists(RENAMED_TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Source table should still exist after failed rename") + .isTrue(); + Assertions.assertThat(catalog.tableExists(RENAMED_TABLE)) + .as("Destination table should still exist after failed rename") + .isTrue(); String sourceTableUUID = ((HasTableOperations) catalog.loadTable(TABLE)).operations().current().uuid(); String destinationTableUUID = ((HasTableOperations) catalog.loadTable(RENAMED_TABLE)).operations().current().uuid(); - Assert.assertNotEquals( - "Source and destination table should remain distinct after failed rename", - sourceTableUUID, - destinationTableUUID); + Assertions.assertThat(sourceTableUUID) + .as("Source and destination table should remain distinct after failed rename") + .isNotEqualTo(destinationTableUUID); } @Test @@ -797,14 +824,20 @@ public void testDropTable() { catalog.createNamespace(NS); } - Assert.assertFalse("Table should not exist before create", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should not exist before create") + .isFalse(); catalog.buildTable(TABLE, SCHEMA).create(); - Assert.assertTrue("Table should exist after create", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should exist after create") + .isTrue(); boolean dropped = catalog.dropTable(TABLE); - Assert.assertTrue("Should drop a table that does exist", dropped); - Assert.assertFalse("Table should not exist after drop", catalog.tableExists(TABLE)); + Assertions.assertThat(dropped).as("Should drop a table that does exist").isTrue(); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should not exist after drop") + .isFalse(); } @Test @@ -815,14 +848,20 @@ public void testDropTableWithPurge() { catalog.createNamespace(NS); } - Assert.assertFalse("Table should not exist before create", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should not exist before create") + .isFalse(); catalog.buildTable(TABLE, SCHEMA).create(); - Assert.assertTrue("Table should exist after create", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should exist after create") + .isTrue(); boolean dropped = catalog.dropTable(TABLE, true); - Assert.assertTrue("Should drop a table that does exist", dropped); - Assert.assertFalse("Table should not exist after drop", catalog.tableExists(TABLE)); + Assertions.assertThat(dropped).as("Should drop a table that does exist").isTrue(); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should not exist after drop") + .isFalse(); } @Test @@ -833,15 +872,21 @@ public void testDropTableWithoutPurge() { catalog.createNamespace(NS); } - Assert.assertFalse("Table should not exist before create", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should not exist before create") + .isFalse(); Table table = catalog.buildTable(TABLE, SCHEMA).create(); - Assert.assertTrue("Table should exist after create", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should exist after create") + .isTrue(); Set actualMetadataFileLocations = ReachableFileUtil.metadataFileLocations(table, false); boolean dropped = catalog.dropTable(TABLE, false); - Assert.assertTrue("Should drop a table that does exist", dropped); - Assert.assertFalse("Table should not exist after drop", catalog.tableExists(TABLE)); + Assertions.assertThat(dropped).as("Should drop a table that does exist").isTrue(); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should not exist after drop") + .isFalse(); Set expectedMetadataFileLocations = ReachableFileUtil.metadataFileLocations(table, false); Assertions.assertThat(actualMetadataFileLocations) @@ -859,9 +904,12 @@ public void testDropMissingTable() { } TableIdentifier noSuchTableIdent = TableIdentifier.of(NS, "notable"); - Assert.assertFalse("Table should not exist", catalog.tableExists(noSuchTableIdent)); - Assert.assertFalse( - "Should not drop a table that does not exist", catalog.dropTable(noSuchTableIdent)); + Assertions.assertThat(catalog.tableExists(noSuchTableIdent)) + .as("Table should not exist") + .isFalse(); + Assertions.assertThat(catalog.dropTable(noSuchTableIdent)) + .as("Should not drop a table that does not exist") + .isFalse(); } @Test @@ -885,50 +933,43 @@ public void testListTables() { catalog.buildTable(ns1Table1, SCHEMA).create(); - Assert.assertEquals( - "Should contain ns_1.table_1 after create", - ImmutableSet.of(ns1Table1), - Sets.newHashSet(catalog.listTables(ns1))); + Assertions.assertThat(catalog.listTables(ns1)) + .as("Should contain ns_1.table_1 after create") + .containsExactlyInAnyOrder(ns1Table1); catalog.buildTable(ns2Table1, SCHEMA).create(); - Assert.assertEquals( - "Should contain ns_2.table_1 after create", - ImmutableSet.of(ns2Table1), - Sets.newHashSet(catalog.listTables(ns2))); - Assert.assertEquals( - "Should not show changes to ns_2 in ns_1", - ImmutableSet.of(ns1Table1), - Sets.newHashSet(catalog.listTables(ns1))); + Assertions.assertThat(catalog.listTables(ns2)) + .as("Should contain ns_2.table_1 after create") + .containsExactlyInAnyOrder(ns2Table1); + Assertions.assertThat(catalog.listTables(ns1)) + .as("Should not show changes to ns_2 in ns_1") + .containsExactlyInAnyOrder(ns1Table1); catalog.buildTable(ns1Table2, SCHEMA).create(); - Assert.assertEquals( - "Should not show changes to ns_1 in ns_2", - ImmutableSet.of(ns2Table1), - Sets.newHashSet(catalog.listTables(ns2))); - Assert.assertEquals( - "Should contain ns_1.table_2 after create", - ImmutableSet.of(ns1Table1, ns1Table2), - Sets.newHashSet(catalog.listTables(ns1))); + Assertions.assertThat(catalog.listTables(ns2)) + .as("Should not show changes to ns_1 in ns_2") + .containsExactlyInAnyOrder(ns2Table1); + Assertions.assertThat(catalog.listTables(ns1)) + .as("Should contain ns_1.table_2 after create") + .containsExactlyInAnyOrder(ns1Table1, ns1Table2); catalog.dropTable(ns1Table1); - Assert.assertEquals( - "Should not show changes to ns_1 in ns_2", - ImmutableSet.of(ns2Table1), - Sets.newHashSet(catalog.listTables(ns2))); - Assert.assertEquals( - "Should not contain ns_1.table_1 after drop", - ImmutableSet.of(ns1Table2), - Sets.newHashSet(catalog.listTables(ns1))); + Assertions.assertThat(catalog.listTables(ns2)) + .as("Should not show changes to ns_1 in ns_2") + .containsExactlyInAnyOrder(ns2Table1); + Assertions.assertThat(catalog.listTables(ns1)) + .as("Should not contain ns_1.table_1 after drop") + .containsExactlyInAnyOrder(ns1Table2); catalog.dropTable(ns1Table2); - Assert.assertEquals( - "Should not show changes to ns_1 in ns_2", - ImmutableSet.of(ns2Table1), - Sets.newHashSet(catalog.listTables(ns2))); + Assertions.assertThat(catalog.listTables(ns2)) + .as("Should not show changes to ns_1 in ns_2") + .containsExactlyInAnyOrder(ns2Table1); + assertEmpty("Should not contain ns_1.table_2 after drop", catalog, ns1); catalog.dropTable(ns2Table1); @@ -952,10 +993,9 @@ public void testUpdateTableSchema() { Table loaded = catalog.loadTable(TABLE); - Assert.assertEquals( - "Loaded table should have expected schema", - expected.asStruct(), - loaded.schema().asStruct()); + Assertions.assertThat(loaded.schema().asStruct()) + .as("Loaded table should have expected schema") + .isEqualTo(expected.asStruct()); } @Test @@ -969,7 +1009,7 @@ public void testUUIDValidation() { Table table = catalog.buildTable(TABLE, SCHEMA).create(); UpdateSchema update = table.updateSchema().addColumn("new_col", Types.LongType.get()); - Assert.assertTrue("Should successfully drop table", catalog.dropTable(TABLE)); + Assertions.assertThat(catalog.dropTable(TABLE)).as("Should successfully drop table").isTrue(); catalog.buildTable(TABLE, OTHER_SCHEMA).create(); String expectedMessage = @@ -979,17 +1019,16 @@ public void testUUIDValidation() { .hasMessageContaining(expectedMessage); Table loaded = catalog.loadTable(TABLE); - Assert.assertEquals( - "Loaded table should have expected schema", - OTHER_SCHEMA.asStruct(), - loaded.schema().asStruct()); + Assertions.assertThat(loaded.schema().asStruct()) + .as("Loaded table should have expected schema") + .isEqualTo(OTHER_SCHEMA.asStruct()); } @Test public void testUpdateTableSchemaServerSideRetry() { - Assume.assumeTrue( - "Schema update recovery is only supported with server-side retry", - supportsServerSideRetry()); + Assumptions.assumeTrue( + supportsServerSideRetry(), + "Schema update recovery is only supported with server-side retry"); C catalog = catalog(); if (requiresNamespaceCreate()) { @@ -1008,10 +1047,9 @@ public void testUpdateTableSchemaServerSideRetry() { update.commit(); Table loaded = catalog.loadTable(TABLE); - Assert.assertEquals( - "Loaded table should have expected schema", - expected.asStruct(), - loaded.schema().asStruct()); + Assertions.assertThat(loaded.schema().asStruct()) + .as("Loaded table should have expected schema") + .isEqualTo(expected.asStruct()); } @Test @@ -1039,10 +1077,9 @@ public void testUpdateTableSchemaConflict() { .hasMessageContaining(expectedMessage); Table loaded = catalog.loadTable(TABLE); - Assert.assertEquals( - "Loaded table should have expected schema", - expected.asStruct(), - loaded.schema().asStruct()); + Assertions.assertThat(loaded.schema().asStruct()) + .as("Loaded table should have expected schema") + .isEqualTo(expected.asStruct()); } @Test @@ -1073,10 +1110,9 @@ public void testUpdateTableSchemaAssignmentConflict() { .hasMessageContaining(expectedMessage); Table loaded = catalog.loadTable(TABLE); - Assert.assertEquals( - "Loaded table should have expected schema", - expected.asStruct(), - loaded.schema().asStruct()); + Assertions.assertThat(loaded.schema().asStruct()) + .as("Loaded table should have expected schema") + .isEqualTo(expected.asStruct()); } @Test @@ -1098,10 +1134,9 @@ public void testUpdateTableSchemaThenRevert() { table.updateSchema().deleteColumn("col1").deleteColumn("col2").deleteColumn("col3").commit(); - Assert.assertEquals( - "Loaded table should have expected schema", - TABLE_SCHEMA.asStruct(), - table.schema().asStruct()); + Assertions.assertThat(table.schema().asStruct()) + .as("Loaded table should have expected schema") + .isEqualTo(TABLE_SCHEMA.asStruct()); } @Test @@ -1122,14 +1157,15 @@ public void testUpdateTableSpec() { Table loaded = catalog.loadTable(TABLE); // the spec ID may not match, so check equality of the fields - Assert.assertEquals( - "Loaded table should have expected spec", expected.fields(), loaded.spec().fields()); + Assertions.assertThat(loaded.spec().fields()) + .as("Loaded table should have expected spec") + .isEqualTo(expected.fields()); } @Test public void testUpdateTableSpecServerSideRetry() { - Assume.assumeTrue( - "Spec update recovery is only supported with server-side retry", supportsServerSideRetry()); + Assumptions.assumeTrue( + supportsServerSideRetry(), "Spec update recovery is only supported with server-side retry"); C catalog = catalog(); if (requiresNamespaceCreate()) { @@ -1154,8 +1190,9 @@ public void testUpdateTableSpecServerSideRetry() { Table loaded = catalog.loadTable(TABLE); // the spec ID may not match, so check equality of the fields - Assert.assertEquals( - "Loaded table should have expected spec", expected.fields(), loaded.spec().fields()); + Assertions.assertThat(loaded.spec().fields()) + .as("Loaded table should have expected spec") + .isEqualTo(expected.fields()); } @Test @@ -1189,8 +1226,9 @@ public void testUpdateTableSpecConflict() { Table loaded = catalog.loadTable(TABLE); // the spec ID may not match, so check equality of the fields - Assert.assertEquals( - "Loaded table should have expected spec", expected.fields(), loaded.spec().fields()); + Assertions.assertThat(loaded.spec().fields()) + .as("Loaded table should have expected spec") + .isEqualTo(expected.fields()); } @Test @@ -1223,8 +1261,9 @@ public void testUpdateTableAssignmentSpecConflict() { Table loaded = catalog.loadTable(TABLE); // the spec ID may not match, so check equality of the fields - Assert.assertEquals( - "Loaded table should have expected spec", expected.fields(), loaded.spec().fields()); + Assertions.assertThat(loaded.spec().fields()) + .as("Loaded table should have expected spec") + .isEqualTo(expected.fields()); } @Test @@ -1243,14 +1282,17 @@ public void testUpdateTableSpecThenRevert() { .withPartitionSpec(SPEC) .withProperty("format-version", "2") .create(); - Assert.assertEquals( - "Should be a v2 table", 2, ((BaseTable) table).operations().current().formatVersion()); + Assertions.assertThat(((BaseTable) table).operations().current().formatVersion()) + .as("Should be a v2 table") + .isEqualTo(2); table.updateSpec().addField("id").commit(); table.updateSpec().removeField("id").commit(); - Assert.assertEquals("Loaded table should have expected spec", TABLE_SPEC, table.spec()); + Assertions.assertThat(table.spec()) + .as("Loaded table should have expected spec") + .isEqualTo(TABLE_SPEC); } @Test @@ -1271,15 +1313,16 @@ public void testUpdateTableSortOrder() { Table loaded = catalog.loadTable(TABLE); // the sort order ID may not match, so check equality of the fields - Assert.assertEquals( - "Loaded table should have expected order", expected.fields(), loaded.sortOrder().fields()); + Assertions.assertThat(loaded.sortOrder().fields()) + .as("Loaded table should have expected order") + .isEqualTo(expected.fields()); } @Test public void testUpdateTableSortOrderServerSideRetry() { - Assume.assumeTrue( - "Sort order update recovery is only supported with server-side retry", - supportsServerSideRetry()); + Assumptions.assumeTrue( + supportsServerSideRetry(), + "Sort order update recovery is only supported with server-side retry"); C catalog = catalog(); if (requiresNamespaceCreate()) { @@ -1304,8 +1347,9 @@ public void testUpdateTableSortOrderServerSideRetry() { Table loaded = catalog.loadTable(TABLE); // the sort order ID may not match, so check equality of the fields - Assert.assertEquals( - "Loaded table should have expected order", expected.fields(), loaded.sortOrder().fields()); + Assertions.assertThat(loaded.sortOrder().fields()) + .as("Loaded table should have expected order") + .isEqualTo(expected.fields()); } @Test @@ -1322,8 +1366,9 @@ public void testUpdateTableOrderThenRevert() { table.replaceSortOrder().asc(Expressions.bucket("id", 16)).asc("id").commit(); - Assert.assertEquals( - "Loaded table should have expected order", TABLE_WRITE_ORDER, table.sortOrder()); + Assertions.assertThat(table.sortOrder()) + .as("Loaded table should have expected order") + .isEqualTo(TABLE_WRITE_ORDER); } @Test @@ -1337,7 +1382,7 @@ public void testAppend() throws IOException { Table table = catalog.buildTable(TABLE, SCHEMA).withPartitionSpec(SPEC).create(); try (CloseableIterable tasks = table.newScan().planFiles()) { - Assert.assertFalse("Should contain no files", tasks.iterator().hasNext()); + Assertions.assertThat(tasks.iterator().hasNext()).as("Should contain no files").isFalse(); } table.newFastAppend().appendFile(FILE_A).commit(); @@ -1424,12 +1469,12 @@ public void testUpdateTransaction() { Table loaded = catalog.loadTable(TABLE); - Assert.assertEquals( - "Loaded table should have expected schema", - expectedSchema.asStruct(), - loaded.schema().asStruct()); - Assert.assertEquals( - "Loaded table should have expected spec", expectedSpec.fields(), loaded.spec().fields()); + Assertions.assertThat(loaded.schema().asStruct()) + .as("Loaded table should have expected schema") + .isEqualTo(expectedSchema.asStruct()); + Assertions.assertThat(loaded.spec().fields()) + .as("Loaded table should have expected spec") + .isEqualTo(expectedSpec.fields()); assertPreviousMetadataFileCount(loaded, 1); } @@ -1444,16 +1489,21 @@ public void testCreateTransaction() { Transaction create = catalog.buildTable(TABLE, SCHEMA).createTransaction(); - Assert.assertFalse( - "Table should not exist after createTransaction", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should not exist after createTransaction") + .isFalse(); create.newFastAppend().appendFile(FILE_A).commit(); - Assert.assertFalse("Table should not exist after append commit", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should not exist after append commit") + .isFalse(); create.commitTransaction(); - Assert.assertTrue("Table should exist after append commit", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should exist after append commit") + .isTrue(); Table table = catalog.loadTable(TABLE); assertFiles(table, FILE_A); assertPreviousMetadataFileCount(table, 0); @@ -1478,33 +1528,39 @@ public void testCompleteCreateTransaction() { .withProperties(properties) .createTransaction(); - Assert.assertFalse( - "Table should not exist after createTransaction", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should not exist after createTransaction") + .isFalse(); create.newFastAppend().appendFile(FILE_A).commit(); - Assert.assertFalse("Table should not exist after append commit", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should not exist after append commit") + .isFalse(); create.commitTransaction(); - Assert.assertTrue("Table should exist after append commit", catalog.tableExists(TABLE)); - Table table = catalog.loadTable(TABLE); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should exist after append commit") + .isTrue(); - Assert.assertEquals( - "Table schema should match the new schema", - TABLE_SCHEMA.asStruct(), - table.schema().asStruct()); - Assert.assertEquals( - "Table should have create partition spec", TABLE_SPEC.fields(), table.spec().fields()); - Assert.assertEquals( - "Table should have create sort order", TABLE_WRITE_ORDER, table.sortOrder()); - Assert.assertEquals( - "Table properties should be a superset of the requested properties", - properties.entrySet(), - Sets.intersection(properties.entrySet(), table.properties().entrySet())); + Table table = catalog.loadTable(TABLE); + Assertions.assertThat(table.schema().asStruct()) + .as("Table schema should match the new schema") + .isEqualTo(TABLE_SCHEMA.asStruct()); + Assertions.assertThat(table.spec().fields()) + .as("Table should have create partition spec") + .isEqualTo(TABLE_SPEC.fields()); + Assertions.assertThat(table.sortOrder()) + .as("Table should have create sort order") + .isEqualTo(TABLE_WRITE_ORDER); + Assertions.assertThat(table.properties().entrySet()) + .as("Table properties should be a superset of the requested properties") + .containsAll(properties.entrySet()); if (!overridesRequestedLocation()) { - Assert.assertEquals( - "Table location should match requested", "file:/tmp/ns/table", table.location()); + Assertions.assertThat(table.location()) + .as("Table location should match requested") + .isEqualTo("file:/tmp/ns/table"); } assertFiles(table, FILE_A); assertFilesPartitionSpec(table); @@ -1530,8 +1586,9 @@ public void testCompleteCreateTransactionMultipleSchemas() { .withProperties(properties) .createTransaction(); - Assert.assertFalse( - "Table should not exist after createTransaction", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should not exist after createTransaction") + .isFalse(); create.newFastAppend().appendFile(FILE_A).commit(); @@ -1557,11 +1614,16 @@ public void testCompleteCreateTransactionMultipleSchemas() { create.newFastAppend().appendFile(anotherFile).commit(); - Assert.assertFalse("Table should not exist after append commit", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should not exist after append commit") + .isFalse(); create.commitTransaction(); - Assert.assertTrue("Table should exist after append commit", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should exist after append commit") + .isTrue(); + Table table = catalog.loadTable(TABLE); // initial IDs taken from TableMetadata constants @@ -1572,27 +1634,31 @@ public void testCompleteCreateTransactionMultipleSchemas() { final int updateSpecId = initialSpecId + 1; final int updateOrderId = initialOrderId + 1; - Assert.assertEquals( - "Table schema should match the new schema", - newSchema.asStruct(), - table.schema().asStruct()); - Assert.assertEquals( - "Table schema should match the new schema ID", updateSchemaId, table.schema().schemaId()); - Assert.assertEquals( - "Table should have updated partition spec", newSpec.fields(), table.spec().fields()); - Assert.assertEquals( - "Table should have updated partition spec ID", updateSpecId, table.spec().specId()); - Assert.assertEquals( - "Table should have updated sort order", newSortOrder.fields(), table.sortOrder().fields()); - Assert.assertEquals( - "Table should have updated sort order ID", updateOrderId, table.sortOrder().orderId()); - Assert.assertEquals( - "Table properties should be a superset of the requested properties", - properties.entrySet(), - Sets.intersection(properties.entrySet(), table.properties().entrySet())); + Assertions.assertThat(table.schema().asStruct()) + .as("Table schema should match the new schema") + .isEqualTo(newSchema.asStruct()); + Assertions.assertThat(table.schema().schemaId()) + .as("Table schema should match the new schema ID") + .isEqualTo(updateSchemaId); + Assertions.assertThat(table.spec().fields()) + .as("Table should have updated partition spec") + .isEqualTo(newSpec.fields()); + Assertions.assertThat(table.spec().specId()) + .as("Table should have updated partition spec ID") + .isEqualTo(updateSpecId); + Assertions.assertThat(table.sortOrder().fields()) + .as("Table should have updated sort order") + .isEqualTo(newSortOrder.fields()); + Assertions.assertThat(table.sortOrder().orderId()) + .as("Table should have updated sort order ID") + .isEqualTo(updateOrderId); + Assertions.assertThat(table.properties().entrySet()) + .as("Table properties should be a superset of the requested properties") + .containsAll(properties.entrySet()); if (!overridesRequestedLocation()) { - Assert.assertEquals( - "Table location should match requested", "file:/tmp/ns/table", table.location()); + Assertions.assertThat(table.location()) + .as("Table location should match requested") + .isEqualTo("file:/tmp/ns/table"); } assertFiles(table, FILE_A, anotherFile); assertFilePartitionSpec(table, FILE_A, initialSpecId); @@ -1621,41 +1687,48 @@ public void testCompleteCreateTransactionV2() { .withProperties(properties) .createTransaction(); - Assert.assertFalse( - "Table should not exist after createTransaction", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should not exist after createTransaction") + .isFalse(); create.newFastAppend().appendFile(FILE_A).commit(); - Assert.assertFalse("Table should not exist after append commit", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should not exist after append commit") + .isFalse(); create.commitTransaction(); - Assert.assertTrue("Table should exist after append commit", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should exist after append commit") + .isTrue(); Table table = catalog.loadTable(TABLE); Map expectedProps = Maps.newHashMap(properties); + expectedProps.remove("format-version"); - Assert.assertEquals( - "Table schema should match the new schema", - TABLE_SCHEMA.asStruct(), - table.schema().asStruct()); - Assert.assertEquals( - "Table should have create partition spec", TABLE_SPEC.fields(), table.spec().fields()); - Assert.assertEquals( - "Table should have create sort order", TABLE_WRITE_ORDER, table.sortOrder()); - Assert.assertEquals( - "Table properties should be a superset of the requested properties", - expectedProps.entrySet(), - Sets.intersection(properties.entrySet(), table.properties().entrySet())); - Assert.assertEquals( - "Sequence number should start at 1 for v2 format", - 1, - table.currentSnapshot().sequenceNumber()); + Assertions.assertThat(table.schema().asStruct()) + .as("Table schema should match the new schema") + .isEqualTo(TABLE_SCHEMA.asStruct()); + Assertions.assertThat(table.spec().fields()) + .as("Table should have create partition spec") + .isEqualTo(TABLE_SPEC.fields()); + Assertions.assertThat(table.sortOrder()) + .as("Table should have create sort order") + .isEqualTo(TABLE_WRITE_ORDER); + Assertions.assertThat(Sets.intersection(properties.entrySet(), table.properties().entrySet())) + .as("Table properties should be a superset of the requested properties") + .containsExactlyInAnyOrderElementsOf(expectedProps.entrySet()); + Assertions.assertThat(table.currentSnapshot().sequenceNumber()) + .as("Sequence number should start at 1 for v2 format") + .isEqualTo(1); if (!overridesRequestedLocation()) { - Assert.assertEquals( - "Table location should match requested", "file:/tmp/ns/table", table.location()); + Assertions.assertThat(table.location()) + .as("Table location should match requested") + .isEqualTo("file:/tmp/ns/table"); } + assertFiles(table, FILE_A); assertFilesPartitionSpec(table); assertPreviousMetadataFileCount(table, 0); @@ -1671,12 +1744,15 @@ public void testConcurrentCreateTransaction() { Transaction create = catalog.buildTable(TABLE, SCHEMA).createTransaction(); - Assert.assertFalse( - "Table should not exist after createTransaction", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should not exist after createTransaction") + .isFalse(); create.newFastAppend().appendFile(FILE_A).commit(); - Assert.assertFalse("Table should not exist after append commit", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should not exist after append commit") + .isFalse(); catalog.buildTable(TABLE, OTHER_SCHEMA).create(); @@ -1691,10 +1767,9 @@ public void testConcurrentCreateTransaction() { // validate the concurrently created table is unmodified Table table = catalog.loadTable(TABLE); - Assert.assertEquals( - "Table schema should match concurrent create", - OTHER_SCHEMA.asStruct(), - table.schema().asStruct()); + Assertions.assertThat(table.schema().asStruct()) + .as("Table schema should match concurrent create") + .isEqualTo(OTHER_SCHEMA.asStruct()); assertNoFiles(table); } @@ -1708,16 +1783,22 @@ public void testCreateOrReplaceTransactionCreate() { Transaction create = catalog.buildTable(TABLE, SCHEMA).createOrReplaceTransaction(); - Assert.assertFalse( - "Table should not exist after createTransaction", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should not exist after createTransaction") + .isFalse(); create.newFastAppend().appendFile(FILE_A).commit(); - Assert.assertFalse("Table should not exist after append commit", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should not exist after append commit") + .isFalse(); create.commitTransaction(); - Assert.assertTrue("Table should exist after append commit", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should exist after append commit") + .isTrue(); + Table table = catalog.loadTable(TABLE); assertFiles(table, FILE_A); assertPreviousMetadataFileCount(table, 0); @@ -1742,34 +1823,42 @@ public void testCompleteCreateOrReplaceTransactionCreate() { .withProperties(properties) .createOrReplaceTransaction(); - Assert.assertFalse( - "Table should not exist after createTransaction", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should not exist after createTransaction") + .isFalse(); createOrReplace.newFastAppend().appendFile(FILE_A).commit(); - Assert.assertFalse("Table should not exist after append commit", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should not exist after append commit") + .isFalse(); createOrReplace.commitTransaction(); - Assert.assertTrue("Table should exist after append commit", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should exist after append commit") + .isTrue(); + Table table = catalog.loadTable(TABLE); - Assert.assertEquals( - "Table schema should match the new schema", - TABLE_SCHEMA.asStruct(), - table.schema().asStruct()); - Assert.assertEquals( - "Table should have create partition spec", TABLE_SPEC.fields(), table.spec().fields()); - Assert.assertEquals( - "Table should have create sort order", TABLE_WRITE_ORDER, table.sortOrder()); - Assert.assertEquals( - "Table properties should be a superset of the requested properties", - properties.entrySet(), - Sets.intersection(properties.entrySet(), table.properties().entrySet())); + Assertions.assertThat(table.schema().asStruct()) + .as("Table schema should match the new schema") + .isEqualTo(TABLE_SCHEMA.asStruct()); + Assertions.assertThat(table.spec().fields()) + .as("Table should have create partition spec") + .isEqualTo(TABLE_SPEC.fields()); + Assertions.assertThat(table.sortOrder()) + .as("Table should have create sort order") + .isEqualTo(TABLE_WRITE_ORDER); + Assertions.assertThat(table.properties().entrySet()) + .as("Table properties should be a superset of the requested properties") + .containsAll(properties.entrySet()); if (!overridesRequestedLocation()) { - Assert.assertEquals( - "Table location should match requested", "file:/tmp/ns/table", table.location()); + Assertions.assertThat(table.location()) + .as("Table location should match requested") + .isEqualTo("file:/tmp/ns/table"); } + assertFiles(table, FILE_A); assertFilesPartitionSpec(table); assertPreviousMetadataFileCount(table, 0); @@ -1785,36 +1874,41 @@ public void testCreateOrReplaceReplaceTransactionReplace() { Table original = catalog.buildTable(TABLE, OTHER_SCHEMA).create(); - Assert.assertTrue("Table should exist before replaceTransaction", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should exist before replaceTransaction") + .isTrue(); Transaction createOrReplace = catalog.buildTable(TABLE, SCHEMA).createOrReplaceTransaction(); - Assert.assertTrue( - "Table should still exist after replaceTransaction", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should still exist after replaceTransaction") + .isTrue(); createOrReplace.newFastAppend().appendFile(FILE_A).commit(); // validate table has not changed Table table = catalog.loadTable(TABLE); - Assert.assertEquals( - "Table schema should match concurrent create", - OTHER_SCHEMA.asStruct(), - table.schema().asStruct()); + + Assertions.assertThat(table.schema().asStruct()) + .as("Table schema should match concurrent create") + .isEqualTo(OTHER_SCHEMA.asStruct()); + assertUUIDsMatch(original, table); assertNoFiles(table); createOrReplace.commitTransaction(); // validate the table after replace - Assert.assertTrue("Table should exist after append commit", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should exist after append commit") + .isTrue(); table.refresh(); // refresh should work with UUID validation Table loaded = catalog.loadTable(TABLE); - Assert.assertEquals( - "Table schema should match the new schema", - REPLACE_SCHEMA.asStruct(), - loaded.schema().asStruct()); + Assertions.assertThat(loaded.schema().asStruct()) + .as("Table schema should match the new schema") + .isEqualTo(REPLACE_SCHEMA.asStruct()); assertUUIDsMatch(original, loaded); assertFiles(loaded, FILE_A); assertPreviousMetadataFileCount(loaded, 1); @@ -1830,7 +1924,9 @@ public void testCompleteCreateOrReplaceTransactionReplace() { Table original = catalog.buildTable(TABLE, OTHER_SCHEMA).create(); - Assert.assertTrue("Table should exist before replaceTransaction", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should exist before replaceTransaction") + .isTrue(); Map properties = ImmutableMap.of("user", "someone", "created-at", "2022-02-25T00:38:19"); @@ -1843,47 +1939,55 @@ public void testCompleteCreateOrReplaceTransactionReplace() { .withProperties(properties) .createOrReplaceTransaction(); - Assert.assertTrue( - "Table should still exist after replaceTransaction", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should still exist after replaceTransaction") + .isTrue(); createOrReplace.newFastAppend().appendFile(FILE_A).commit(); // validate table has not changed Table table = catalog.loadTable(TABLE); - Assert.assertEquals( - "Table schema should match concurrent create", - OTHER_SCHEMA.asStruct(), - table.schema().asStruct()); - Assert.assertTrue("Table should be unpartitioned", table.spec().isUnpartitioned()); - Assert.assertTrue("Table should be unsorted", table.sortOrder().isUnsorted()); - Assert.assertNotEquals( - "Created at should not match", table.properties().get("created-at"), "2022-02-25T00:38:19"); + Assertions.assertThat(table.schema().asStruct()) + .as("Table schema should match concurrent create") + .isEqualTo(OTHER_SCHEMA.asStruct()); + Assertions.assertThat(table.spec().isUnpartitioned()) + .as("Table should be unpartitioned") + .isTrue(); + Assertions.assertThat(table.sortOrder().isUnsorted()).as("Table should be unsorted").isTrue(); + Assertions.assertThat(table.properties().get("created-at")) + .as("Created at should not match") + .isNotEqualTo("2022-02-25T00:38:19"); assertUUIDsMatch(original, table); assertNoFiles(table); createOrReplace.commitTransaction(); // validate the table after replace - Assert.assertTrue("Table should exist after append commit", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should exist after append commit") + .isTrue(); table.refresh(); // refresh should work with UUID validation Table loaded = catalog.loadTable(TABLE); - Assert.assertEquals( - "Table schema should match the new schema", - REPLACE_SCHEMA.asStruct(), - loaded.schema().asStruct()); - Assert.assertEquals("Table should have replace partition spec", REPLACE_SPEC, loaded.spec()); - Assert.assertEquals( - "Table should have replace sort order", REPLACE_WRITE_ORDER, loaded.sortOrder()); - Assert.assertEquals( - "Table properties should be a superset of the requested properties", - properties.entrySet(), - Sets.intersection(properties.entrySet(), loaded.properties().entrySet())); + Assertions.assertThat(loaded.schema().asStruct()) + .as("Table schema should match the new schema") + .isEqualTo(REPLACE_SCHEMA.asStruct()); + Assertions.assertThat(loaded.spec()) + .as("Table should have replace partition spec") + .isEqualTo(REPLACE_SPEC); + Assertions.assertThat(loaded.sortOrder()) + .as("Table should have replace sort order") + .isEqualTo(REPLACE_WRITE_ORDER); + Assertions.assertThat(loaded.properties().entrySet()) + .as("Table properties should be a superset of the requested properties") + .containsAll(properties.entrySet()); if (!overridesRequestedLocation()) { - Assert.assertEquals( - "Table location should be replaced", "file:/tmp/ns/table", table.location()); + Assertions.assertThat(table.location()) + .as("Table location should be replaced") + .isEqualTo("file:/tmp/ns/table"); } + assertUUIDsMatch(original, loaded); assertFiles(loaded, FILE_A); assertPreviousMetadataFileCount(loaded, 1); @@ -1891,9 +1995,9 @@ public void testCompleteCreateOrReplaceTransactionReplace() { @Test public void testCreateOrReplaceTransactionConcurrentCreate() { - Assume.assumeTrue( - "Conversion to replace transaction is not supported by REST catalog", - supportsServerSideRetry()); + Assumptions.assumeTrue( + supportsServerSideRetry(), + "Conversion to replace transaction is not supported by REST catalog"); C catalog = catalog(); @@ -1903,12 +2007,15 @@ public void testCreateOrReplaceTransactionConcurrentCreate() { Transaction createOrReplace = catalog.buildTable(TABLE, SCHEMA).createOrReplaceTransaction(); - Assert.assertFalse( - "Table should not exist after createTransaction", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should not exist after createTransaction") + .isFalse(); createOrReplace.newFastAppend().appendFile(FILE_A).commit(); - Assert.assertFalse("Table should not exist after append commit", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should not exist after append commit") + .isFalse(); catalog.buildTable(TABLE, OTHER_SCHEMA).create(); @@ -1918,14 +2025,13 @@ public void testCreateOrReplaceTransactionConcurrentCreate() { : "Table already exists"; Assertions.assertThatThrownBy(createOrReplace::commitTransaction) .isInstanceOf(AlreadyExistsException.class) - .hasMessage(expectedMessage); + .hasMessageStartingWith(expectedMessage); // validate the concurrently created table is unmodified Table table = catalog.loadTable(TABLE); - Assert.assertEquals( - "Table schema should match concurrent create", - OTHER_SCHEMA.asStruct(), - table.schema().asStruct()); + Assertions.assertThat(table.schema().asStruct()) + .as("Table schema should match concurrent create") + .isEqualTo(OTHER_SCHEMA.asStruct()); assertNoFiles(table); } @@ -1939,36 +2045,40 @@ public void testReplaceTransaction() { Table original = catalog.buildTable(TABLE, OTHER_SCHEMA).create(); - Assert.assertTrue("Table should exist before replaceTransaction", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should exist before replaceTransaction") + .isTrue(); Transaction replace = catalog.buildTable(TABLE, SCHEMA).replaceTransaction(); - Assert.assertTrue( - "Table should still exist after replaceTransaction", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should still exist after replaceTransaction") + .isTrue(); replace.newFastAppend().appendFile(FILE_A).commit(); // validate table has not changed Table table = catalog.loadTable(TABLE); - Assert.assertEquals( - "Table schema should match concurrent create", - OTHER_SCHEMA.asStruct(), - table.schema().asStruct()); + Assertions.assertThat(table.schema().asStruct()) + .as("Table schema should match concurrent create") + .isEqualTo(OTHER_SCHEMA.asStruct()); assertUUIDsMatch(original, table); assertNoFiles(table); replace.commitTransaction(); // validate the table after replace - Assert.assertTrue("Table should exist after append commit", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should exist after append commit") + .isTrue(); table.refresh(); // refresh should work with UUID validation Table loaded = catalog.loadTable(TABLE); - Assert.assertEquals( - "Table schema should match the new schema", - REPLACE_SCHEMA.asStruct(), - loaded.schema().asStruct()); + Assertions.assertThat(loaded.schema().asStruct()) + .as("Table schema should match the new schema") + .isEqualTo(REPLACE_SCHEMA.asStruct()); + assertUUIDsMatch(original, loaded); assertFiles(loaded, FILE_A); assertPreviousMetadataFileCount(loaded, 1); @@ -1984,7 +2094,9 @@ public void testCompleteReplaceTransaction() { Table original = catalog.buildTable(TABLE, OTHER_SCHEMA).create(); - Assert.assertTrue("Table should exist before replaceTransaction", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should exist before replaceTransaction") + .isTrue(); Map properties = ImmutableMap.of("user", "someone", "created-at", "2022-02-25T00:38:19"); @@ -1997,47 +2109,57 @@ public void testCompleteReplaceTransaction() { .withProperties(properties) .replaceTransaction(); - Assert.assertTrue( - "Table should still exist after replaceTransaction", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should still exist after replaceTransaction") + .isTrue(); replace.newFastAppend().appendFile(FILE_A).commit(); // validate table has not changed Table table = catalog.loadTable(TABLE); - Assert.assertEquals( - "Table schema should match concurrent create", - OTHER_SCHEMA.asStruct(), - table.schema().asStruct()); - Assert.assertTrue("Table should be unpartitioned", table.spec().isUnpartitioned()); - Assert.assertTrue("Table should be unsorted", table.sortOrder().isUnsorted()); - Assert.assertNotEquals( - "Created at should not match", table.properties().get("created-at"), "2022-02-25T00:38:19"); + + Assertions.assertThat(table.schema().asStruct()) + .as("Table schema should match concurrent create") + .isEqualTo(OTHER_SCHEMA.asStruct()); + Assertions.assertThat(table.spec().isUnpartitioned()) + .as("Table should be unpartitioned") + .isTrue(); + Assertions.assertThat(table.sortOrder().isUnsorted()).as("Table should be unsorted").isTrue(); + Assertions.assertThat(table.properties().get("created-at")) + .as("Created at should not match") + .isNotEqualTo("2022-02-25T00:38:19"); + assertUUIDsMatch(original, table); assertNoFiles(table); replace.commitTransaction(); // validate the table after replace - Assert.assertTrue("Table should exist after append commit", catalog.tableExists(TABLE)); + Assertions.assertThat(catalog.tableExists(TABLE)) + .as("Table should exist after append commit") + .isTrue(); table.refresh(); // refresh should work with UUID validation Table loaded = catalog.loadTable(TABLE); - Assert.assertEquals( - "Table schema should match the new schema", - REPLACE_SCHEMA.asStruct(), - loaded.schema().asStruct()); - Assert.assertEquals("Table should have replace partition spec", REPLACE_SPEC, loaded.spec()); - Assert.assertEquals( - "Table should have replace sort order", REPLACE_WRITE_ORDER, loaded.sortOrder()); - Assert.assertEquals( - "Table properties should be a superset of the requested properties", - properties.entrySet(), - Sets.intersection(properties.entrySet(), loaded.properties().entrySet())); + Assertions.assertThat(loaded.schema().asStruct()) + .as("Table schema should match the new schema") + .isEqualTo(REPLACE_SCHEMA.asStruct()); + Assertions.assertThat(loaded.spec()) + .as("Table should have replace partition spec") + .isEqualTo(REPLACE_SPEC); + Assertions.assertThat(loaded.sortOrder()) + .as("Table should have replace sort order") + .isEqualTo(REPLACE_WRITE_ORDER); + Assertions.assertThat(loaded.properties().entrySet()) + .as("Table properties should be a superset of the requested properties") + .containsAll(properties.entrySet()); if (!overridesRequestedLocation()) { - Assert.assertEquals( - "Table location should be replaced", "file:/tmp/ns/table", table.location()); + Assertions.assertThat(table.location()) + .as("Table location should be replaced") + .isEqualTo("file:/tmp/ns/table"); } + assertUUIDsMatch(original, loaded); assertFiles(loaded, FILE_A); assertPreviousMetadataFileCount(loaded, 1); @@ -2053,7 +2175,7 @@ public void testReplaceTransactionRequiresTableExists() { Assertions.assertThatThrownBy(() -> catalog.buildTable(TABLE, SCHEMA).replaceTransaction()) .isInstanceOf(NoSuchTableException.class) - .hasMessage("Table does not exist: newdb.table"); + .hasMessageStartingWith("Table does not exist: newdb.table"); } @Test @@ -2079,24 +2201,30 @@ public void testConcurrentReplaceTransactions() { firstReplace.commitTransaction(); Table afterFirstReplace = catalog.loadTable(TABLE); - Assert.assertEquals( - "Table schema should match the original schema", - original.schema().asStruct(), - afterFirstReplace.schema().asStruct()); - Assert.assertTrue("Table should be unpartitioned", afterFirstReplace.spec().isUnpartitioned()); - Assert.assertTrue("Table should be unsorted", afterFirstReplace.sortOrder().isUnsorted()); + Assertions.assertThat(afterFirstReplace.schema().asStruct()) + .as("Table schema should match the original schema") + .isEqualTo(original.schema().asStruct()); + Assertions.assertThat(afterFirstReplace.spec().isUnpartitioned()) + .as("Table should be unpartitioned") + .isTrue(); + Assertions.assertThat(afterFirstReplace.sortOrder().isUnsorted()) + .as("Table should be unsorted") + .isTrue(); assertUUIDsMatch(original, afterFirstReplace); assertFiles(afterFirstReplace, FILE_B); secondReplace.commitTransaction(); Table afterSecondReplace = catalog.loadTable(TABLE); - Assert.assertEquals( - "Table schema should match the original schema", - original.schema().asStruct(), - afterSecondReplace.schema().asStruct()); - Assert.assertTrue("Table should be unpartitioned", afterSecondReplace.spec().isUnpartitioned()); - Assert.assertTrue("Table should be unsorted", afterSecondReplace.sortOrder().isUnsorted()); + Assertions.assertThat(afterSecondReplace.schema().asStruct()) + .as("Table schema should match the original schema") + .isEqualTo(original.schema().asStruct()); + Assertions.assertThat(afterSecondReplace.spec().isUnpartitioned()) + .as("Table should be unpartitioned") + .isTrue(); + Assertions.assertThat(afterSecondReplace.sortOrder().isUnsorted()) + .as("Table should be unsorted") + .isTrue(); assertUUIDsMatch(original, afterSecondReplace); assertFiles(afterSecondReplace, FILE_C); } @@ -2124,20 +2252,18 @@ public void testConcurrentReplaceTransactionSchema() { firstReplace.commitTransaction(); Table afterFirstReplace = catalog.loadTable(TABLE); - Assert.assertEquals( - "Table schema should match the new schema", - REPLACE_SCHEMA.asStruct(), - afterFirstReplace.schema().asStruct()); + Assertions.assertThat(afterFirstReplace.schema().asStruct()) + .as("Table schema should match the new schema") + .isEqualTo(REPLACE_SCHEMA.asStruct()); assertUUIDsMatch(original, afterFirstReplace); assertFiles(afterFirstReplace, FILE_B); secondReplace.commitTransaction(); Table afterSecondReplace = catalog.loadTable(TABLE); - Assert.assertEquals( - "Table schema should match the original schema", - original.schema().asStruct(), - afterSecondReplace.schema().asStruct()); + Assertions.assertThat(afterSecondReplace.schema().asStruct()) + .as("Table schema should match the original schema") + .isEqualTo(original.schema().asStruct()); assertUUIDsMatch(original, afterSecondReplace); assertFiles(afterSecondReplace, FILE_C); } @@ -2165,27 +2291,25 @@ public void testConcurrentReplaceTransactionSchema2() { firstReplace.commitTransaction(); Table afterFirstReplace = catalog.loadTable(TABLE); - Assert.assertEquals( - "Table schema should match the original schema", - original.schema().asStruct(), - afterFirstReplace.schema().asStruct()); + Assertions.assertThat(afterFirstReplace.schema().asStruct()) + .as("Table schema should match the original schema") + .isEqualTo(original.schema().asStruct()); assertUUIDsMatch(original, afterFirstReplace); assertFiles(afterFirstReplace, FILE_B); secondReplace.commitTransaction(); Table afterSecondReplace = catalog.loadTable(TABLE); - Assert.assertEquals( - "Table schema should match the new schema", - REPLACE_SCHEMA.asStruct(), - afterSecondReplace.schema().asStruct()); + Assertions.assertThat(afterSecondReplace.schema().asStruct()) + .as("Table schema should match the new schema") + .isEqualTo(REPLACE_SCHEMA.asStruct()); assertUUIDsMatch(original, afterSecondReplace); assertFiles(afterSecondReplace, FILE_C); } @Test public void testConcurrentReplaceTransactionSchemaConflict() { - Assume.assumeTrue("Schema conflicts are detected server-side", supportsServerSideRetry()); + Assumptions.assumeTrue(supportsServerSideRetry(), "Schema conflicts are detected server-side"); C catalog = catalog(); @@ -2208,10 +2332,10 @@ public void testConcurrentReplaceTransactionSchemaConflict() { firstReplace.commitTransaction(); Table afterFirstReplace = catalog.loadTable(TABLE); - Assert.assertEquals( - "Table schema should match the original schema", - REPLACE_SCHEMA.asStruct(), - afterFirstReplace.schema().asStruct()); + Assertions.assertThat(afterFirstReplace.schema().asStruct()) + .as("Table schema should match the original schema") + .isEqualTo(REPLACE_SCHEMA.asStruct()); + assertUUIDsMatch(original, afterFirstReplace); assertFiles(afterFirstReplace, FILE_B); @@ -2247,17 +2371,18 @@ public void testConcurrentReplaceTransactionPartitionSpec() { firstReplace.commitTransaction(); Table afterFirstReplace = catalog.loadTable(TABLE); - Assert.assertEquals( - "Table spec should match the new spec", - TABLE_SPEC.fields(), - afterFirstReplace.spec().fields()); + Assertions.assertThat(afterFirstReplace.spec().fields()) + .as("Table spec should match the new spec") + .isEqualTo(TABLE_SPEC.fields()); assertUUIDsMatch(original, afterFirstReplace); assertFiles(afterFirstReplace, FILE_B); secondReplace.commitTransaction(); Table afterSecondReplace = catalog.loadTable(TABLE); - Assert.assertTrue("Table should be unpartitioned", afterSecondReplace.spec().isUnpartitioned()); + Assertions.assertThat(afterSecondReplace.spec().isUnpartitioned()) + .as("Table should be unpartitioned") + .isTrue(); assertUUIDsMatch(original, afterSecondReplace); assertFiles(afterSecondReplace, FILE_C); } @@ -2286,24 +2411,25 @@ public void testConcurrentReplaceTransactionPartitionSpec2() { firstReplace.commitTransaction(); Table afterFirstReplace = catalog.loadTable(TABLE); - Assert.assertTrue("Table should be unpartitioned", afterFirstReplace.spec().isUnpartitioned()); + Assertions.assertThat(afterFirstReplace.spec().isUnpartitioned()) + .as("Table should be unpartitioned") + .isTrue(); assertUUIDsMatch(original, afterFirstReplace); assertFiles(afterFirstReplace, FILE_B); secondReplace.commitTransaction(); Table afterSecondReplace = catalog.loadTable(TABLE); - Assert.assertEquals( - "Table spec should match the new spec", - TABLE_SPEC.fields(), - afterSecondReplace.spec().fields()); + Assertions.assertThat(afterSecondReplace.spec().fields()) + .as("Table spec should match the new spec") + .isEqualTo(TABLE_SPEC.fields()); assertUUIDsMatch(original, afterSecondReplace); assertFiles(afterSecondReplace, FILE_C); } @Test public void testConcurrentReplaceTransactionPartitionSpecConflict() { - Assume.assumeTrue("Spec conflicts are detected server-side", supportsServerSideRetry()); + Assumptions.assumeTrue(supportsServerSideRetry(), "Spec conflicts are detected server-side"); C catalog = catalog(); if (requiresNamespaceCreate()) { @@ -2327,10 +2453,9 @@ public void testConcurrentReplaceTransactionPartitionSpecConflict() { firstReplace.commitTransaction(); Table afterFirstReplace = catalog.loadTable(TABLE); - Assert.assertEquals( - "Table spec should match the new spec", - TABLE_SPEC.fields(), - afterFirstReplace.spec().fields()); + Assertions.assertThat(afterFirstReplace.spec().fields()) + .as("Table spec should match the new spec") + .isEqualTo(TABLE_SPEC.fields()); assertUUIDsMatch(original, afterFirstReplace); assertFiles(afterFirstReplace, FILE_B); @@ -2366,15 +2491,18 @@ public void testConcurrentReplaceTransactionSortOrder() { firstReplace.commitTransaction(); Table afterFirstReplace = catalog.loadTable(TABLE); - Assert.assertEquals( - "Table order should match the new order", TABLE_WRITE_ORDER, afterFirstReplace.sortOrder()); + Assertions.assertThat(afterFirstReplace.sortOrder()) + .as("Table order should match the new order") + .isEqualTo(TABLE_WRITE_ORDER); assertUUIDsMatch(original, afterFirstReplace); assertFiles(afterFirstReplace, FILE_B); secondReplace.commitTransaction(); Table afterSecondReplace = catalog.loadTable(TABLE); - Assert.assertTrue("Table should be unsorted", afterSecondReplace.sortOrder().isUnsorted()); + Assertions.assertThat(afterSecondReplace.sortOrder().isUnsorted()) + .as("Table should be unsorted") + .isTrue(); assertUUIDsMatch(original, afterSecondReplace); assertFiles(afterSecondReplace, FILE_C); } @@ -2408,17 +2536,18 @@ public void testConcurrentReplaceTransactionSortOrderConflict() { firstReplace.commitTransaction(); Table afterFirstReplace = catalog.loadTable(TABLE); - Assert.assertTrue("Table order should be set", afterFirstReplace.sortOrder().isSorted()); + Assertions.assertThat(afterFirstReplace.sortOrder().isSorted()) + .as("Table order should be set") + .isTrue(); assertUUIDsMatch(original, afterFirstReplace); assertFiles(afterFirstReplace, FILE_B); secondReplace.commitTransaction(); Table afterSecondReplace = catalog.loadTable(TABLE); - Assert.assertEquals( - "Table order should match the new order", - TABLE_WRITE_ORDER.fields(), - afterSecondReplace.sortOrder().fields()); + Assertions.assertThat(afterSecondReplace.sortOrder().fields()) + .as("Table order should match the new order") + .isEqualTo(TABLE_WRITE_ORDER.fields()); assertUUIDsMatch(original, afterSecondReplace); assertFiles(afterSecondReplace, FILE_C); } @@ -2484,35 +2613,118 @@ public void tableCreationWithoutNamespace() { () -> catalog().buildTable(TableIdentifier.of("non-existing", "table"), SCHEMA).create()) .isInstanceOf(NoSuchNamespaceException.class) - .hasMessageEndingWith("Namespace does not exist: non-existing"); + .hasMessageContaining("Namespace does not exist: non-existing"); + } + + @Test + public void testRegisterTable() { + C catalog = catalog(); + + if (requiresNamespaceCreate()) { + catalog.createNamespace(TABLE.namespace()); + } + + Map properties = + ImmutableMap.of("user", "someone", "created-at", "2023-01-15T00:00:01"); + Table originalTable = + catalog + .buildTable(TABLE, SCHEMA) + .withPartitionSpec(SPEC) + .withSortOrder(WRITE_ORDER) + .withProperties(properties) + .create(); + + originalTable.newFastAppend().appendFile(FILE_A).commit(); + originalTable.newFastAppend().appendFile(FILE_B).commit(); + originalTable.newDelete().deleteFile(FILE_A).commit(); + originalTable.newFastAppend().appendFile(FILE_C).commit(); + + TableOperations ops = ((BaseTable) originalTable).operations(); + String metadataLocation = ops.current().metadataFileLocation(); + + catalog.dropTable(TABLE, false /* do not purge */); + + Table registeredTable = catalog.registerTable(TABLE, metadataLocation); + + Assertions.assertThat(registeredTable).isNotNull(); + Assertions.assertThat(catalog.tableExists(TABLE)).as("Table must exist").isTrue(); + Assertions.assertThat(registeredTable.properties()) + .as("Props must match") + .containsAllEntriesOf(properties); + Assertions.assertThat(registeredTable.schema().asStruct()) + .as("Schema must match") + .isEqualTo(originalTable.schema().asStruct()); + Assertions.assertThat(registeredTable.specs()) + .as("Specs must match") + .isEqualTo(originalTable.specs()); + Assertions.assertThat(registeredTable.sortOrders()) + .as("Sort orders must match") + .isEqualTo(originalTable.sortOrders()); + Assertions.assertThat(registeredTable.currentSnapshot()) + .as("Current snapshot must match") + .isEqualTo(originalTable.currentSnapshot()); + Assertions.assertThat(registeredTable.snapshots()) + .as("Snapshots must match") + .isEqualTo(originalTable.snapshots()); + Assertions.assertThat(registeredTable.history()) + .as("History must match") + .isEqualTo(originalTable.history()); + + TestHelpers.assertSameSchemaMap(registeredTable.schemas(), originalTable.schemas()); + assertFiles(registeredTable, FILE_B, FILE_C); + + registeredTable.newFastAppend().appendFile(FILE_A).commit(); + assertFiles(registeredTable, FILE_B, FILE_C, FILE_A); + + Assertions.assertThat(catalog.loadTable(TABLE)).isNotNull(); + Assertions.assertThat(catalog.dropTable(TABLE)).isTrue(); + Assertions.assertThat(catalog.tableExists(TABLE)).isFalse(); + } + + @Test + public void testRegisterExistingTable() { + C catalog = catalog(); + + TableIdentifier identifier = TableIdentifier.of("a", "t1"); + + if (requiresNamespaceCreate()) { + catalog.createNamespace(identifier.namespace()); + } + + catalog.createTable(identifier, SCHEMA); + Table table = catalog.loadTable(identifier); + TableOperations ops = ((BaseTable) table).operations(); + String metadataLocation = ops.current().metadataFileLocation(); + Assertions.assertThatThrownBy(() -> catalog.registerTable(identifier, metadataLocation)) + .isInstanceOf(AlreadyExistsException.class) + .hasMessage("Table already exists: a.t1"); + Assertions.assertThat(catalog.dropTable(identifier)).isTrue(); } private static void assertEmpty(String context, Catalog catalog, Namespace ns) { try { - Assert.assertEquals(context, 0, catalog.listTables(ns).size()); + Assertions.assertThat(catalog.listTables(ns)).as(context).isEmpty(); } catch (NoSuchNamespaceException e) { // it is okay if the catalog throws NoSuchNamespaceException when it is empty } } public void assertUUIDsMatch(Table expected, Table actual) { - Assert.assertEquals( - "Table UUID should not change", - ((BaseTable) expected).operations().current().uuid(), - ((BaseTable) actual).operations().current().uuid()); + Assertions.assertThat(((BaseTable) actual).operations().current().uuid()) + .as("Table UUID should not change") + .isEqualTo(((BaseTable) expected).operations().current().uuid()); } public void assertPreviousMetadataFileCount(Table table, int metadataFileCount) { TableOperations ops = ((BaseTable) table).operations(); - Assert.assertEquals( - "Table should have correct number of previous metadata locations", - metadataFileCount, - ops.current().previousFiles().size()); + Assertions.assertThat(ops.current().previousFiles().size()) + .as("Table should have correct number of previous metadata locations") + .isEqualTo(metadataFileCount); } public void assertNoFiles(Table table) { try (CloseableIterable tasks = table.newScan().planFiles()) { - Assert.assertFalse("Should contain no files", tasks.iterator().hasNext()); + Assertions.assertThat(tasks).as("Should contain no files").isEmpty(); } catch (IOException e) { throw new UncheckedIOException(e); } @@ -2525,12 +2737,12 @@ public void assertFiles(Table table, DataFile... files) { .map(FileScanTask::file) .map(DataFile::path) .collect(Collectors.toList()); - Assert.assertEquals( - "Should contain expected number of data files", files.length, paths.size()); - Assert.assertEquals( - "Should contain correct file paths", - CharSequenceSet.of(Iterables.transform(Arrays.asList(files), DataFile::path)), - CharSequenceSet.of(paths)); + Assertions.assertThat(paths.size()) + .as("Should contain expected number of data files") + .isEqualTo(files.length); + Assertions.assertThat(CharSequenceSet.of(paths)) + .as("Should contain correct file paths") + .isEqualTo(CharSequenceSet.of(Iterables.transform(Arrays.asList(files), DataFile::path))); } catch (IOException e) { throw new UncheckedIOException(e); } @@ -2541,7 +2753,11 @@ public void assertFilePartitionSpec(Table table, DataFile dataFile, int specId) Streams.stream(tasks) .map(FileScanTask::file) .filter(file -> file.path().equals(dataFile.path())) - .forEach(file -> Assert.assertEquals(specId, file.specId())); + .forEach( + file -> + Assertions.assertThat(file.specId()) + .as("Spec ID should match") + .isEqualTo(specId)); } catch (IOException e) { throw new UncheckedIOException(e); } @@ -2551,7 +2767,11 @@ public void assertFilesPartitionSpec(Table table) { try (CloseableIterable tasks = table.newScan().planFiles()) { Streams.stream(tasks) .map(FileScanTask::file) - .forEach(file -> Assert.assertEquals(table.spec().specId(), file.specId())); + .forEach( + file -> + Assertions.assertThat(file.specId()) + .as("Spec ID should match") + .isEqualTo(table.spec().specId())); } catch (IOException e) { throw new UncheckedIOException(e); } diff --git a/core/src/test/java/org/apache/iceberg/catalog/TestTableIdentifierParser.java b/core/src/test/java/org/apache/iceberg/catalog/TestTableIdentifierParser.java index 05ca49293f7e..587b9395f514 100644 --- a/core/src/test/java/org/apache/iceberg/catalog/TestTableIdentifierParser.java +++ b/core/src/test/java/org/apache/iceberg/catalog/TestTableIdentifierParser.java @@ -19,8 +19,7 @@ package org.apache.iceberg.catalog; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestTableIdentifierParser { @@ -28,40 +27,36 @@ public class TestTableIdentifierParser { public void testTableIdentifierToJson() { String json = "{\"namespace\":[\"accounting\",\"tax\"],\"name\":\"paid\"}"; TableIdentifier identifier = TableIdentifier.of(Namespace.of("accounting", "tax"), "paid"); - Assert.assertEquals( - "Should be able to serialize a table identifier with both namespace and name", - json, - TableIdentifierParser.toJson(identifier)); + Assertions.assertThat(TableIdentifierParser.toJson(identifier)) + .as("Should be able to serialize a table identifier with both namespace and name") + .isEqualTo(json); TableIdentifier identifierWithEmptyNamespace = TableIdentifier.of(Namespace.empty(), "paid"); String jsonWithEmptyNamespace = "{\"namespace\":[],\"name\":\"paid\"}"; - Assert.assertEquals( - "Should be able to serialize a table identifier that uses the empty namespace", - jsonWithEmptyNamespace, - TableIdentifierParser.toJson(identifierWithEmptyNamespace)); + Assertions.assertThat(TableIdentifierParser.toJson(identifierWithEmptyNamespace)) + .as("Should be able to serialize a table identifier that uses the empty namespace") + .isEqualTo(jsonWithEmptyNamespace); } @Test public void testTableIdentifierFromJson() { String json = "{\"namespace\":[\"accounting\",\"tax\"],\"name\":\"paid\"}"; TableIdentifier identifier = TableIdentifier.of(Namespace.of("accounting", "tax"), "paid"); - Assert.assertEquals( - "Should be able to deserialize a valid table identifier", - identifier, - TableIdentifierParser.fromJson(json)); + Assertions.assertThat(TableIdentifierParser.fromJson(json)) + .as("Should be able to deserialize a valid table identifier") + .isEqualTo(identifier); TableIdentifier identifierWithEmptyNamespace = TableIdentifier.of(Namespace.empty(), "paid"); String jsonWithEmptyNamespace = "{\"namespace\":[],\"name\":\"paid\"}"; - Assert.assertEquals( - "Should be able to deserialize a valid multi-level table identifier", - identifierWithEmptyNamespace, - TableIdentifierParser.fromJson(jsonWithEmptyNamespace)); + Assertions.assertThat(TableIdentifierParser.fromJson(jsonWithEmptyNamespace)) + .as("Should be able to deserialize a valid multi-level table identifier") + .isEqualTo(identifierWithEmptyNamespace); String identifierMissingNamespace = "{\"name\":\"paid\"}"; - Assert.assertEquals( - "Should implicitly convert a missing namespace into the the empty namespace when parsing", - identifierWithEmptyNamespace, - TableIdentifierParser.fromJson(identifierMissingNamespace)); + Assertions.assertThat(TableIdentifierParser.fromJson(identifierMissingNamespace)) + .as( + "Should implicitly convert a missing namespace into the the empty namespace when parsing") + .isEqualTo(identifierWithEmptyNamespace); } @Test @@ -100,7 +95,7 @@ public void testFailWhenFieldsHaveInvalidValues() { String invalidNamespace = "{\"namespace\":\"accounting.tax\",\"name\":\"paid\"}"; Assertions.assertThatThrownBy(() -> TableIdentifierParser.fromJson(invalidNamespace)) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot parse from non-array value: namespace: \"accounting.tax\""); + .hasMessage("Cannot parse JSON array from non-array value: namespace: \"accounting.tax\""); String invalidName = "{\"namespace\":[\"accounting\",\"tax\"],\"name\":1234}"; Assertions.assertThatThrownBy(() -> TableIdentifierParser.fromJson(invalidName)) diff --git a/core/src/test/java/org/apache/iceberg/data/avro/TestDecoderResolver.java b/core/src/test/java/org/apache/iceberg/data/avro/TestDecoderResolver.java new file mode 100644 index 000000000000..5855e80998d3 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/data/avro/TestDecoderResolver.java @@ -0,0 +1,167 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.data.avro; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Set; +import java.util.concurrent.TimeUnit; +import org.apache.avro.Schema; +import org.apache.avro.io.Decoder; +import org.apache.avro.io.DecoderFactory; +import org.apache.avro.io.ResolvingDecoder; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.awaitility.Awaitility; +import org.junit.Before; +import org.junit.Test; + +public class TestDecoderResolver { + + @Before + public void before() { + DecoderResolver.DECODER_CACHES.get().clear(); + } + + @Test + public void testDecoderCachingReadSchemaSameAsFileSchema() throws Exception { + Decoder dummyDecoder = DecoderFactory.get().binaryDecoder(new byte[] {}, null); + Schema fileSchema = avroSchema(); + ResolvingDecoder resolvingDecoder = + DecoderResolver.resolve(dummyDecoder, fileSchema, fileSchema); + + assertThat(DecoderResolver.DECODER_CACHES.get().size()).isEqualTo(1); + assertThat(DecoderResolver.DECODER_CACHES.get().get(fileSchema).size()).isEqualTo(1); + checkCached(fileSchema, fileSchema); + + // Equal but new one + Schema fileSchema1 = avroSchema(); + assertThat(fileSchema1).isEqualTo(fileSchema); + ResolvingDecoder resolvingDecoder1 = + DecoderResolver.resolve(dummyDecoder, fileSchema1, fileSchema1); + assertThat(resolvingDecoder1).isNotSameAs(resolvingDecoder); + + assertThat(DecoderResolver.DECODER_CACHES.get().size()).isEqualTo(2); + assertThat(DecoderResolver.DECODER_CACHES.get().get(fileSchema1).size()).isEqualTo(1); + checkCached(fileSchema1, fileSchema1); + + // New one + Schema fileSchema2 = avroSchema("manifest_path", "manifest_length"); + ResolvingDecoder resolvingDecoder2 = + DecoderResolver.resolve(dummyDecoder, fileSchema2, fileSchema2); + assertThat(resolvingDecoder2).isNotSameAs(resolvingDecoder); + + assertThat(DecoderResolver.DECODER_CACHES.get().size()).isEqualTo(3); + assertThat(DecoderResolver.DECODER_CACHES.get().get(fileSchema2).size()).isEqualTo(1); + checkCached(fileSchema2, fileSchema2); + + checkCachedSize(3); + + fileSchema = null; + checkCachedSize(2); + + fileSchema1 = null; + checkCachedSize(1); + + fileSchema2 = null; + checkCachedSize(0); + } + + @Test + public void testDecoderCachingReadSchemaNotSameAsFileSchema() throws Exception { + Decoder dummyDecoder = DecoderFactory.get().binaryDecoder(new byte[] {}, null); + Schema fileSchema = avroSchema(); + Schema readSchema = avroSchema("manifest_path", "manifest_length", "partition_spec_id"); + ResolvingDecoder resolvingDecoder = + DecoderResolver.resolve(dummyDecoder, readSchema, fileSchema); + + assertThat(DecoderResolver.DECODER_CACHES.get().size()).isEqualTo(1); + assertThat(DecoderResolver.DECODER_CACHES.get().get(readSchema).size()).isEqualTo(1); + checkCached(readSchema, fileSchema); + + // Equal but new one + Schema fileSchema1 = avroSchema(); + Schema readSchema1 = avroSchema("manifest_path", "manifest_length", "partition_spec_id"); + assertThat(fileSchema1).isEqualTo(fileSchema); + assertThat(readSchema1).isEqualTo(readSchema); + ResolvingDecoder resolvingDecoder1 = + DecoderResolver.resolve(dummyDecoder, readSchema1, fileSchema1); + assertThat(resolvingDecoder1).isNotSameAs(resolvingDecoder); + + assertThat(DecoderResolver.DECODER_CACHES.get().size()).isEqualTo(2); + assertThat(DecoderResolver.DECODER_CACHES.get().get(readSchema1).size()).isEqualTo(1); + checkCached(readSchema1, fileSchema1); + + // New read schema + Schema readSchema2 = avroSchema("manifest_path", "manifest_length"); + ResolvingDecoder resolvingDecoder2 = + DecoderResolver.resolve(dummyDecoder, readSchema2, fileSchema); + assertThat(resolvingDecoder2).isNotSameAs(resolvingDecoder); + + assertThat(DecoderResolver.DECODER_CACHES.get().size()).isEqualTo(3); + assertThat(DecoderResolver.DECODER_CACHES.get().get(readSchema2).size()).isEqualTo(1); + checkCached(readSchema2, fileSchema); + + checkCachedSize(3); + + readSchema = null; + checkCachedSize(2); + + readSchema1 = null; + checkCachedSize(1); + + readSchema2 = null; + checkCachedSize(0); + } + + private Schema avroSchema(String... columns) { + if (columns.length == 0) { + return AvroSchemaUtil.convert(ManifestFile.schema(), "manifest_file"); + } else { + return AvroSchemaUtil.convert(ManifestFile.schema().select(columns), "manifest_file"); + } + } + + private void checkCached(Schema readSchema, Schema fileSchema) { + assertThat(DecoderResolver.DECODER_CACHES.get()).containsKey(readSchema); + assertThat(DecoderResolver.DECODER_CACHES.get().get(readSchema)).containsKey(fileSchema); + } + + private int getActualSize() { + // The size of keys included the GCed keys + Set keys = DecoderResolver.DECODER_CACHES.get().keySet(); + Set identityKeys = Sets.newIdentityHashSet(); + // Forcefully remove keys that have been garbage collected + identityKeys.addAll(keys); + return identityKeys.size(); + } + + private void checkCachedSize(int expected) { + System.gc(); + // Wait the weak reference keys are GCed + Awaitility.await() + .atMost(5, TimeUnit.SECONDS) + .pollInSameThread() + .untilAsserted( + () -> { + assertThat(getActualSize()).isEqualTo(expected); + }); + } +} diff --git a/core/src/test/java/org/apache/iceberg/encryption/TestCiphers.java b/core/src/test/java/org/apache/iceberg/encryption/TestCiphers.java index e5b7d7245a64..a428586c3b46 100644 --- a/core/src/test/java/org/apache/iceberg/encryption/TestCiphers.java +++ b/core/src/test/java/org/apache/iceberg/encryption/TestCiphers.java @@ -18,10 +18,11 @@ */ package org.apache.iceberg.encryption; +import static org.assertj.core.api.Assertions.assertThat; + import java.nio.charset.StandardCharsets; import java.security.SecureRandom; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestCiphers { @@ -49,7 +50,7 @@ private void testEncryptDecrypt(byte[] aad) { Ciphers.AesGcmDecryptor decryptor = new Ciphers.AesGcmDecryptor(key); byte[] decryptedText = decryptor.decrypt(ciphertext, aad); - Assert.assertArrayEquals("Key length " + keyLength, plaintext, decryptedText); + assertThat(decryptedText).as("Key length " + keyLength).isEqualTo(plaintext); } } } diff --git a/core/src/test/java/org/apache/iceberg/encryption/TestKeyMetadataParser.java b/core/src/test/java/org/apache/iceberg/encryption/TestKeyMetadataParser.java new file mode 100644 index 000000000000..27b00b1829c6 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/encryption/TestKeyMetadataParser.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.encryption; + +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Test; + +public class TestKeyMetadataParser { + + @Test + public void testParser() { + ByteBuffer encryptionKey = ByteBuffer.wrap("0123456789012345".getBytes(StandardCharsets.UTF_8)); + ByteBuffer aadPrefix = ByteBuffer.wrap("1234567890123456".getBytes(StandardCharsets.UTF_8)); + KeyMetadata metadata = new KeyMetadata(encryptionKey, aadPrefix); + ByteBuffer serialized = metadata.buffer(); + + KeyMetadata parsedMetadata = KeyMetadata.parse(serialized); + Assert.assertEquals(parsedMetadata.encryptionKey(), encryptionKey); + Assert.assertEquals(parsedMetadata.aadPrefix(), aadPrefix); + } + + @Test + public void testUnsupportedVersion() { + ByteBuffer badBuffer = ByteBuffer.wrap(new byte[] {0x02}); + Assertions.assertThatThrownBy(() -> KeyMetadata.parse(badBuffer)) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage("Cannot resolve schema for version: 2"); + } +} diff --git a/core/src/test/java/org/apache/iceberg/hadoop/HadoopFileIOTest.java b/core/src/test/java/org/apache/iceberg/hadoop/HadoopFileIOTest.java index 818dfaabc67e..636c94069d5c 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/HadoopFileIOTest.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/HadoopFileIOTest.java @@ -18,9 +18,6 @@ */ package org.apache.iceberg.hadoop; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertThrows; - import java.io.File; import java.io.IOException; import java.io.UncheckedIOException; @@ -38,7 +35,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Streams; -import org.junit.Assert; +import org.assertj.core.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -72,14 +69,15 @@ public void testListPrefix() { Path scalePath = new Path(parent, Integer.toString(scale)); createRandomFiles(scalePath, scale); - assertEquals( - (long) scale, - Streams.stream(hadoopFileIO.listPrefix(scalePath.toUri().toString())).count()); + Assertions.assertThat( + Streams.stream(hadoopFileIO.listPrefix(scalePath.toUri().toString())).count()) + .isEqualTo((long) scale); }); long totalFiles = scaleSizes.stream().mapToLong(Integer::longValue).sum(); - assertEquals( - totalFiles, Streams.stream(hadoopFileIO.listPrefix(parent.toUri().toString())).count()); + Assertions.assertThat( + Streams.stream(hadoopFileIO.listPrefix(parent.toUri().toString())).count()) + .isEqualTo(totalFiles); } @Test @@ -89,10 +87,11 @@ public void testFileExists() throws IOException { fs.createNewFile(randomFilePath); // check existence of the created file - Assert.assertTrue(hadoopFileIO.newInputFile(randomFilePath.toUri().toString()).exists()); - + Assertions.assertThat(hadoopFileIO.newInputFile(randomFilePath.toUri().toString()).exists()) + .isTrue(); fs.delete(randomFilePath, false); - Assert.assertFalse(hadoopFileIO.newInputFile(randomFilePath.toUri().toString()).exists()); + Assertions.assertThat(hadoopFileIO.newInputFile(randomFilePath.toUri().toString()).exists()) + .isFalse(); } @Test @@ -111,16 +110,18 @@ public void testDeletePrefix() { hadoopFileIO.deletePrefix(scalePath.toUri().toString()); // Hadoop filesystem will throw if the path does not exist - assertThrows( - UncheckedIOException.class, - () -> hadoopFileIO.listPrefix(scalePath.toUri().toString()).iterator()); + Assertions.assertThatThrownBy( + () -> hadoopFileIO.listPrefix(scalePath.toUri().toString()).iterator()) + .isInstanceOf(UncheckedIOException.class) + .hasMessageContaining("java.io.FileNotFoundException"); }); hadoopFileIO.deletePrefix(parent.toUri().toString()); // Hadoop filesystem will throw if the path does not exist - assertThrows( - UncheckedIOException.class, - () -> hadoopFileIO.listPrefix(parent.toUri().toString()).iterator()); + Assertions.assertThatThrownBy( + () -> hadoopFileIO.listPrefix(parent.toUri().toString()).iterator()) + .isInstanceOf(UncheckedIOException.class) + .hasMessageContaining("java.io.FileNotFoundException"); } @Test @@ -130,17 +131,17 @@ public void testDeleteFiles() { hadoopFileIO.deleteFiles( filesCreated.stream().map(Path::toString).collect(Collectors.toList())); filesCreated.forEach( - file -> Assert.assertFalse(hadoopFileIO.newInputFile(file.toString()).exists())); + file -> + Assertions.assertThat(hadoopFileIO.newInputFile(file.toString()).exists()).isFalse()); } @Test public void testDeleteFilesErrorHandling() { List filesCreated = random.ints(2).mapToObj(x -> "fakefsnotreal://file-" + x).collect(Collectors.toList()); - Assert.assertThrows( - "Should throw a BulkDeletionFailure Exceptions when files can't be deleted", - BulkDeletionFailureException.class, - () -> hadoopFileIO.deleteFiles(filesCreated)); + Assertions.assertThatThrownBy(() -> hadoopFileIO.deleteFiles(filesCreated)) + .isInstanceOf(BulkDeletionFailureException.class) + .hasMessage("Failed to delete 2 files"); } @Test @@ -151,7 +152,8 @@ public void testHadoopFileIOKryoSerialization() throws IOException { testHadoopFileIO.initialize(ImmutableMap.of("k1", "v1")); FileIO roundTripSerializedFileIO = TestHelpers.KryoHelpers.roundTripSerialize(testHadoopFileIO); - Assert.assertEquals(testHadoopFileIO.properties(), roundTripSerializedFileIO.properties()); + Assertions.assertThat(roundTripSerializedFileIO.properties()) + .isEqualTo(testHadoopFileIO.properties()); } @Test @@ -162,7 +164,8 @@ public void testHadoopFileIOJavaSerialization() throws IOException, ClassNotFoun testHadoopFileIO.initialize(ImmutableMap.of("k1", "v1")); FileIO roundTripSerializedFileIO = TestHelpers.roundTripSerialize(testHadoopFileIO); - Assert.assertEquals(testHadoopFileIO.properties(), roundTripSerializedFileIO.properties()); + Assertions.assertThat(roundTripSerializedFileIO.properties()) + .isEqualTo(testHadoopFileIO.properties()); } private List createRandomFiles(Path parent, int count) { diff --git a/core/src/test/java/org/apache/iceberg/hadoop/HadoopTableTestBase.java b/core/src/test/java/org/apache/iceberg/hadoop/HadoopTableTestBase.java index 5774fbd43205..b8f7dcb80ebc 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/HadoopTableTestBase.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/HadoopTableTestBase.java @@ -48,9 +48,8 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.io.Files; import org.apache.iceberg.types.Types; -import org.junit.Before; -import org.junit.Rule; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.io.TempDir; public class HadoopTableTestBase { // Schema passed to create tables @@ -113,19 +112,16 @@ public class HadoopTableTestBase { .withRecordCount(2) // needs at least one record or else metrics will filter it out .build(); - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir File tempDir; + @TempDir File tableDir; - File tableDir = null; String tableLocation = null; File metadataDir = null; File versionHintFile = null; Table table = null; - @Before + @BeforeEach public void setupTable() throws Exception { - this.tableDir = temp.newFolder(); - tableDir.delete(); // created by table create - this.tableLocation = tableDir.toURI().toString(); this.metadataDir = new File(tableDir, "metadata"); this.versionHintFile = new File(metadataDir, "version-hint.text"); @@ -197,7 +193,7 @@ protected HadoopCatalog hadoopCatalog(Map catalogProperties) thr "hadoop", ImmutableMap.builder() .putAll(catalogProperties) - .put(CatalogProperties.WAREHOUSE_LOCATION, temp.newFolder().getAbsolutePath()) + .put(CatalogProperties.WAREHOUSE_LOCATION, tempDir.getAbsolutePath()) .buildOrThrow()); return hadoopCatalog; } diff --git a/core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java b/core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java index 5e323a5ffe4f..31b41434ec69 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java @@ -42,10 +42,9 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.FakeTicker; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; public class TestCachingCatalog extends HadoopTableTestBase { @@ -54,12 +53,12 @@ public class TestCachingCatalog extends HadoopTableTestBase { private FakeTicker ticker; - @Before + @BeforeEach public void beforeEach() { this.ticker = new FakeTicker(); } - @After + @AfterEach public void afterEach() { this.ticker = null; } @@ -87,15 +86,15 @@ public void testInvalidateMetadataTablesIfBaseTableIsModified() throws Exception Table manifestsMetaTable2 = catalog.loadTable(manifestsMetaTableIdent); // metadata tables are cached - Assert.assertEquals(filesMetaTable2, filesMetaTable); - Assert.assertEquals(manifestsMetaTable2, manifestsMetaTable); + Assertions.assertThat(filesMetaTable2).isEqualTo(filesMetaTable); + Assertions.assertThat(manifestsMetaTable2).isEqualTo(manifestsMetaTable); // the current snapshot of origin table is updated after committing - Assert.assertNotEquals(table.currentSnapshot(), oldSnapshot); + Assertions.assertThat(table.currentSnapshot()).isNotEqualTo(oldSnapshot); // underlying table operation in metadata tables are shared with the origin table - Assert.assertEquals(filesMetaTable2.currentSnapshot(), table.currentSnapshot()); - Assert.assertEquals(manifestsMetaTable2.currentSnapshot(), table.currentSnapshot()); + Assertions.assertThat(filesMetaTable2.currentSnapshot()).isEqualTo(table.currentSnapshot()); + Assertions.assertThat(manifestsMetaTable2.currentSnapshot()).isEqualTo(table.currentSnapshot()); } @Test @@ -129,18 +128,22 @@ public void testInvalidateMetadataTablesIfBaseTableIsDropped() throws IOExceptio // remember the new snapshot Snapshot newSnapshot = table.currentSnapshot(); - Assert.assertNotEquals("Snapshots must be different", oldSnapshot, newSnapshot); + Assertions.assertThat(newSnapshot).as("Snapshots must be different").isNotEqualTo(oldSnapshot); // validate metadata tables were correctly invalidated for (MetadataTableType type : MetadataTableType.values()) { TableIdentifier metadataIdent1 = TableIdentifier.parse(tableIdent + "." + type.name()); Table metadataTable1 = catalog.loadTable(metadataIdent1); - Assert.assertEquals("Snapshot must be new", newSnapshot, metadataTable1.currentSnapshot()); + Assertions.assertThat(metadataTable1.currentSnapshot()) + .as("Snapshot must be new") + .isEqualTo(newSnapshot); TableIdentifier metadataIdent2 = TableIdentifier.parse(tableIdent + "." + type.name().toLowerCase(Locale.ROOT)); Table metadataTable2 = catalog.loadTable(metadataIdent2); - Assert.assertEquals("Snapshot must be new", newSnapshot, metadataTable2.currentSnapshot()); + Assertions.assertThat(metadataTable2.currentSnapshot()) + .as("Snapshot must be new") + .isEqualTo(newSnapshot); } } @@ -151,13 +154,14 @@ public void testTableName() throws Exception { catalog.createTable(tableIdent, SCHEMA, SPEC, ImmutableMap.of("key2", "value2")); Table table = catalog.loadTable(tableIdent); - Assert.assertEquals("Name must match", "hadoop.db.ns1.ns2.tbl", table.name()); + Assertions.assertThat(table.name()).as("Name must match").isEqualTo("hadoop.db.ns1.ns2.tbl"); TableIdentifier snapshotsTableIdent = TableIdentifier.of("db", "ns1", "ns2", "tbl", "snapshots"); Table snapshotsTable = catalog.loadTable(snapshotsTableIdent); - Assert.assertEquals( - "Name must match", "hadoop.db.ns1.ns2.tbl.snapshots", snapshotsTable.name()); + Assertions.assertThat(snapshotsTable.name()) + .as("Name must match") + .isEqualTo("hadoop.db.ns1.ns2.tbl.snapshots"); } @Test @@ -185,10 +189,9 @@ public void testTableExpiresAfterInterval() throws IOException { ticker.advance(HALF_OF_EXPIRATION.plus(Duration.ofSeconds(10))); Assertions.assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent); - Assert.assertNotSame( - "CachingCatalog should return a new instance after expiration", - table, - catalog.loadTable(tableIdent)); + Assertions.assertThat(catalog.loadTable(tableIdent)) + .as("CachingCatalog should return a new instance after expiration") + .isNotSameAs(table); } @Test @@ -267,10 +270,9 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException .isNotEmpty() .allMatch(age -> age.isPresent() && age.get().equals(Duration.ZERO)); - Assert.assertEquals( - "Loading a non-cached metadata table should refresh the main table's age", - Optional.of(EXPIRATION_TTL), - catalog.remainingAgeFor(tableIdent)); + Assertions.assertThat(catalog.remainingAgeFor(tableIdent)) + .as("Loading a non-cached metadata table should refresh the main table's age") + .isEqualTo(Optional.of(EXPIRATION_TTL)); // Move time forward and access already cached metadata tables. ticker.advance(HALF_OF_EXPIRATION); @@ -279,10 +281,9 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException .isNotEmpty() .allMatch(age -> age.isPresent() && age.get().equals(Duration.ZERO)); - Assert.assertEquals( - "Accessing a cached metadata table should not affect the main table's age", - Optional.of(HALF_OF_EXPIRATION), - catalog.remainingAgeFor(tableIdent)); + Assertions.assertThat(catalog.remainingAgeFor(tableIdent)) + .as("Accessing a cached metadata table should not affect the main table's age") + .isEqualTo(Optional.of(HALF_OF_EXPIRATION)); // Move time forward so the data table drops. ticker.advance(HALF_OF_EXPIRATION); @@ -291,9 +292,10 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException Arrays.stream(metadataTables(tableIdent)) .forEach( metadataTable -> - Assert.assertFalse( - "When a data table expires, its metadata tables should expire regardless of age", - catalog.cache().asMap().containsKey(metadataTable))); + Assertions.assertThat(catalog.cache().asMap()) + .as( + "When a data table expires, its metadata tables should expire regardless of age") + .doesNotContainKeys(metadataTable)); } @Test @@ -357,9 +359,10 @@ public void testCacheExpirationIsDisabledByANegativeValue() throws IOException { Duration.ofMillis(CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS_OFF), ticker); - Assert.assertFalse( - "When a negative value is used as the expiration interval, the cache should not expire entries based on a TTL", - catalog.isCacheExpirationEnabled()); + Assertions.assertThat(catalog.isCacheExpirationEnabled()) + .as( + "When a negative value is used as the expiration interval, the cache should not expire entries based on a TTL") + .isFalse(); } @Test diff --git a/core/src/test/java/org/apache/iceberg/hadoop/TestCatalogUtilDropTable.java b/core/src/test/java/org/apache/iceberg/hadoop/TestCatalogUtilDropTable.java index c61bb3da6383..478ac3a8c2b4 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/TestCatalogUtilDropTable.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/TestCatalogUtilDropTable.java @@ -29,8 +29,8 @@ import org.apache.iceberg.TableProperties; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.junit.Assert; -import org.junit.Test; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; import org.mockito.ArgumentCaptor; import org.mockito.ArgumentMatchers; import org.mockito.Mockito; @@ -49,8 +49,8 @@ public void dropTableDataDeletesExpectedFiles() { Set manifestLocations = manifestLocations(snapshotSet, table.io()); Set dataLocations = dataLocations(snapshotSet, table.io()); Set metadataLocations = metadataLocations(tableMetadata); - Assert.assertEquals("should have 2 manifest lists", 2, manifestListLocations.size()); - Assert.assertEquals("should have 3 metadata locations", 3, metadataLocations.size()); + Assertions.assertThat(manifestListLocations).as("should have 2 manifest lists").hasSize(2); + Assertions.assertThat(metadataLocations).as("should have 3 metadata locations").hasSize(3); FileIO fileIO = Mockito.mock(FileIO.class); Mockito.when(fileIO.newInputFile(Mockito.anyString())) @@ -73,15 +73,18 @@ public void dropTableDataDeletesExpectedFiles() { .deleteFile(argumentCaptor.capture()); List deletedPaths = argumentCaptor.getAllValues(); - Assert.assertTrue( - "should contain all created manifest lists", - deletedPaths.containsAll(manifestListLocations)); - Assert.assertTrue( - "should contain all created manifests", deletedPaths.containsAll(manifestLocations)); - Assert.assertTrue("should contain all created data", deletedPaths.containsAll(dataLocations)); - Assert.assertTrue( - "should contain all created metadata locations", - deletedPaths.containsAll(metadataLocations)); + Assertions.assertThat(deletedPaths) + .as("should contain all created manifest lists") + .containsAll(manifestListLocations); + Assertions.assertThat(deletedPaths) + .as("should contain all created manifests") + .containsAll(manifestLocations); + Assertions.assertThat(deletedPaths) + .as("should contain all created data") + .containsAll(dataLocations); + Assertions.assertThat(deletedPaths) + .as("should contain all created metadata locations") + .containsAll(metadataLocations); } @Test @@ -124,8 +127,8 @@ public void shouldNotDropDataFilesIfGcNotEnabled() { Set manifestListLocations = manifestListLocations(snapshotSet); Set manifestLocations = manifestLocations(snapshotSet, table.io()); Set metadataLocations = metadataLocations(tableMetadata); - Assert.assertEquals("should have 2 manifest lists", 2, manifestListLocations.size()); - Assert.assertEquals("should have 4 metadata locations", 4, metadataLocations.size()); + Assertions.assertThat(manifestListLocations).as("should have 2 manifest lists").hasSize(2); + Assertions.assertThat(metadataLocations).as("should have 4 metadata locations").hasSize(4); FileIO fileIO = Mockito.mock(FileIO.class); Mockito.when(fileIO.newInputFile(Mockito.anyString())) @@ -141,14 +144,15 @@ public void shouldNotDropDataFilesIfGcNotEnabled() { .deleteFile(argumentCaptor.capture()); List deletedPaths = argumentCaptor.getAllValues(); - Assert.assertTrue( - "should contain all created manifest lists", - deletedPaths.containsAll(manifestListLocations)); - Assert.assertTrue( - "should contain all created manifests", deletedPaths.containsAll(manifestLocations)); - Assert.assertTrue( - "should contain all created metadata locations", - deletedPaths.containsAll(metadataLocations)); + Assertions.assertThat(deletedPaths) + .as("should contain all created manifest lists") + .containsAll(manifestListLocations); + Assertions.assertThat(deletedPaths) + .as("should contain all created manifests") + .containsAll(manifestLocations); + Assertions.assertThat(deletedPaths) + .as("should contain all created metadata locations") + .containsAll(metadataLocations); } private Set manifestListLocations(Set snapshotSet) { diff --git a/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCatalog.java b/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCatalog.java index 00c3058a4ccb..4f889b24cae8 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCatalog.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCatalog.java @@ -51,8 +51,7 @@ import org.apache.iceberg.transforms.Transform; import org.apache.iceberg.transforms.Transforms; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestHadoopCatalog extends HadoopTableTestBase { private static ImmutableMap meta = ImmutableMap.of(); @@ -69,10 +68,11 @@ public void testCreateTableBuilder() throws Exception { .withProperties(ImmutableMap.of("key2", "value2")) .create(); - Assert.assertEquals(TABLE_SCHEMA.toString(), table.schema().toString()); - Assert.assertEquals(1, table.spec().fields().size()); - Assert.assertEquals("value1", table.properties().get("key1")); - Assert.assertEquals("value2", table.properties().get("key2")); + Assertions.assertThat(table.schema().toString()).isEqualTo(TABLE_SCHEMA.toString()); + Assertions.assertThat(table.spec().fields()).hasSize(1); + Assertions.assertThat(table.properties()) + .containsEntry("key1", "value1") + .containsEntry("key2", "value2"); } @Test @@ -84,8 +84,8 @@ public void testCreateTableTxnBuilder() throws Exception { txn.commitTransaction(); Table table = catalog.loadTable(tableIdent); - Assert.assertEquals(TABLE_SCHEMA.toString(), table.schema().toString()); - Assert.assertTrue(table.spec().isUnpartitioned()); + Assertions.assertThat(table.schema().toString()).isEqualTo(TABLE_SCHEMA.toString()); + Assertions.assertThat(table.spec().isUnpartitioned()).isTrue(); } @Test @@ -105,23 +105,26 @@ public void testReplaceTxnBuilder() throws Exception { createTxn.commitTransaction(); Table table = catalog.loadTable(tableIdent); - Assert.assertNotNull(table.currentSnapshot()); + Assertions.assertThat(table.currentSnapshot()).isNotNull(); Transaction replaceTxn = catalog.buildTable(tableIdent, SCHEMA).withProperty("key2", "value2").replaceTransaction(); replaceTxn.commitTransaction(); table = catalog.loadTable(tableIdent); - Assert.assertNull(table.currentSnapshot()); + Assertions.assertThat(table.currentSnapshot()).isNull(); PartitionSpec v1Expected = PartitionSpec.builderFor(table.schema()) .alwaysNull("data", "data_bucket") .withSpecId(1) .build(); - Assert.assertEquals("Table should have a spec with one void field", v1Expected, table.spec()); + Assertions.assertThat(table.spec()) + .as("Table should have a spec with one void field") + .isEqualTo(v1Expected); - Assert.assertEquals("value1", table.properties().get("key1")); - Assert.assertEquals("value2", table.properties().get("key2")); + Assertions.assertThat(table.properties()) + .containsEntry("key1", "value1") + .containsEntry("key2", "value2"); } @Test @@ -155,8 +158,8 @@ public void testCreateTableDefaultSortOrder() throws Exception { Table table = hadoopCatalog().createTable(tableIdent, SCHEMA, SPEC); SortOrder sortOrder = table.sortOrder(); - Assert.assertEquals("Order ID must match", 0, sortOrder.orderId()); - Assert.assertTrue("Order must unsorted", sortOrder.isUnsorted()); + Assertions.assertThat(sortOrder.orderId()).as("Order ID must match").isEqualTo(0); + Assertions.assertThat(sortOrder.isUnsorted()).as("Order must be unsorted").isTrue(); } @Test @@ -171,13 +174,18 @@ public void testCreateTableCustomSortOrder() throws Exception { .create(); SortOrder sortOrder = table.sortOrder(); - Assert.assertEquals("Order ID must match", 1, sortOrder.orderId()); - Assert.assertEquals("Order must have 1 field", 1, sortOrder.fields().size()); - Assert.assertEquals("Direction must match ", ASC, sortOrder.fields().get(0).direction()); - Assert.assertEquals( - "Null order must match ", NULLS_FIRST, sortOrder.fields().get(0).nullOrder()); + Assertions.assertThat(sortOrder.orderId()).as("Order ID must match").isEqualTo(1); + Assertions.assertThat(sortOrder.fields().size()).as("Order must have 1 field").isEqualTo(1); + Assertions.assertThat(sortOrder.fields().get(0).direction()) + .as("Direction must match") + .isEqualTo(ASC); + Assertions.assertThat(sortOrder.fields().get(0).nullOrder()) + .as("Null order must match") + .isEqualTo(NULLS_FIRST); Transform transform = Transforms.identity(); - Assert.assertEquals("Transform must match", transform, sortOrder.fields().get(0).transform()); + Assertions.assertThat(sortOrder.fields().get(0).transform()) + .as("Transform must match") + .isEqualTo(transform); } @Test @@ -188,10 +196,10 @@ public void testBasicCatalog() throws Exception { String metaLocation = catalog.defaultWarehouseLocation(testTable); FileSystem fs = Util.getFs(new Path(metaLocation), catalog.getConf()); - Assert.assertTrue(fs.isDirectory(new Path(metaLocation))); + Assertions.assertThat(fs.isDirectory(new Path(metaLocation))).isTrue(); catalog.dropTable(testTable); - Assert.assertFalse(fs.isDirectory(new Path(metaLocation))); + Assertions.assertThat(fs.isDirectory(new Path(metaLocation))).isFalse(); } @Test @@ -201,15 +209,15 @@ public void testCreateAndDropTableWithoutNamespace() throws Exception { TableIdentifier testTable = TableIdentifier.of("tbl"); Table table = catalog.createTable(testTable, SCHEMA, PartitionSpec.unpartitioned()); - Assert.assertEquals(table.schema().toString(), TABLE_SCHEMA.toString()); - Assert.assertEquals("hadoop.tbl", table.name()); + Assertions.assertThat(table.schema().toString()).isEqualTo(TABLE_SCHEMA.toString()); + Assertions.assertThat(table.name()).isEqualTo("hadoop.tbl"); String metaLocation = catalog.defaultWarehouseLocation(testTable); FileSystem fs = Util.getFs(new Path(metaLocation), catalog.getConf()); - Assert.assertTrue(fs.isDirectory(new Path(metaLocation))); + Assertions.assertThat(fs.isDirectory(new Path(metaLocation))).isTrue(); catalog.dropTable(testTable); - Assert.assertFalse(fs.isDirectory(new Path(metaLocation))); + Assertions.assertThat(fs.isDirectory(new Path(metaLocation))).isFalse(); } @Test @@ -220,10 +228,10 @@ public void testDropTable() throws Exception { String metaLocation = catalog.defaultWarehouseLocation(testTable); FileSystem fs = Util.getFs(new Path(metaLocation), catalog.getConf()); - Assert.assertTrue(fs.isDirectory(new Path(metaLocation))); + Assertions.assertThat(fs.isDirectory(new Path(metaLocation))).isTrue(); catalog.dropTable(testTable); - Assert.assertFalse(fs.isDirectory(new Path(metaLocation))); + Assertions.assertThat(fs.isDirectory(new Path(metaLocation))).isFalse(); } @Test @@ -232,14 +240,14 @@ public void testDropNonIcebergTable() throws Exception { TableIdentifier testTable = TableIdentifier.of("db", "ns1", "ns2", "tbl"); String metaLocation = catalog.defaultWarehouseLocation(testTable); // testing with non existent directory - Assert.assertFalse(catalog.dropTable(testTable)); + Assertions.assertThat(catalog.dropTable(testTable)).isFalse(); FileSystem fs = Util.getFs(new Path(metaLocation), catalog.getConf()); fs.mkdirs(new Path(metaLocation)); - Assert.assertTrue(fs.isDirectory(new Path(metaLocation))); + Assertions.assertThat(fs.isDirectory(new Path(metaLocation))).isTrue(); - Assert.assertFalse(catalog.dropTable(testTable)); - Assert.assertTrue(fs.isDirectory(new Path(metaLocation))); + Assertions.assertThat(catalog.dropTable(testTable)).isFalse(); + Assertions.assertThat(fs.isDirectory(new Path(metaLocation))).isTrue(); } @Test @@ -267,13 +275,11 @@ public void testListTables() throws Exception { List tbls1 = catalog.listTables(Namespace.of("db")); Set tblSet = Sets.newHashSet(tbls1.stream().map(t -> t.name()).iterator()); - Assert.assertEquals(2, tblSet.size()); - Assert.assertTrue(tblSet.contains("tbl1")); - Assert.assertTrue(tblSet.contains("tbl2")); + Assertions.assertThat(tblSet).hasSize(2).contains("tbl1").contains("tbl2"); List tbls2 = catalog.listTables(Namespace.of("db", "ns1")); - Assert.assertEquals("table identifiers", 1, tbls2.size()); - Assert.assertEquals("table name", "tbl3", tbls2.get(0).name()); + Assertions.assertThat(tbls2).hasSize(1); + Assertions.assertThat(tbls2.get(0).name()).isEqualTo("tbl3"); Assertions.assertThatThrownBy(() -> catalog.listTables(Namespace.of("db", "ns1", "ns2"))) .isInstanceOf(NoSuchNamespaceException.class) @@ -289,14 +295,15 @@ public void testCallingLocationProviderWhenNoCurrentMetadata() throws IOExceptio create.table().locationProvider(); // NPE triggered if not handled appropriately create.commitTransaction(); - Assert.assertEquals( - "1 table expected", 1, catalog.listTables(Namespace.of("ns1", "ns2")).size()); + Assertions.assertThat(catalog.listTables(Namespace.of("ns1", "ns2"))) + .as("1 table expected") + .hasSize(1); catalog.dropTable(tableIdent, true); } @Test public void testCreateNamespace() throws Exception { - String warehouseLocation = temp.newFolder().getAbsolutePath(); + String warehouseLocation = tableDir.getAbsolutePath(); HadoopCatalog catalog = new HadoopCatalog(); catalog.setConf(new Configuration()); catalog.initialize( @@ -309,14 +316,14 @@ public void testCreateNamespace() throws Exception { String metaLocation1 = warehouseLocation + "/" + "db/ns1/ns2"; FileSystem fs1 = Util.getFs(new Path(metaLocation1), catalog.getConf()); - Assert.assertTrue(fs1.isDirectory(new Path(metaLocation1))); + Assertions.assertThat(fs1.isDirectory(new Path(metaLocation1))).isTrue(); String metaLocation2 = warehouseLocation + "/" + "db/ns2/ns3"; FileSystem fs2 = Util.getFs(new Path(metaLocation2), catalog.getConf()); - Assert.assertTrue(fs2.isDirectory(new Path(metaLocation2))); + Assertions.assertThat(fs2.isDirectory(new Path(metaLocation2))).isTrue(); Assertions.assertThatThrownBy(() -> catalog.createNamespace(tbl1.namespace())) - .isInstanceOf(org.apache.iceberg.exceptions.AlreadyExistsException.class) + .isInstanceOf(AlreadyExistsException.class) .hasMessage("Namespace already exists: " + tbl1.namespace()); } @@ -335,26 +342,23 @@ public void testListNamespace() throws Exception { List nsp1 = catalog.listNamespaces(Namespace.of("db")); Set tblSet = Sets.newHashSet(nsp1.stream().map(t -> t.toString()).iterator()); - Assert.assertEquals(3, tblSet.size()); - Assert.assertTrue(tblSet.contains("db.ns1")); - Assert.assertTrue(tblSet.contains("db.ns2")); - Assert.assertTrue(tblSet.contains("db.ns3")); + Assertions.assertThat(tblSet) + .hasSize(3) + .contains("db.ns1") + .contains("db.ns2") + .contains("db.ns3"); List nsp2 = catalog.listNamespaces(Namespace.of("db", "ns1")); - Assert.assertEquals(1, nsp2.size()); - Assert.assertTrue(nsp2.get(0).toString().equals("db.ns1.ns2")); + Assertions.assertThat(nsp2).hasSize(1); + Assertions.assertThat(nsp2.get(0).toString()).isEqualTo("db.ns1.ns2"); List nsp3 = catalog.listNamespaces(); Set tblSet2 = Sets.newHashSet(nsp3.stream().map(t -> t.toString()).iterator()); - Assert.assertEquals(2, tblSet2.size()); - Assert.assertTrue(tblSet2.contains("db")); - Assert.assertTrue(tblSet2.contains("db2")); + Assertions.assertThat(tblSet2).hasSize(2).contains("db").contains("db2"); List nsp4 = catalog.listNamespaces(); Set tblSet3 = Sets.newHashSet(nsp4.stream().map(t -> t.toString()).iterator()); - Assert.assertEquals(2, tblSet3.size()); - Assert.assertTrue(tblSet3.contains("db")); - Assert.assertTrue(tblSet3.contains("db2")); + Assertions.assertThat(tblSet3).hasSize(2).contains("db").contains("db2"); Assertions.assertThatThrownBy(() -> catalog.listNamespaces(Namespace.of("db", "db2", "ns2"))) .isInstanceOf(NoSuchNamespaceException.class) @@ -391,12 +395,12 @@ public void testNamespaceExists() throws IOException { Lists.newArrayList(tbl1, tbl2, tbl3, tbl4) .forEach(t -> catalog.createTable(t, SCHEMA, PartitionSpec.unpartitioned())); - Assert.assertTrue( - "Should true to namespace exist", - catalog.namespaceExists(Namespace.of("db", "ns1", "ns2"))); - Assert.assertTrue( - "Should false to namespace doesn't exist", - !catalog.namespaceExists(Namespace.of("db", "db2", "ns2"))); + Assertions.assertThat(catalog.namespaceExists(Namespace.of("db", "ns1", "ns2"))) + .as("Should be true as namespace exists") + .isTrue(); + Assertions.assertThat(catalog.namespaceExists(Namespace.of("db", "db2", "ns2"))) + .as("Should be false as namespace doesn't exist") + .isFalse(); } @Test @@ -412,7 +416,7 @@ public void testAlterNamespaceMeta() throws IOException { @Test public void testDropNamespace() throws IOException { - String warehouseLocation = temp.newFolder().getAbsolutePath(); + String warehouseLocation = tableDir.getAbsolutePath(); HadoopCatalog catalog = new HadoopCatalog(); catalog.setConf(new Configuration()); catalog.initialize( @@ -429,15 +433,16 @@ public void testDropNamespace() throws IOException { Assertions.assertThatThrownBy(() -> catalog.dropNamespace(Namespace.of("db"))) .isInstanceOf(NamespaceNotEmptyException.class) .hasMessage("Namespace " + namespace1 + " is not empty."); - Assert.assertFalse( - "Should fail to drop namespace doesn't exist", catalog.dropNamespace(Namespace.of("db2"))); - Assert.assertTrue(catalog.dropTable(tbl1)); - Assert.assertTrue(catalog.dropTable(tbl2)); - Assert.assertTrue(catalog.dropNamespace(namespace2)); - Assert.assertTrue(catalog.dropNamespace(namespace1)); + Assertions.assertThat(catalog.dropNamespace(Namespace.of("db2"))) + .as("Should fail to drop namespace that doesn't exist") + .isFalse(); + Assertions.assertThat(catalog.dropTable(tbl1)).isTrue(); + Assertions.assertThat(catalog.dropTable(tbl2)).isTrue(); + Assertions.assertThat(catalog.dropNamespace(namespace2)).isTrue(); + Assertions.assertThat(catalog.dropNamespace(namespace1)).isTrue(); String metaLocation = warehouseLocation + "/" + "db"; FileSystem fs = Util.getFs(new Path(metaLocation), catalog.getConf()); - Assert.assertFalse(fs.isDirectory(new Path(metaLocation))); + Assertions.assertThat(fs.isDirectory(new Path(metaLocation))).isFalse(); } @Test @@ -457,9 +462,9 @@ public void testVersionHintFileErrorWithFile() throws Exception { } // Check the result of the findVersion(), and load the table and check the current snapshotId - Assert.assertEquals(1, tableOperations.findVersion()); - Assert.assertEquals( - secondSnapshotId, TABLES.load(tableLocation).currentSnapshot().snapshotId()); + Assertions.assertThat(tableOperations.findVersion()).isEqualTo(1); + Assertions.assertThat(TABLES.load(tableLocation).currentSnapshot().snapshotId()) + .isEqualTo(secondSnapshotId); // Write newer data to confirm that we are writing the correct file io.deleteFile(versionHintFile.getPath()); @@ -468,26 +473,26 @@ public void testVersionHintFileErrorWithFile() throws Exception { } // Check the result of the findVersion(), and load the table and check the current snapshotId - Assert.assertEquals(3, tableOperations.findVersion()); - Assert.assertEquals( - secondSnapshotId, TABLES.load(tableLocation).currentSnapshot().snapshotId()); + Assertions.assertThat(tableOperations.findVersion()).isEqualTo(3); + Assertions.assertThat(TABLES.load(tableLocation).currentSnapshot().snapshotId()) + .isEqualTo(secondSnapshotId); // Write an empty version hint file io.deleteFile(versionHintFile.getPath()); io.newOutputFile(versionHintFile.getPath()).create().close(); // Check the result of the findVersion(), and load the table and check the current snapshotId - Assert.assertEquals(3, tableOperations.findVersion()); - Assert.assertEquals( - secondSnapshotId, TABLES.load(tableLocation).currentSnapshot().snapshotId()); + Assertions.assertThat(tableOperations.findVersion()).isEqualTo(3); + Assertions.assertThat(TABLES.load(tableLocation).currentSnapshot().snapshotId()) + .isEqualTo(secondSnapshotId); // Just delete the file io.deleteFile(versionHintFile.getPath()); // Check the result of the versionHint(), and load the table and check the current snapshotId - Assert.assertEquals(3, tableOperations.findVersion()); - Assert.assertEquals( - secondSnapshotId, TABLES.load(tableLocation).currentSnapshot().snapshotId()); + Assertions.assertThat(tableOperations.findVersion()).isEqualTo(3); + Assertions.assertThat(TABLES.load(tableLocation).currentSnapshot().snapshotId()) + .isEqualTo(secondSnapshotId); } @Test @@ -507,9 +512,9 @@ public void testVersionHintFileMissingMetadata() throws Exception { io.deleteFile(tableOperations.getMetadataFile(1).toString()); // Check the result of the findVersion(), and load the table and check the current snapshotId - Assert.assertEquals(3, tableOperations.findVersion()); - Assert.assertEquals( - secondSnapshotId, TABLES.load(tableLocation).currentSnapshot().snapshotId()); + Assertions.assertThat(tableOperations.findVersion()).isEqualTo(3); + Assertions.assertThat(TABLES.load(tableLocation).currentSnapshot().snapshotId()) + .isEqualTo(secondSnapshotId); // Remove all the version files, and see if we can recover. Hint... not :) io.deleteFile(tableOperations.getMetadataFile(2).toString()); @@ -517,7 +522,7 @@ public void testVersionHintFileMissingMetadata() throws Exception { // Check that we got 0 findVersion, and a NoSuchTableException is thrown when trying to load the // table - Assert.assertEquals(0, tableOperations.findVersion()); + Assertions.assertThat(tableOperations.findVersion()).isEqualTo(0); Assertions.assertThatThrownBy(() -> TABLES.load(tableLocation)) .isInstanceOf(NoSuchTableException.class) .hasMessageStartingWith("Table does not exist"); @@ -530,13 +535,12 @@ public void testTableName() throws Exception { catalog.buildTable(tableIdent, SCHEMA).withPartitionSpec(SPEC).create(); Table table = catalog.loadTable(tableIdent); - Assert.assertEquals("Name must match", "hadoop.db.ns1.ns2.tbl", table.name()); + Assertions.assertThat(table.name()).isEqualTo("hadoop.db.ns1.ns2.tbl"); TableIdentifier snapshotsTableIdent = TableIdentifier.of("db", "ns1", "ns2", "tbl", "snapshots"); Table snapshotsTable = catalog.loadTable(snapshotsTableIdent); - Assert.assertEquals( - "Name must match", "hadoop.db.ns1.ns2.tbl.snapshots", snapshotsTable.name()); + Assertions.assertThat(snapshotsTable.name()).isEqualTo("hadoop.db.ns1.ns2.tbl.snapshots"); } private static void addVersionsToTable(Table table) { @@ -579,28 +583,25 @@ public void testTablePropsDefinedAtCatalogLevel() throws IOException { .withProperty("key5", "table-key5") .create(); - Assert.assertEquals( - "Table defaults set for the catalog must be added to the table properties.", - "catalog-default-key1", - table.properties().get("key1")); - Assert.assertEquals( - "Table property must override table default properties set at catalog level.", - "table-key2", - table.properties().get("key2")); - Assert.assertEquals( - "Table property override set at catalog level must override table default" - + " properties set at catalog level and table property specified.", - "catalog-override-key3", - table.properties().get("key3")); - Assert.assertEquals( - "Table override not in table props or defaults should be added to table properties", - "catalog-override-key4", - table.properties().get("key4")); - Assert.assertEquals( - "Table properties without any catalog level default or override should be added to table" - + " properties.", - "table-key5", - table.properties().get("key5")); + Assertions.assertThat(table.properties().get("key1")) + .as("Table defaults set for the catalog must be added to the table properties.") + .isEqualTo("catalog-default-key1"); + Assertions.assertThat(table.properties().get("key2")) + .as("Table property must override table default properties set at catalog level.") + .isEqualTo("table-key2"); + Assertions.assertThat(table.properties().get("key3")) + .as( + "Table property override set at catalog level must override table default" + + " properties set at catalog level and table property specified.") + .isEqualTo("catalog-override-key3"); + Assertions.assertThat(table.properties().get("key4")) + .as("Table override not in table props or defaults should be added to table properties") + .isEqualTo("catalog-override-key4"); + Assertions.assertThat(table.properties().get("key5")) + .as( + "Table properties without any catalog level default or override should be added to table" + + " properties.") + .isEqualTo("table-key5"); } @Test diff --git a/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCommits.java b/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCommits.java index 31f8200ca555..b9c23f33a583 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCommits.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCommits.java @@ -21,9 +21,6 @@ import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES; import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.any; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; @@ -57,8 +54,8 @@ import org.apache.iceberg.types.Types; import org.apache.iceberg.util.Tasks; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; import org.mockito.Mockito; public class TestHadoopCommits extends HadoopTableTestBase { @@ -67,55 +64,53 @@ public class TestHadoopCommits extends HadoopTableTestBase { public void testCreateTable() throws Exception { PartitionSpec expectedSpec = PartitionSpec.builderFor(TABLE_SCHEMA).bucket("data", 16).build(); - Assert.assertEquals( - "Table schema should match schema with reassigned ids", - TABLE_SCHEMA.asStruct(), - table.schema().asStruct()); - Assert.assertEquals( - "Table partition spec should match with reassigned ids", expectedSpec, table.spec()); + Assertions.assertThat(table.schema().asStruct()) + .as("Table schema should match schema with reassigned ids") + .isEqualTo(TABLE_SCHEMA.asStruct()); + Assertions.assertThat(table.spec()) + .as("Table partition spec should match with reassigned ids") + .isEqualTo(expectedSpec); List tasks = Lists.newArrayList(table.newScan().planFiles()); - Assert.assertEquals("Should not create any scan tasks", 0, tasks.size()); - - Assert.assertTrue("Table location should exist", tableDir.exists()); - Assert.assertTrue( - "Should create metadata folder", metadataDir.exists() && metadataDir.isDirectory()); - Assert.assertTrue("Should create v1 metadata", version(1).exists() && version(1).isFile()); - Assert.assertFalse("Should not create v2 or newer versions", version(2).exists()); - Assert.assertTrue("Should create version hint file", versionHintFile.exists()); - Assert.assertEquals("Should write the current version to the hint file", 1, readVersionHint()); - + Assertions.assertThat(tasks).as("Should not create any scan tasks").isEmpty(); + Assertions.assertThat(tableDir).as("Table location should exist").exists(); + Assertions.assertThat(metadataDir).as("Should create metadata folder").exists().isDirectory(); + Assertions.assertThat(version(1)).as("Should create v1 metadata").exists().isFile(); + Assertions.assertThat(version(2)).as("Should not create v2 or newer versions").doesNotExist(); + Assertions.assertThat(versionHintFile).as("Should create version hint file").exists(); + Assertions.assertThat(readVersionHint()) + .as("Should write the current version to the hint file") + .isEqualTo(1); List manifests = listManifestFiles(); - Assert.assertEquals("Should contain 0 Avro manifest files", 0, manifests.size()); + Assertions.assertThat(manifests).as("Should contain 0 Avro manifest files").isEmpty(); } @Test public void testSchemaUpdate() throws Exception { - Assert.assertTrue("Should create v1 metadata", version(1).exists() && version(1).isFile()); - Assert.assertFalse("Should not create v2 or newer versions", version(2).exists()); + Assertions.assertThat(version(1)).as("Should create v1 metadata").exists().isFile(); + Assertions.assertThat(version(2)).as("Should not create v2 or newer versions").doesNotExist(); table.updateSchema().addColumn("n", Types.IntegerType.get()).commit(); - Assert.assertTrue( - "Should create v2 for the update", version(2).exists() && version(2).isFile()); - Assert.assertEquals("Should write the current version to the hint file", 2, readVersionHint()); - - Assert.assertEquals( - "Table schema should match schema with reassigned ids", - UPDATED_SCHEMA.asStruct(), - table.schema().asStruct()); + Assertions.assertThat(version(2)).as("Should create v2 for the update").exists().isFile(); + Assertions.assertThat(readVersionHint()) + .as("Should write the current version to the hint file") + .isEqualTo(2); + Assertions.assertThat(table.schema().asStruct()) + .as("Table schema should match schema with reassigned ids") + .isEqualTo(UPDATED_SCHEMA.asStruct()); List tasks = Lists.newArrayList(table.newScan().planFiles()); - Assert.assertEquals("Should not create any scan tasks", 0, tasks.size()); + Assertions.assertThat(tasks).as("Should not create any scan tasks").isEmpty(); List manifests = listManifestFiles(); - Assert.assertEquals("Should contain 0 Avro manifest files", 0, manifests.size()); + Assertions.assertThat(manifests).as("Should contain 0 Avro manifest files").isEmpty(); } @Test public void testSchemaUpdateComplexType() throws Exception { - Assert.assertTrue("Should create v1 metadata", version(1).exists() && version(1).isFile()); - Assert.assertFalse("Should not create v2 or newer versions", version(2).exists()); + Assertions.assertThat(version(1)).as("Should create v1 metadata").exists().isFile(); + Assertions.assertThat(version(2)).as("Should not create v2 or newer versions").doesNotExist(); Types.StructType complexColumn = Types.StructType.of( @@ -145,25 +140,25 @@ public void testSchemaUpdateComplexType() throws Exception { table.updateSchema().addColumn("complex", complexColumn).commit(); - Assert.assertTrue( - "Should create v2 for the update", version(2).exists() && version(2).isFile()); - Assert.assertEquals("Should write the current version to the hint file", 2, readVersionHint()); - Assert.assertEquals( - "Table schema should match schema with reassigned ids", - updatedSchema.asStruct(), - table.schema().asStruct()); + Assertions.assertThat(version(2)).as("Should create v2 for the update").exists().isFile(); + Assertions.assertThat(readVersionHint()) + .as("Should write the current version to the hint file") + .isEqualTo(2); + Assertions.assertThat(table.schema().asStruct()) + .as("Table schema should match schema with reassigned ids") + .isEqualTo(updatedSchema.asStruct()); List tasks = Lists.newArrayList(table.newScan().planFiles()); - Assert.assertEquals("Should not create any scan tasks", 0, tasks.size()); + Assertions.assertThat(tasks).as("Should not create any scan tasks").isEmpty(); List manifests = listManifestFiles(); - Assert.assertEquals("Should contain 0 Avro manifest files", 0, manifests.size()); + Assertions.assertThat(manifests).as("Should contain 0 Avro manifest files").isEmpty(); } @Test public void testSchemaUpdateIdentifierFields() throws Exception { - Assert.assertTrue("Should create v1 metadata", version(1).exists() && version(1).isFile()); - Assert.assertFalse("Should not create v2 or newer versions", version(2).exists()); + Assertions.assertThat(version(1)).as("Should create v1 metadata").exists().isFile(); + Assertions.assertThat(version(2)).as("Should not create v2 or newer versions").doesNotExist(); Schema updatedSchema = new Schema( @@ -174,17 +169,16 @@ public void testSchemaUpdateIdentifierFields() throws Exception { table.updateSchema().setIdentifierFields("id").commit(); - Assert.assertTrue( - "Should create v2 for the update", version(2).exists() && version(2).isFile()); - Assert.assertEquals("Should write the current version to the hint file", 2, readVersionHint()); - Assert.assertEquals( - "Table schema should match schema with reassigned ids", - updatedSchema.asStruct(), - table.schema().asStruct()); - Assert.assertEquals( - "Identifier fields should match schema with reassigned ids", - updatedSchema.identifierFieldIds(), - table.schema().identifierFieldIds()); + Assertions.assertThat(version(2)).as("Should create v2 for the update").exists().isFile(); + Assertions.assertThat(readVersionHint()) + .as("Should write the current version to the hint file") + .isEqualTo(2); + Assertions.assertThat(table.schema().asStruct()) + .as("Table schema should match schema with reassigned ids") + .isEqualTo(updatedSchema.asStruct()); + Assertions.assertThat(table.schema().identifierFieldIds()) + .as("Identifier fields should match schema with reassigned ids") + .isEqualTo(updatedSchema.identifierFieldIds()); } @Test @@ -193,8 +187,8 @@ public void testFailedCommit() throws Exception { UpdateSchema update = table.updateSchema().addColumn("n", Types.IntegerType.get()); update.apply(); - Assert.assertTrue("Should create v1 metadata", version(1).exists() && version(1).isFile()); - Assert.assertFalse("Should not create v2 or newer versions", version(2).exists()); + Assertions.assertThat(version(1)).as("Should create v1 metadata").exists().isFile(); + Assertions.assertThat(version(2)).as("Should not create v2 or newer versions").doesNotExist(); version(2).createNewFile(); @@ -203,15 +197,15 @@ public void testFailedCommit() throws Exception { .hasMessageStartingWith("Version 2 already exists"); List manifests = listManifestFiles(); - Assert.assertEquals("Should contain 0 Avro manifest files", 0, manifests.size()); + Assertions.assertThat(manifests).as("Should contain 0 Avro manifest files").isEmpty(); } @Test public void testStaleMetadata() throws Exception { Table tableCopy = TABLES.load(tableLocation); - Assert.assertTrue("Should create v1 metadata", version(1).exists() && version(1).isFile()); - Assert.assertFalse("Should not create v2 or newer versions", version(2).exists()); + Assertions.assertThat(version(1)).as("Should create v1 metadata").exists().isFile(); + Assertions.assertThat(version(2)).as("Should not create v2 or newer versions").doesNotExist(); // prepare changes on the copy without committing UpdateSchema updateCopy = tableCopy.updateSchema().addColumn("m", Types.IntegerType.get()); @@ -219,59 +213,55 @@ public void testStaleMetadata() throws Exception { table.updateSchema().addColumn("n", Types.IntegerType.get()).commit(); - Assert.assertTrue( - "Should create v2 for the update", version(2).exists() && version(2).isFile()); - Assert.assertNotEquals( - "Unmodified copy should be out of date after update", - table.schema().asStruct(), - tableCopy.schema().asStruct()); + Assertions.assertThat(version(2)).as("Should create v2 for the update").exists().isFile(); + Assertions.assertThat(table.schema().asStruct()) + .as("Unmodified copy should be out of date after update") + .isNotEqualTo(tableCopy.schema().asStruct()); // update the table tableCopy.refresh(); - Assert.assertEquals( - "Copy should be back in sync", table.schema().asStruct(), tableCopy.schema().asStruct()); + Assertions.assertThat(table.schema().asStruct()) + .as("Copy should be back in sync") + .isEqualTo(tableCopy.schema().asStruct()); Assertions.assertThatThrownBy(updateCopy::commit) .isInstanceOf(CommitFailedException.class) .hasMessage("Cannot commit changes based on stale table metadata"); List manifests = listManifestFiles(); - Assert.assertEquals("Should contain 0 Avro manifest files", 0, manifests.size()); + Assertions.assertThat(manifests).as("Should contain 0 Avro manifest files").isEmpty(); } @Test public void testStaleVersionHint() throws Exception { Table stale = TABLES.load(tableLocation); - Assert.assertTrue("Should create v1 metadata", version(1).exists() && version(1).isFile()); - Assert.assertFalse("Should not create v2 or newer versions", version(2).exists()); + Assertions.assertThat(version(1)).as("Should create v1 metadata").exists().isFile(); + Assertions.assertThat(version(2)).as("Should not create v2 or newer versions").doesNotExist(); table.updateSchema().addColumn("n", Types.IntegerType.get()).commit(); - Assert.assertTrue( - "Should create v2 for the update", version(2).exists() && version(2).isFile()); - Assert.assertEquals("Should write the current version to the hint file", 2, readVersionHint()); - - Assert.assertNotEquals( - "Stable table schema should not match", - UPDATED_SCHEMA.asStruct(), - stale.schema().asStruct()); + Assertions.assertThat(version(2)).as("Should create v2 for the update").exists().isFile(); + Assertions.assertThat(readVersionHint()) + .as("Should write the current version to the hint file") + .isEqualTo(2); + Assertions.assertThat(stale.schema().asStruct()) + .as("Stable table schema should not match") + .isNotEqualTo(UPDATED_SCHEMA.asStruct()); // roll the version hint back to 1 replaceVersionHint(1); Table reloaded = TABLES.load(tableLocation); - Assert.assertEquals( - "Updated schema for newly loaded table should match", - UPDATED_SCHEMA.asStruct(), - reloaded.schema().asStruct()); + Assertions.assertThat(reloaded.schema().asStruct()) + .as("Updated schema for newly loaded table should match") + .isEqualTo(UPDATED_SCHEMA.asStruct()); stale.refresh(); - Assert.assertEquals( - "Refreshed schema for stale table should match", - UPDATED_SCHEMA.asStruct(), - reloaded.schema().asStruct()); + Assertions.assertThat(reloaded.schema().asStruct()) + .as("Refreshed schema for stale table should match") + .isEqualTo(UPDATED_SCHEMA.asStruct()); } @Test @@ -279,33 +269,35 @@ public void testFastAppend() throws Exception { // first append table.newFastAppend().appendFile(FILE_A).commit(); - Assert.assertTrue( - "Should create v2 for the update", version(2).exists() && version(2).isFile()); - Assert.assertEquals("Should write the current version to the hint file", 2, readVersionHint()); + Assertions.assertThat(version(2)).as("Should create v2 for the update").exists().isFile(); + Assertions.assertThat(readVersionHint()) + .as("Should write the current version to the hint file") + .isEqualTo(2); List tasks = Lists.newArrayList(table.newScan().planFiles()); - Assert.assertEquals("Should scan 1 file", 1, tasks.size()); + Assertions.assertThat(tasks).as("Should scan 1 file").hasSize(1); List manifests = listManifestFiles(); - Assert.assertEquals("Should contain only one Avro manifest file", 1, manifests.size()); + Assertions.assertThat(manifests).as("Should contain only one Avro manifest file").hasSize(1); // second append table.newFastAppend().appendFile(FILE_B).commit(); - Assert.assertTrue( - "Should create v3 for the update", version(3).exists() && version(3).isFile()); - Assert.assertEquals("Should write the current version to the hint file", 3, readVersionHint()); + Assertions.assertThat(version(3)).as("Should create v3 for the update").exists().isFile(); + Assertions.assertThat(readVersionHint()) + .as("Should write the current version to the hint file") + .isEqualTo(3); tasks = Lists.newArrayList(table.newScan().planFiles()); - Assert.assertEquals("Should scan 2 files", 2, tasks.size()); - - Assert.assertEquals("Should contain 2 Avro manifest files", 2, listManifestFiles().size()); + Assertions.assertThat(tasks).as("Should scan 2 files").hasSize(2); + Assertions.assertThat(listManifestFiles()) + .as("Should contain 2 Avro manifest files") + .hasSize(2); TableMetadata metadata = readMetadataVersion(3); - Assert.assertEquals( - "Current snapshot should contain 2 manifests", - 2, - metadata.currentSnapshot().allManifests(table.io()).size()); + Assertions.assertThat(metadata.currentSnapshot().allManifests(table.io())) + .as("Current snapshot should contain 2 manifests") + .hasSize(2); } @Test @@ -319,15 +311,16 @@ public void testMergeAppend() throws Exception { table.newAppend().appendFile(FILE_C).commit(); List tasks = Lists.newArrayList(table.newScan().planFiles()); - Assert.assertEquals("Should scan 3 files", 3, tasks.size()); + Assertions.assertThat(tasks).as("Should scan 3 files").hasSize(3); - Assert.assertEquals("Should contain 3 Avro manifest files", 3, listManifestFiles().size()); + Assertions.assertThat(listManifestFiles()) + .as("Should contain 3 Avro manifest files") + .hasSize(3); TableMetadata metadata = readMetadataVersion(5); - Assert.assertEquals( - "Current snapshot should contain 1 merged manifest", - 1, - metadata.currentSnapshot().allManifests(table.io()).size()); + Assertions.assertThat(metadata.currentSnapshot().allManifests(table.io())) + .as("Current snapshot should contain 1 merged manifest") + .hasSize(1); } @Test @@ -351,9 +344,9 @@ public void testRenameThrow() throws Exception { * provided {@link FileSystem} object. The provided FileSystem will be injected for commit call. */ private void testRenameWithFileSystem(FileSystem mockFs) throws Exception { - assertTrue("Should create v1 metadata", version(1).exists() && version(1).isFile()); - assertFalse("Should not create v2 or newer versions", version(2).exists()); - assertTrue(table instanceof BaseTable); + Assertions.assertThat(version(1)).as("Should create v1 metadata").exists().isFile(); + Assertions.assertThat(version(2)).as("Should not create v2 or newer versions").doesNotExist(); + Assertions.assertThat(table).isInstanceOf(BaseTable.class); BaseTable baseTable = (BaseTable) table; // use v1 metafile as the test rename destination. TableMetadata meta1 = baseTable.operations().current(); @@ -362,12 +355,14 @@ private void testRenameWithFileSystem(FileSystem mockFs) throws Exception { // (so that we have 2 valid and different metadata files, which will reach the rename part // during commit) table.updateSchema().addColumn("n", Types.IntegerType.get()).commit(); - assertTrue("Should create v2 for the update", version(2).exists() && version(2).isFile()); - assertEquals("Should write the current version to the hint file", 2, readVersionHint()); + Assertions.assertThat(version(2)).as("Should create v2 for the update").exists().isFile(); + Assertions.assertThat(readVersionHint()) + .as("Should write the current version to the hint file") + .isEqualTo(2); // mock / spy the classes for testing TableOperations tops = baseTable.operations(); - assertTrue(tops instanceof HadoopTableOperations); + Assertions.assertThat(tops).isInstanceOf(HadoopTableOperations.class); HadoopTableOperations spyOps = Mockito.spy((HadoopTableOperations) tops); // inject the mockFS into the TableOperations @@ -379,12 +374,14 @@ private void testRenameWithFileSystem(FileSystem mockFs) throws Exception { Set actual = listMetadataJsonFiles().stream().map(File::getName).collect(Collectors.toSet()); Set expected = Sets.newHashSet("v1.metadata.json", "v2.metadata.json"); - assertEquals("only v1 and v2 metadata.json should exist.", expected, actual); + Assertions.assertThat(actual) + .as("only v1 and v2 metadata.json should exist.") + .isEqualTo(expected); } @Test public void testCanReadOldCompressedManifestFiles() throws Exception { - assertTrue("Should create v1 metadata", version(1).exists() && version(1).isFile()); + Assertions.assertThat(version(1)).as("Should create v1 metadata").exists().isFile(); // do a file append table.newAppend().appendFile(FILE_A).commit(); @@ -396,22 +393,20 @@ public void testCanReadOldCompressedManifestFiles() throws Exception { List metadataFiles = listMetadataJsonFiles(); - assertEquals("Should have two versions", 2, metadataFiles.size()); - assertTrue( - "Metadata should be compressed with old format.", - metadataFiles.stream().allMatch(f -> f.getName().endsWith(".metadata.json.gz"))); + Assertions.assertThat(metadataFiles).as("Should have two versions").hasSize(2); + Assertions.assertThat(metadataFiles.stream().map(File::getName)) + .as("Metadata should be compressed with old format.") + .allMatch(f -> f.endsWith(".metadata.json.gz")); Table reloaded = TABLES.load(tableLocation); List tasks = Lists.newArrayList(reloaded.newScan().planFiles()); - Assert.assertEquals("Should scan 1 files", 1, tasks.size()); + Assertions.assertThat(tasks).as("Should scan 1 files").hasSize(1); } @Test - public void testConcurrentFastAppends() throws Exception { - assertTrue("Should create v1 metadata", version(1).exists() && version(1).isFile()); - File dir = temp.newFolder(); - dir.delete(); + public void testConcurrentFastAppends(@TempDir File dir) throws Exception { + Assertions.assertThat(version(1)).as("Should create v1 metadata").exists().isFile(); int threadsCount = 5; int numberOfCommitedFilesPerThread = 10; Table tableWithHighRetries = @@ -453,8 +448,7 @@ public void testConcurrentFastAppends() throws Exception { }); tableWithHighRetries.refresh(); - assertEquals( - threadsCount * numberOfCommitedFilesPerThread, - Lists.newArrayList(tableWithHighRetries.snapshots()).size()); + Assertions.assertThat(Lists.newArrayList(tableWithHighRetries.snapshots())) + .hasSize(threadsCount * numberOfCommitedFilesPerThread); } } diff --git a/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopTables.java b/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopTables.java index fa80712af32e..e3d32442e2df 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopTables.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopTables.java @@ -41,11 +41,8 @@ import org.apache.iceberg.transforms.Transforms; import org.apache.iceberg.types.Types; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestHadoopTables { @@ -55,20 +52,15 @@ public class TestHadoopTables { required(1, "id", Types.IntegerType.get(), "unique ID"), required(2, "data", Types.StringType.get())); - @Rule public TemporaryFolder temp = new TemporaryFolder(); - private File tableDir = null; - - @Before - public void setupTableLocation() throws Exception { - tableDir = temp.newFolder(); - } + @TempDir private File tableDir; + @TempDir private File dataDir; @Test public void testTableExists() { - Assert.assertFalse(TABLES.exists(tableDir.toURI().toString())); + Assertions.assertThat(TABLES.exists(tableDir.toURI().toString())).isFalse(); PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).bucket("data", 16).build(); TABLES.create(SCHEMA, spec, tableDir.toURI().toString()); - Assert.assertTrue(TABLES.exists(tableDir.toURI().toString())); + Assertions.assertThat(TABLES.exists(tableDir.toURI().toString())).isTrue(); } @Test @@ -83,7 +75,6 @@ public void testDropTable() { @Test public void testDropTableWithPurge() throws IOException { - File dataDir = temp.newFolder(); createDummyTable(tableDir, dataDir); @@ -92,16 +83,13 @@ public void testDropTableWithPurge() throws IOException { .isInstanceOf(NoSuchTableException.class) .hasMessageStartingWith("Table does not exist"); - Assert.assertEquals(0, dataDir.listFiles().length); - Assert.assertFalse(tableDir.exists()); - - Assert.assertFalse(TABLES.dropTable(tableDir.toURI().toString())); + Assertions.assertThat(dataDir.listFiles()).hasSize(0); + Assertions.assertThat(tableDir).doesNotExist(); + Assertions.assertThat(TABLES.dropTable(tableDir.toURI().toString())).isFalse(); } @Test public void testDropTableWithoutPurge() throws IOException { - File dataDir = temp.newFolder(); - createDummyTable(tableDir, dataDir); TABLES.dropTable(tableDir.toURI().toString(), false); @@ -109,10 +97,9 @@ public void testDropTableWithoutPurge() throws IOException { .isInstanceOf(NoSuchTableException.class) .hasMessageStartingWith("Table does not exist"); - Assert.assertEquals(1, dataDir.listFiles().length); - Assert.assertFalse(tableDir.exists()); - - Assert.assertFalse(TABLES.dropTable(tableDir.toURI().toString())); + Assertions.assertThat(dataDir.listFiles()).hasSize(1); + Assertions.assertThat(tableDir).doesNotExist(); + Assertions.assertThat(TABLES.dropTable(tableDir.toURI().toString())).isFalse(); } @Test @@ -121,8 +108,8 @@ public void testDefaultSortOrder() { Table table = TABLES.create(SCHEMA, spec, tableDir.toURI().toString()); SortOrder sortOrder = table.sortOrder(); - Assert.assertEquals("Order ID must match", 0, sortOrder.orderId()); - Assert.assertTrue("Order must unsorted", sortOrder.isUnsorted()); + Assertions.assertThat(sortOrder.orderId()).as("Order ID must match").isEqualTo(0); + Assertions.assertThat(sortOrder.isUnsorted()).as("Order must be unsorted").isTrue(); } @Test @@ -133,13 +120,18 @@ public void testCustomSortOrder() { TABLES.create(SCHEMA, spec, order, Maps.newHashMap(), tableDir.toURI().toString()); SortOrder sortOrder = table.sortOrder(); - Assert.assertEquals("Order ID must match", 1, sortOrder.orderId()); - Assert.assertEquals("Order must have 1 field", 1, sortOrder.fields().size()); - Assert.assertEquals("Direction must match ", ASC, sortOrder.fields().get(0).direction()); - Assert.assertEquals( - "Null order must match ", NULLS_FIRST, sortOrder.fields().get(0).nullOrder()); + Assertions.assertThat(sortOrder.orderId()).as("Order ID must match").isEqualTo(1); + Assertions.assertThat(sortOrder.fields()).as("Order must have 1 field").hasSize(1); + Assertions.assertThat(sortOrder.fields().get(0).direction()) + .as("Direction must match") + .isEqualTo(ASC); + Assertions.assertThat(sortOrder.fields().get(0).nullOrder()) + .as("Null order must match") + .isEqualTo(NULLS_FIRST); Transform transform = Transforms.identity(); - Assert.assertEquals("Transform must match", transform, sortOrder.fields().get(0).transform()); + Assertions.assertThat(sortOrder.fields().get(0).transform()) + .as("Transform must match") + .isEqualTo(transform); } @Test @@ -149,10 +141,12 @@ public void testTableName() { TABLES.create(SCHEMA, spec, location); Table table = TABLES.load(location); - Assert.assertEquals("Name must match", location, table.name()); + Assertions.assertThat(table.name()).as("Name must match").isEqualTo(location); Table snapshotsTable = TABLES.load(location + "#snapshots"); - Assert.assertEquals("Name must match", location + "#snapshots", snapshotsTable.name()); + Assertions.assertThat(snapshotsTable.name()) + .as("Name must match") + .isEqualTo(location + "#snapshots"); } private static void createDummyTable(File tableDir, File dataDir) throws IOException { @@ -170,7 +164,7 @@ private static void createDummyTable(File tableDir, File dataDir) throws IOExcep append.commit(); // Make sure that the data file and the manifest dir is created - Assert.assertEquals(1, dataDir.listFiles().length); - Assert.assertEquals(1, tableDir.listFiles().length); + Assertions.assertThat(dataDir.listFiles()).hasSize(1); + Assertions.assertThat(tableDir.listFiles()).hasSize(1); } } diff --git a/core/src/test/java/org/apache/iceberg/hadoop/TestStaticTable.java b/core/src/test/java/org/apache/iceberg/hadoop/TestStaticTable.java index a0562be3eb0a..7cb57d72d359 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/TestStaticTable.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/TestStaticTable.java @@ -22,10 +22,8 @@ import org.apache.iceberg.MetadataTableType; import org.apache.iceberg.StaticTableOperations; import org.apache.iceberg.Table; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestStaticTable extends HadoopTableTestBase { @@ -41,9 +39,9 @@ private Table getStaticTable(MetadataTableType type) { @Test public void testLoadFromMetadata() { Table staticTable = getStaticTable(); - Assert.assertTrue( - "Loading a metadata file based table should return StaticTableOperations", - ((HasTableOperations) staticTable).operations() instanceof StaticTableOperations); + Assertions.assertThat(((HasTableOperations) staticTable).operations()) + .as("Loading a metadata file based table should return StaticTableOperations") + .isInstanceOf(StaticTableOperations.class); } @Test @@ -88,13 +86,13 @@ public void testHasSameProperties() { table.newAppend().appendFile(FILE_B).commit(); table.newOverwrite().deleteFile(FILE_B).addFile(FILE_C).commit(); Table staticTable = getStaticTable(); - Assert.assertTrue("Same history?", table.history().containsAll(staticTable.history())); - Assert.assertTrue( - "Same snapshot?", - table.currentSnapshot().snapshotId() == staticTable.currentSnapshot().snapshotId()); - Assert.assertTrue( - "Same properties?", - Maps.difference(table.properties(), staticTable.properties()).areEqual()); + Assertions.assertThat(table.history()).as("Same history?").containsAll(staticTable.history()); + Assertions.assertThat(table.currentSnapshot().snapshotId()) + .as("Same snapshot?") + .isEqualTo(staticTable.currentSnapshot().snapshotId()); + Assertions.assertThat(table.properties()) + .as("Same properties?") + .isEqualTo(staticTable.properties()); } @Test @@ -107,19 +105,18 @@ public void testImmutable() { table.newOverwrite().deleteFile(FILE_B).addFile(FILE_C).commit(); staticTable.refresh(); - Assert.assertEquals( - "Snapshot unchanged after table modified", - staticTable.currentSnapshot().snapshotId(), - originalSnapshot); + Assertions.assertThat(staticTable.currentSnapshot().snapshotId()) + .as("Snapshot unchanged after table modified") + .isEqualTo(originalSnapshot); } @Test public void testMetadataTables() { for (MetadataTableType type : MetadataTableType.values()) { String enumName = type.name().replace("_", "").toLowerCase(); - Assert.assertTrue( - "Should be able to get MetadataTable of type : " + type, - getStaticTable(type).getClass().getName().toLowerCase().contains(enumName)); + Assertions.assertThat(getStaticTable(type).getClass().getName().toLowerCase()) + .as("Should be able to get MetadataTable of type : " + type) + .contains(enumName); } } } diff --git a/core/src/test/java/org/apache/iceberg/hadoop/TestTableSerialization.java b/core/src/test/java/org/apache/iceberg/hadoop/TestTableSerialization.java index 380ebcac6df6..ed5b5361df01 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/TestTableSerialization.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/TestTableSerialization.java @@ -41,8 +41,8 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Test; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; public class TestTableSerialization extends HadoopTableTestBase { @@ -112,10 +112,10 @@ public void testSerializableTablePlanning() throws IOException { Set deserializedFiles = getFiles(deserialized); // Checks that the deserialized data stays the same - Assert.assertEquals(expected, deserializedFiles); + Assertions.assertThat(deserializedFiles).isEqualTo(expected); // We expect that the files changed in the meantime - Assert.assertNotEquals(getFiles(table), deserializedFiles); + Assertions.assertThat(deserializedFiles).isNotEqualTo(getFiles(table)); } @Test @@ -143,13 +143,13 @@ public void testSerializableMetadataTablesPlanning() throws IOException { Set deserializedFiles = getFiles(deserializeFromBytes(serialized.get(type))); // Checks that the deserialized data stays the same - Assert.assertEquals(expected.get(type), deserializedFiles); + Assertions.assertThat(deserializedFiles).isEqualTo(expected.get(type)); // Collect the current data Set newFiles = getFiles(getMetaDataTable(table, type)); // Expect that the new data is changed in the meantime - Assert.assertNotEquals(newFiles, deserializedFiles); + Assertions.assertThat(deserializedFiles).isNotEqualTo(newFiles); } } diff --git a/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryFileIO.java b/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryFileIO.java index 9eb2b3d5763c..5f9ec3fbf274 100644 --- a/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryFileIO.java +++ b/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryFileIO.java @@ -24,7 +24,7 @@ import org.apache.iceberg.exceptions.AlreadyExistsException; import org.apache.iceberg.exceptions.NotFoundException; import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestInMemoryFileIO { String location = "s3://foo/bar.txt"; diff --git a/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryInputFile.java b/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryInputFile.java index a796306226a5..5aa5e427c164 100644 --- a/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryInputFile.java +++ b/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryInputFile.java @@ -22,8 +22,7 @@ import java.io.InputStream; import java.nio.charset.StandardCharsets; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestInMemoryInputFile { @Test @@ -31,7 +30,7 @@ public void testReadAfterClose() throws IOException { InMemoryInputFile inputFile = new InMemoryInputFile("abc".getBytes(StandardCharsets.ISO_8859_1)); InputStream inputStream = inputFile.newStream(); - Assert.assertEquals('a', inputStream.read()); + Assertions.assertThat(inputStream.read()).isEqualTo('a'); inputStream.close(); Assertions.assertThatThrownBy(inputStream::read).hasMessage("Stream is closed"); } diff --git a/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryOutputFile.java b/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryOutputFile.java index 58094e86a05d..8015c5d1ca17 100644 --- a/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryOutputFile.java +++ b/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryOutputFile.java @@ -22,7 +22,7 @@ import java.io.OutputStream; import java.nio.charset.StandardCharsets; import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestInMemoryOutputFile { @Test diff --git a/core/src/test/java/org/apache/iceberg/io/TestByteBufferInputStreams.java b/core/src/test/java/org/apache/iceberg/io/TestByteBufferInputStreams.java index ef6ee9ae547c..408de7ce6e71 100644 --- a/core/src/test/java/org/apache/iceberg/io/TestByteBufferInputStreams.java +++ b/core/src/test/java/org/apache/iceberg/io/TestByteBufferInputStreams.java @@ -25,8 +25,7 @@ import java.util.List; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public abstract class TestByteBufferInputStreams { @@ -40,12 +39,14 @@ public void testRead0() throws Exception { ByteBufferInputStream stream = newStream(); - Assert.assertEquals("Should read 0 bytes", 0, stream.read(bytes)); + Assertions.assertThat(stream.read(bytes)).as("Should read 0 bytes").isEqualTo(0); int bytesRead = stream.read(new byte[100]); - Assert.assertTrue("Should read to end of stream", bytesRead < 100); + Assertions.assertThat(bytesRead).as("Should read to end of stream").isLessThan(100); - Assert.assertEquals("Should read 0 bytes at end of stream", 0, stream.read(bytes)); + Assertions.assertThat(stream.read(bytes)) + .as("Should read 0 bytes at end of stream") + .isEqualTo(0); } @Test @@ -55,18 +56,22 @@ public void testReadAll() throws Exception { ByteBufferInputStream stream = newStream(); int bytesRead = stream.read(bytes); - Assert.assertEquals("Should read the entire buffer", bytes.length, bytesRead); + Assertions.assertThat(bytesRead).as("Should read the entire buffer").isEqualTo(bytes.length); for (int i = 0; i < bytes.length; i += 1) { - Assert.assertEquals("Byte i should be i", i, bytes[i]); - Assert.assertEquals("Should advance position", 35, stream.getPos()); + Assertions.assertThat(bytes[i]).as("Byte i should be i").isEqualTo((byte) i); + Assertions.assertThat(stream.getPos()).as("Should advance position").isEqualTo(35); } - Assert.assertEquals("Should have no more remaining content", 0, stream.available()); + Assertions.assertThat(stream.available()) + .as("Should have no more remaining content") + .isEqualTo(0); - Assert.assertEquals("Should return -1 at end of stream", -1, stream.read(bytes)); + Assertions.assertThat(stream.read(bytes)).as("Should return -1 at end of stream").isEqualTo(-1); - Assert.assertEquals("Should have no more remaining content", 0, stream.available()); + Assertions.assertThat(stream.available()) + .as("Should have no more remaining content") + .isEqualTo(0); checkOriginalData(); } @@ -81,28 +86,37 @@ public void testSmallReads() throws Exception { int lastBytesRead = bytes.length; for (int offset = 0; offset < length; offset += bytes.length) { - Assert.assertEquals("Should read requested len", bytes.length, lastBytesRead); + Assertions.assertThat(lastBytesRead) + .as("Should read requested len") + .isEqualTo(bytes.length); lastBytesRead = stream.read(bytes, 0, bytes.length); - Assert.assertEquals("Should advance position", offset + lastBytesRead, stream.getPos()); + Assertions.assertThat(stream.getPos()) + .as("Should advance position") + .isEqualTo(offset + lastBytesRead); // validate the bytes that were read for (int i = 0; i < lastBytesRead; i += 1) { - Assert.assertEquals("Byte i should be i", offset + i, bytes[i]); + Assertions.assertThat(bytes[i]).as("Byte i should be i").isEqualTo((byte) (offset + i)); } } - Assert.assertEquals( - "Should read fewer bytes at end of buffer", - length % bytes.length, - lastBytesRead % bytes.length); + Assertions.assertThat(lastBytesRead % bytes.length) + .as("Should read fewer bytes at end of buffer") + .isEqualTo(length % bytes.length); - Assert.assertEquals("Should have no more remaining content", 0, stream.available()); + Assertions.assertThat(stream.available()) + .as("Should have no more remaining content") + .isEqualTo(0); - Assert.assertEquals("Should return -1 at end of stream", -1, stream.read(bytes)); + Assertions.assertThat(stream.read(bytes)) + .as("Should return -1 at end of stream") + .isEqualTo(-1); - Assert.assertEquals("Should have no more remaining content", 0, stream.available()); + Assertions.assertThat(stream.available()) + .as("Should have no more remaining content") + .isEqualTo(0); } checkOriginalData(); @@ -117,32 +131,40 @@ public void testPartialBufferReads() throws Exception { int lastBytesRead = size; for (int offset = 0; offset < bytes.length; offset += size) { - Assert.assertEquals("Should read requested len", size, lastBytesRead); + Assertions.assertThat(lastBytesRead).as("Should read requested len").isEqualTo(size); lastBytesRead = stream.read(bytes, offset, Math.min(size, bytes.length - offset)); - Assert.assertEquals( - "Should advance position", - lastBytesRead > 0 ? offset + lastBytesRead : offset, - stream.getPos()); + Assertions.assertThat(stream.getPos()) + .as("Should advance position") + .isEqualTo(lastBytesRead > 0 ? offset + lastBytesRead : offset); } - Assert.assertEquals( - "Should read fewer bytes at end of buffer", bytes.length % size, lastBytesRead % size); + Assertions.assertThat(lastBytesRead % size) + .as("Should read fewer bytes at end of buffer") + .isEqualTo(bytes.length % size); for (int i = 0; i < bytes.length; i += 1) { - Assert.assertEquals("Byte i should be i", i, bytes[i]); + Assertions.assertThat(bytes[i]).as("Byte i should be i").isEqualTo((byte) i); } - Assert.assertEquals("Should have no more remaining content", 2, stream.available()); + Assertions.assertThat(stream.available()) + .as("Should have no more remaining content") + .isEqualTo(2); - Assert.assertEquals("Should return 2 more bytes", 2, stream.read(bytes)); + Assertions.assertThat(stream.read(bytes)).as("Should return 2 more bytes").isEqualTo(2); - Assert.assertEquals("Should have no more remaining content", 0, stream.available()); + Assertions.assertThat(stream.available()) + .as("Should have no more remaining content") + .isEqualTo(0); - Assert.assertEquals("Should return -1 at end of stream", -1, stream.read(bytes)); + Assertions.assertThat(stream.read(bytes)) + .as("Should return -1 at end of stream") + .isEqualTo(-1); - Assert.assertEquals("Should have no more remaining content", 0, stream.available()); + Assertions.assertThat(stream.available()) + .as("Should have no more remaining content") + .isEqualTo(0); } checkOriginalData(); @@ -154,8 +176,8 @@ public void testReadByte() throws Exception { int length = stream.available(); for (int i = 0; i < length; i += 1) { - Assert.assertEquals("Position should increment", i, stream.getPos()); - Assert.assertEquals(i, stream.read()); + Assertions.assertThat(stream.getPos()).as("Position should increment").isEqualTo(i); + Assertions.assertThat(stream.read()).isEqualTo(i); } Assertions.assertThatThrownBy(stream::read).isInstanceOf(EOFException.class).hasMessage(null); @@ -170,10 +192,12 @@ public void testSlice() throws Exception { int length = stream.available(); ByteBuffer empty = stream.slice(0); - Assert.assertNotNull("slice(0) should produce a non-null buffer", empty); - Assert.assertEquals("slice(0) should produce an empty buffer", 0, empty.remaining()); - Assert.assertEquals("Position should be at start", 0, stream.getPos()); + Assertions.assertThat(empty).as("slice(0) should produce a non-null buffer").isNotNull(); + Assertions.assertThat(empty.remaining()) + .as("slice(0) should produce an empty buffer") + .isEqualTo(0); + Assertions.assertThat(stream.getPos()).as("Position should be at start").isEqualTo(0); int i = 0; while (stream.available() > 0) { @@ -181,13 +205,13 @@ public void testSlice() throws Exception { ByteBuffer buffer = stream.slice(bytesToSlice); for (int j = 0; j < bytesToSlice; j += 1) { - Assert.assertEquals("Data should be correct", i + j, buffer.get()); + Assertions.assertThat(buffer.get()).as("Data should be correct").isEqualTo((byte) (i + j)); } i += bytesToSlice; } - Assert.assertEquals("Position should be at end", length, stream.getPos()); + Assertions.assertThat(stream.getPos()).as("Position should be at end").isEqualTo(length); checkOriginalData(); } @@ -196,8 +220,9 @@ public void testSlice() throws Exception { public void testSliceBuffers0() throws Exception { ByteBufferInputStream stream = newStream(); - Assert.assertEquals( - "Should return an empty list", Collections.emptyList(), stream.sliceBuffers(0)); + Assertions.assertThat(stream.sliceBuffers(0)) + .as("Should return an empty list") + .isEqualTo(Collections.emptyList()); } @Test @@ -207,7 +232,7 @@ public void testWholeSliceBuffers() throws Exception { List buffers = stream.sliceBuffers(stream.available()); - Assert.assertEquals("Should consume all buffers", length, stream.getPos()); + Assertions.assertThat(stream.getPos()).as("Should consume all buffers").isEqualTo(length); Assertions.assertThatThrownBy(() -> stream.sliceBuffers(length)) .isInstanceOf(EOFException.class) @@ -215,7 +240,7 @@ public void testWholeSliceBuffers() throws Exception { ByteBufferInputStream copy = ByteBufferInputStream.wrap(buffers); for (int i = 0; i < length; i += 1) { - Assert.assertEquals("Slice should have identical data", i, copy.read()); + Assertions.assertThat(copy.read()).as("Slice should have identical data").isEqualTo(i); } checkOriginalData(); @@ -232,12 +257,12 @@ public void testSliceBuffersCoverage() throws Exception { buffers.addAll(stream.sliceBuffers(Math.min(size, stream.available()))); } - Assert.assertEquals("Should consume all content", length, stream.getPos()); + Assertions.assertThat(stream.getPos()).as("Should consume all content").isEqualTo(length); ByteBufferInputStream newStream = new MultiBufferInputStream(buffers); for (int i = 0; i < length; i += 1) { - Assert.assertEquals("Data should be correct", i, newStream.read()); + Assertions.assertThat(newStream.read()).as("Data should be correct").isEqualTo(i); } } @@ -251,27 +276,39 @@ public void testSliceBuffersModification() throws Exception { int sliceLength = 5; List buffers = stream.sliceBuffers(sliceLength); - Assert.assertEquals( - "Should advance the original stream", length - sliceLength, stream.available()); - Assert.assertEquals( - "Should advance the original stream position", sliceLength, stream.getPos()); - Assert.assertEquals("Should return a slice of the first buffer", 1, buffers.size()); + Assertions.assertThat(stream.available()) + .as("Should advance the original stream") + .isEqualTo(length - sliceLength); + + Assertions.assertThat(stream.getPos()) + .as("Should advance the original stream position") + .isEqualTo(sliceLength); + + Assertions.assertThat(buffers.size()) + .as("Should return a slice of the first buffer") + .isEqualTo(1); ByteBuffer buffer = buffers.get(0); - Assert.assertEquals("Should have requested bytes", sliceLength, buffer.remaining()); + + Assertions.assertThat(buffer.remaining()) + .as("Should have requested bytes") + .isEqualTo(sliceLength); // read the buffer one past the returned limit. this should not change the // next value in the original stream buffer.limit(sliceLength + 1); for (int i = 0; i < sliceLength + 1; i += 1) { - Assert.assertEquals("Should have correct data", i, buffer.get()); + Assertions.assertThat(buffer.get()).as("Should have correct data").isEqualTo((byte) i); } - Assert.assertEquals( - "Reading a slice shouldn't advance the original stream", sliceLength, stream.getPos()); - Assert.assertEquals( - "Reading a slice shouldn't change the underlying data", sliceLength, stream.read()); + Assertions.assertThat(stream.getPos()) + .as("Reading a slice shouldn't advance the original stream") + .isEqualTo(sliceLength); + + Assertions.assertThat(stream.read()) + .as("Reading a slice shouldn't change the underlying data") + .isEqualTo(sliceLength); // change the underlying data buffer buffer.limit(sliceLength + 2); @@ -281,12 +318,13 @@ public void testSliceBuffersModification() throws Exception { try { buffer.put((byte) 255); - Assert.assertEquals( - "Writing to a slice shouldn't advance the original stream", - sliceLength + 1, - stream.getPos()); - Assert.assertEquals( - "Writing to a slice should change the underlying data", 255, stream.read()); + Assertions.assertThat(stream.getPos()) + .as("Writing to a slice shouldn't advance the original stream") + .isEqualTo(sliceLength + 1); + + Assertions.assertThat(stream.read()) + .as("Writing to a slice should change the underlying data") + .isEqualTo(255); } finally { undoBuffer.put((byte) originalValue); @@ -299,16 +337,20 @@ public void testSkip() throws Exception { while (stream.available() > 0) { int bytesToSkip = Math.min(stream.available(), 10); - Assert.assertEquals( - "Should skip all, regardless of backing buffers", bytesToSkip, stream.skip(bytesToSkip)); + Assertions.assertThat(stream.skip(bytesToSkip)) + .as("Should skip all, regardless of backing buffers") + .isEqualTo(bytesToSkip); } stream = newStream(); - Assert.assertEquals(0, stream.skip(0)); + Assertions.assertThat(stream.skip(0)).isEqualTo(0); int length = stream.available(); - Assert.assertEquals("Should stop at end when out of bytes", length, stream.skip(length + 10)); - Assert.assertEquals("Should return -1 when at end", -1, stream.skip(10)); + Assertions.assertThat(stream.skip(length + 10)) + .as("Should stop at end when out of bytes") + .isEqualTo(length); + + Assertions.assertThat(stream.skip(10)).as("Should return -1 when at end").isEqualTo(-1); } @Test @@ -321,17 +363,16 @@ public void testSkipFully() throws Exception { stream.skipFully(bytesToSkip); - Assert.assertEquals( - "Should skip all, regardless of backing buffers", - bytesToSkip, - stream.getPos() - lastPosition); + Assertions.assertThat(stream.getPos() - lastPosition) + .as("Should skip all, regardless of backing buffers") + .isEqualTo(bytesToSkip); lastPosition = stream.getPos(); } ByteBufferInputStream stream2 = newStream(); stream2.skipFully(0); - Assert.assertEquals(0, stream2.getPos()); + Assertions.assertThat(stream2.getPos()).as("Check initial position").isEqualTo(0); int length = stream2.available(); @@ -356,17 +397,20 @@ public void testMark() throws Exception { stream.reset(); - Assert.assertEquals("Position should return to the mark", mark, stream.getPos()); + Assertions.assertThat(stream.getPos()).as("Position should return to the mark").isEqualTo(mark); byte[] afterReset = new byte[100]; int bytesReadAfterReset = stream.read(afterReset); - Assert.assertEquals( - "Should read the same number of bytes", expectedBytesRead, bytesReadAfterReset); + Assertions.assertThat(bytesReadAfterReset) + .as("Should read the same number of bytes") + .isEqualTo(expectedBytesRead); - Assert.assertEquals("Read should end at the same position", end, stream.getPos()); + Assertions.assertThat(stream.getPos()) + .as("Read should end at the same position") + .isEqualTo(end); - Assert.assertArrayEquals("Content should be equal", expected, afterReset); + Assertions.assertThat(afterReset).as("Content should be equal").isEqualTo(expected); } @Test @@ -386,17 +430,19 @@ public void testMarkTwice() throws Exception { stream.reset(); - Assert.assertEquals("Position should return to the mark", mark, stream.getPos()); + Assertions.assertThat(stream.getPos()).as("Position should return to the mark").isEqualTo(mark); byte[] afterReset = new byte[100]; int bytesReadAfterReset = stream.read(afterReset); + Assertions.assertThat(bytesReadAfterReset) + .as("Should read the same number of bytes") + .isEqualTo(expectedBytesRead); - Assert.assertEquals( - "Should read the same number of bytes", expectedBytesRead, bytesReadAfterReset); - - Assert.assertEquals("Read should end at the same position", end, stream.getPos()); + Assertions.assertThat(stream.getPos()) + .as("Read should end at the same position") + .isEqualTo(end); - Assert.assertArrayEquals("Content should be equal", expected, afterReset); + Assertions.assertThat(afterReset).as("Content should be equal").isEqualTo(expected); } @Test @@ -408,20 +454,22 @@ public void testMarkAtStart() throws Exception { long mark = stream.getPos(); byte[] expected = new byte[10]; - Assert.assertEquals("Should read 10 bytes", 10, stream.read(expected)); + Assertions.assertThat(stream.read(expected)).as("Should read 10 bytes").isEqualTo(10); long end = stream.getPos(); stream.reset(); - Assert.assertEquals("Position should return to the mark", mark, stream.getPos()); + Assertions.assertThat(stream.getPos()).as("Position should return to the mark").isEqualTo(mark); byte[] afterReset = new byte[10]; - Assert.assertEquals("Should read 10 bytes", 10, stream.read(afterReset)); + Assertions.assertThat(stream.read(afterReset)).as("Should read 10 bytes").isEqualTo(10); - Assert.assertEquals("Read should end at the same position", end, stream.getPos()); + Assertions.assertThat(stream.getPos()) + .as("Read should end at the same position") + .isEqualTo(end); - Assert.assertArrayEquals("Content should be equal", expected, afterReset); + Assertions.assertThat(afterReset).as("Content should be equal").isEqualTo(expected); } @Test @@ -429,27 +477,29 @@ public void testMarkAtEnd() throws Exception { ByteBufferInputStream stream = newStream(); int bytesRead = stream.read(new byte[100]); - Assert.assertTrue("Should read to end of stream", bytesRead < 100); + Assertions.assertThat(bytesRead < 100).as("Should read to end of stream").isTrue(); stream.mark(100); long mark = stream.getPos(); byte[] expected = new byte[10]; - Assert.assertEquals("Should read 0 bytes", -1, stream.read(expected)); + Assertions.assertThat(stream.read(expected)).as("Should read 0 bytes").isEqualTo(-1); long end = stream.getPos(); stream.reset(); - Assert.assertEquals("Position should return to the mark", mark, stream.getPos()); + Assertions.assertThat(stream.getPos()).as("Position should return to the mark").isEqualTo(mark); byte[] afterReset = new byte[10]; - Assert.assertEquals("Should read 0 bytes", -1, stream.read(afterReset)); + Assertions.assertThat(stream.read(afterReset)).as("Should read 0 bytes").isEqualTo(-1); - Assert.assertEquals("Read should end at the same position", end, stream.getPos()); + Assertions.assertThat(stream.getPos()) + .as("Read should end at the same position") + .isEqualTo(end); - Assert.assertArrayEquals("Content should be equal", expected, afterReset); + Assertions.assertThat(afterReset).as("Content should be equal").isEqualTo(expected); } @Test @@ -467,22 +517,28 @@ public void testMarkAndResetTwiceOverSameRange() throws Exception { byte[] expected = new byte[6]; stream.mark(10); - Assert.assertEquals("Should read expected bytes", expected.length, stream.read(expected)); + Assertions.assertThat(stream.read(expected)) + .as("Should read expected bytes") + .isEqualTo(expected.length); stream.reset(); stream.mark(10); byte[] firstRead = new byte[6]; - Assert.assertEquals("Should read firstRead bytes", firstRead.length, stream.read(firstRead)); + Assertions.assertThat(stream.read(firstRead)) + .as("Should read firstRead bytes") + .isEqualTo(firstRead.length); stream.reset(); byte[] secondRead = new byte[6]; - Assert.assertEquals("Should read secondRead bytes", secondRead.length, stream.read(secondRead)); + Assertions.assertThat(stream.read(secondRead)) + .as("Should read secondRead bytes") + .isEqualTo(secondRead.length); - Assert.assertArrayEquals("First read should be correct", expected, firstRead); + Assertions.assertThat(firstRead).as("First read should be correct").isEqualTo(expected); - Assert.assertArrayEquals("Second read should be correct", expected, secondRead); + Assertions.assertThat(secondRead).as("Second read should be correct").isEqualTo(expected); } @Test @@ -490,11 +546,11 @@ public void testMarkLimit() throws Exception { ByteBufferInputStream stream = newStream(); stream.mark(5); - Assert.assertEquals("Should read 5 bytes", 5, stream.read(new byte[5])); + Assertions.assertThat(stream.read(new byte[5])).as("Should read 5 bytes").isEqualTo(5); stream.reset(); - Assert.assertEquals("Should read 6 bytes", 6, stream.read(new byte[6])); + Assertions.assertThat(stream.read(new byte[6])).as("Should read 6 bytes").isEqualTo(6); Assertions.assertThatThrownBy(stream::reset) .isInstanceOf(IOException.class) @@ -506,7 +562,7 @@ public void testMarkDoubleReset() throws Exception { ByteBufferInputStream stream = newStream(); stream.mark(5); - Assert.assertEquals("Should read 5 bytes", 5, stream.read(new byte[5])); + Assertions.assertThat(stream.read(new byte[5])).as("Should read 5 bytes").isEqualTo(5); stream.reset(); diff --git a/core/src/test/java/org/apache/iceberg/io/TestIOUtil.java b/core/src/test/java/org/apache/iceberg/io/TestIOUtil.java index 568a5dbe9ba4..e6b4cb967df4 100644 --- a/core/src/test/java/org/apache/iceberg/io/TestIOUtil.java +++ b/core/src/test/java/org/apache/iceberg/io/TestIOUtil.java @@ -26,8 +26,7 @@ import org.apache.iceberg.inmemory.InMemoryOutputFile; import org.apache.iceberg.relocated.com.google.common.base.Strings; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestIOUtil { @Test @@ -37,11 +36,13 @@ public void testReadFully() throws Exception { MockInputStream stream = new MockInputStream(); IOUtil.readFully(stream, buffer, 0, buffer.length); - Assert.assertArrayEquals( - "Byte array contents should match", - Arrays.copyOfRange(MockInputStream.TEST_ARRAY, 0, 5), - buffer); - Assert.assertEquals("Stream position should reflect bytes read", 5, stream.getPos()); + Assertions.assertThat(buffer) + .as("Byte array contents should match") + .isEqualTo(Arrays.copyOfRange(MockInputStream.TEST_ARRAY, 0, 5)); + + Assertions.assertThat(stream.getPos()) + .as("Stream position should reflect bytes read") + .isEqualTo(5); } @Test @@ -51,11 +52,13 @@ public void testReadFullySmallReads() throws Exception { MockInputStream stream = new MockInputStream(2, 3, 3); IOUtil.readFully(stream, buffer, 0, buffer.length); - Assert.assertArrayEquals( - "Byte array contents should match", - Arrays.copyOfRange(MockInputStream.TEST_ARRAY, 0, 5), - buffer); - Assert.assertEquals("Stream position should reflect bytes read", 5, stream.getPos()); + Assertions.assertThat(buffer) + .as("Byte array contents should match") + .containsExactly(Arrays.copyOfRange(MockInputStream.TEST_ARRAY, 0, 5)); + + Assertions.assertThat(stream.getPos()) + .as("Stream position should reflect bytes read") + .isEqualTo(5); } @Test @@ -65,9 +68,13 @@ public void testReadFullyJustRight() throws Exception { final MockInputStream stream = new MockInputStream(2, 3, 3); IOUtil.readFully(stream, buffer, 0, buffer.length); - Assert.assertArrayEquals( - "Byte array contents should match", MockInputStream.TEST_ARRAY, buffer); - Assert.assertEquals("Stream position should reflect bytes read", 10, stream.getPos()); + Assertions.assertThat(buffer) + .as("Byte array contents should match") + .isEqualTo(MockInputStream.TEST_ARRAY); + + Assertions.assertThat(stream.getPos()) + .as("Stream position should reflect bytes read") + .isEqualTo(10); Assertions.assertThatThrownBy(() -> IOUtil.readFully(stream, buffer, 0, 1)) .isInstanceOf(EOFException.class) @@ -84,11 +91,13 @@ public void testReadFullyUnderflow() { .isInstanceOf(EOFException.class) .hasMessage("Reached the end of stream with 1 bytes left to read"); - Assert.assertArrayEquals( - "Should have consumed bytes", - MockInputStream.TEST_ARRAY, - Arrays.copyOfRange(buffer, 0, 10)); - Assert.assertEquals("Stream position should reflect bytes read", 10, stream.getPos()); + Assertions.assertThat(Arrays.copyOfRange(buffer, 0, 10)) + .as("Should have consumed bytes") + .isEqualTo(MockInputStream.TEST_ARRAY); + + Assertions.assertThat(stream.getPos()) + .as("Stream position should reflect bytes read") + .isEqualTo(10); } @Test @@ -98,11 +107,13 @@ public void testReadFullyStartAndLength() throws IOException { MockInputStream stream = new MockInputStream(); IOUtil.readFully(stream, buffer, 2, 5); - Assert.assertArrayEquals( - "Byte array contents should match", - Arrays.copyOfRange(MockInputStream.TEST_ARRAY, 0, 5), - Arrays.copyOfRange(buffer, 2, 7)); - Assert.assertEquals("Stream position should reflect bytes read", 5, stream.getPos()); + Assertions.assertThat(Arrays.copyOfRange(buffer, 2, 7)) + .as("Byte array contents should match") + .isEqualTo(Arrays.copyOfRange(MockInputStream.TEST_ARRAY, 0, 5)); + + Assertions.assertThat(stream.getPos()) + .as("Stream position should reflect bytes read") + .isEqualTo(5); } @Test @@ -112,7 +123,9 @@ public void testReadFullyZeroByteRead() throws IOException { MockInputStream stream = new MockInputStream(); IOUtil.readFully(stream, buffer, 0, buffer.length); - Assert.assertEquals("Stream position should reflect bytes read", 0, stream.getPos()); + Assertions.assertThat(stream.getPos()) + .as("Stream position should reflect bytes read") + .isEqualTo(0); } @Test @@ -122,11 +135,13 @@ public void testReadFullySmallReadsWithStartAndLength() throws IOException { MockInputStream stream = new MockInputStream(2, 2, 3); IOUtil.readFully(stream, buffer, 2, 5); - Assert.assertArrayEquals( - "Byte array contents should match", - Arrays.copyOfRange(MockInputStream.TEST_ARRAY, 0, 5), - Arrays.copyOfRange(buffer, 2, 7)); - Assert.assertEquals("Stream position should reflect bytes read", 5, stream.getPos()); + Assertions.assertThat(Arrays.copyOfRange(buffer, 2, 7)) + .as("Byte array contents should match") + .isEqualTo(Arrays.copyOfRange(MockInputStream.TEST_ARRAY, 0, 5)); + + Assertions.assertThat(stream.getPos()) + .as("Stream position should reflect bytes read") + .isEqualTo(5); } @Test diff --git a/core/src/test/java/org/apache/iceberg/io/TestMultiBufferInputStream.java b/core/src/test/java/org/apache/iceberg/io/TestMultiBufferInputStream.java index c4b244d01adb..99c6636f3f48 100644 --- a/core/src/test/java/org/apache/iceberg/io/TestMultiBufferInputStream.java +++ b/core/src/test/java/org/apache/iceberg/io/TestMultiBufferInputStream.java @@ -18,12 +18,13 @@ */ package org.apache.iceberg.io; +import static org.assertj.core.api.Assertions.assertThat; + import java.nio.ByteBuffer; import java.util.Arrays; import java.util.List; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestMultiBufferInputStream extends TestByteBufferInputStreams { private static final List DATA = @@ -44,8 +45,8 @@ protected ByteBufferInputStream newStream() { @Override protected void checkOriginalData() { for (ByteBuffer buffer : DATA) { - Assert.assertEquals("Position should not change", 0, buffer.position()); - Assert.assertEquals("Limit should not change", buffer.array().length, buffer.limit()); + assertThat(buffer.position()).as("Position should not change").isEqualTo(0); + assertThat(buffer.limit()).as("Limit should not change").isEqualTo(buffer.array().length); } } @@ -62,62 +63,65 @@ public void testSliceData() throws Exception { buffers.add(stream.slice(bytesToSlice)); } - Assert.assertEquals("Position should be at end", length, stream.getPos()); - Assert.assertEquals("Should produce 5 buffers", 5, buffers.size()); + assertThat(stream.getPos()).as("Position should be at end").isEqualTo(length); + assertThat(buffers.size()).as("Should produce 5 buffers").isEqualTo(5); int i = 0; // one is a view of the first buffer because it is smaller ByteBuffer one = buffers.get(0); - Assert.assertSame("Should be a duplicate of the first array", one.array(), DATA.get(0).array()); - Assert.assertEquals(8, one.remaining()); - Assert.assertEquals(0, one.position()); - Assert.assertEquals(8, one.limit()); - Assert.assertEquals(9, one.capacity()); + assertThat(DATA.get(0).array()) + .as("Should be a duplicate of the first array") + .isSameAs(one.array()); + assertThat(one.remaining()).isEqualTo(8); + assertThat(one.position()).isEqualTo(0); + assertThat(one.limit()).isEqualTo(8); + assertThat(one.capacity()).isEqualTo(9); for (; i < 8; i += 1) { - Assert.assertEquals("Should produce correct values", i, one.get()); + assertThat(one.get()).as("Should produce correct values").isEqualTo((byte) i); } // two should be a copy of the next 8 bytes ByteBuffer two = buffers.get(1); - Assert.assertEquals(8, two.remaining()); - Assert.assertEquals(0, two.position()); - Assert.assertEquals(8, two.limit()); - Assert.assertEquals(8, two.capacity()); + assertThat(two.remaining()).isEqualTo(8); + assertThat(two.position()).isEqualTo(0); + assertThat(two.limit()).isEqualTo(8); + assertThat(two.capacity()).isEqualTo(8); for (; i < 16; i += 1) { - Assert.assertEquals("Should produce correct values", i, two.get()); + assertThat(two.get()).as("Should produce correct values").isEqualTo((byte) i); } // three is a copy of part of the 4th buffer ByteBuffer three = buffers.get(2); - Assert.assertSame( - "Should be a duplicate of the fourth array", three.array(), DATA.get(3).array()); - Assert.assertEquals(8, three.remaining()); - Assert.assertEquals(3, three.position()); - Assert.assertEquals(11, three.limit()); - Assert.assertEquals(12, three.capacity()); + assertThat(DATA.get(3).array()) + .as("Should be a duplicate of the fourth array") + .isSameAs(three.array()); + assertThat(three.remaining()).isEqualTo(8); + assertThat(three.position()).isEqualTo(3); + assertThat(three.limit()).isEqualTo(11); + assertThat(three.capacity()).isEqualTo(12); for (; i < 24; i += 1) { - Assert.assertEquals("Should produce correct values", i, three.get()); + assertThat(three.get()).as("Should produce correct values").isEqualTo((byte) i); } // four should be a copy of the next 8 bytes ByteBuffer four = buffers.get(3); - Assert.assertEquals(8, four.remaining()); - Assert.assertEquals(0, four.position()); - Assert.assertEquals(8, four.limit()); - Assert.assertEquals(8, four.capacity()); + assertThat(four.remaining()).isEqualTo(8); + assertThat(four.position()).isEqualTo(0); + assertThat(four.limit()).isEqualTo(8); + assertThat(four.capacity()).isEqualTo(8); for (; i < 32; i += 1) { - Assert.assertEquals("Should produce correct values", i, four.get()); + assertThat(four.get()).as("Should produce correct values").isEqualTo((byte) i); } // five should be a copy of the next 8 bytes ByteBuffer five = buffers.get(4); - Assert.assertEquals(3, five.remaining()); - Assert.assertEquals(0, five.position()); - Assert.assertEquals(3, five.limit()); - Assert.assertEquals(3, five.capacity()); + assertThat(five.remaining()).isEqualTo(3); + assertThat(five.position()).isEqualTo(0); + assertThat(five.limit()).isEqualTo(3); + assertThat(five.capacity()).isEqualTo(3); for (; i < 35; i += 1) { - Assert.assertEquals("Should produce correct values", i, five.get()); + assertThat(five.get()).as("Should produce correct values").isEqualTo((byte) i); } } @@ -133,7 +137,8 @@ public void testSliceBuffersData() throws Exception { } } - Assert.assertEquals( - "Should return duplicates of all non-empty buffers", nonEmptyBuffers, buffers); + assertThat(buffers) + .as("Should return duplicates of all non-empty buffers") + .isEqualTo(nonEmptyBuffers); } } diff --git a/core/src/test/java/org/apache/iceberg/io/TestResolvingIO.java b/core/src/test/java/org/apache/iceberg/io/TestResolvingIO.java index fccdffeab8a4..8745c45c72ea 100644 --- a/core/src/test/java/org/apache/iceberg/io/TestResolvingIO.java +++ b/core/src/test/java/org/apache/iceberg/io/TestResolvingIO.java @@ -18,11 +18,12 @@ */ package org.apache.iceberg.io; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; import org.apache.iceberg.TestHelpers; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestResolvingIO { @@ -34,8 +35,7 @@ public void testResolvingFileIOKryoSerialization() throws IOException { testResolvingFileIO.initialize(ImmutableMap.of("k1", "v1")); FileIO roundTripSerializedFileIO = TestHelpers.KryoHelpers.roundTripSerialize(testResolvingFileIO); - - Assert.assertEquals(testResolvingFileIO.properties(), roundTripSerializedFileIO.properties()); + assertThat(roundTripSerializedFileIO.properties()).isEqualTo(testResolvingFileIO.properties()); } @Test @@ -45,7 +45,6 @@ public void testResolvingFileIOJavaSerialization() throws IOException, ClassNotF // resolving fileIO should be serializable when properties are passed as immutable map testResolvingFileIO.initialize(ImmutableMap.of("k1", "v1")); FileIO roundTripSerializedFileIO = TestHelpers.roundTripSerialize(testResolvingFileIO); - - Assert.assertEquals(testResolvingFileIO.properties(), roundTripSerializedFileIO.properties()); + assertThat(roundTripSerializedFileIO.properties()).isEqualTo(testResolvingFileIO.properties()); } } diff --git a/core/src/test/java/org/apache/iceberg/io/TestSingleBufferInputStream.java b/core/src/test/java/org/apache/iceberg/io/TestSingleBufferInputStream.java index 5715f3f74e29..7b91cf9b2da8 100644 --- a/core/src/test/java/org/apache/iceberg/io/TestSingleBufferInputStream.java +++ b/core/src/test/java/org/apache/iceberg/io/TestSingleBufferInputStream.java @@ -18,12 +18,13 @@ */ package org.apache.iceberg.io; +import static org.assertj.core.api.Assertions.assertThat; + import java.nio.ByteBuffer; import java.util.Collections; import java.util.List; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestSingleBufferInputStream extends TestByteBufferInputStreams { private static final ByteBuffer DATA = @@ -40,8 +41,8 @@ protected ByteBufferInputStream newStream() { @Override protected void checkOriginalData() { - Assert.assertEquals("Position should not change", 0, DATA.position()); - Assert.assertEquals("Limit should not change", DATA.array().length, DATA.limit()); + assertThat(DATA.position()).as("Position should not change").isEqualTo(0); + assertThat(DATA.limit()).as("Limit should not change").isEqualTo(DATA.array().length); } @Test @@ -57,62 +58,61 @@ public void testSliceData() throws Exception { buffers.add(stream.slice(bytesToSlice)); } - Assert.assertEquals("Position should be at end", length, stream.getPos()); - Assert.assertEquals("Should produce 5 buffers", 5, buffers.size()); + assertThat(stream.getPos()).as("Position should be at end").isEqualTo(length); + assertThat(buffers.size()).as("Should produce 5 buffers").isEqualTo(5); int i = 0; ByteBuffer one = buffers.get(0); - Assert.assertSame("Should use the same backing array", one.array(), DATA.array()); - Assert.assertEquals(8, one.remaining()); - Assert.assertEquals(0, one.position()); - Assert.assertEquals(8, one.limit()); - Assert.assertEquals(35, one.capacity()); + assertThat(one.array()).isSameAs(DATA.array()); + assertThat(one.remaining()).isEqualTo(8); + assertThat(one.position()).isEqualTo(0); + assertThat(one.limit()).isEqualTo(8); + assertThat(one.capacity()).isEqualTo(35); for (; i < 8; i += 1) { - Assert.assertEquals("Should produce correct values", i, one.get()); + assertThat(one.get()).as("Should produce correct values").isEqualTo((byte) i); } ByteBuffer two = buffers.get(1); - Assert.assertSame("Should use the same backing array", two.array(), DATA.array()); - Assert.assertEquals(8, two.remaining()); - Assert.assertEquals(8, two.position()); - Assert.assertEquals(16, two.limit()); - Assert.assertEquals(35, two.capacity()); + assertThat(two.array()).as("Should use the same backing array").isSameAs(DATA.array()); + assertThat(two.remaining()).isEqualTo(8); + assertThat(two.position()).isEqualTo(8); + assertThat(two.limit()).isEqualTo(16); + assertThat(two.capacity()).isEqualTo(35); for (; i < 16; i += 1) { - Assert.assertEquals("Should produce correct values", i, two.get()); + assertThat(two.get()).as("Should produce correct values").isEqualTo((byte) i); } - // three is a copy of part of the 4th buffer ByteBuffer three = buffers.get(2); - Assert.assertSame("Should use the same backing array", three.array(), DATA.array()); - Assert.assertEquals(8, three.remaining()); - Assert.assertEquals(16, three.position()); - Assert.assertEquals(24, three.limit()); - Assert.assertEquals(35, three.capacity()); + assertThat(three.array()).as("Should use the same backing array").isSameAs(DATA.array()); + assertThat(three.remaining()).isEqualTo(8); + assertThat(three.position()).isEqualTo(16); + assertThat(three.limit()).isEqualTo(24); + assertThat(three.capacity()).isEqualTo(35); for (; i < 24; i += 1) { - Assert.assertEquals("Should produce correct values", i, three.get()); + assertThat(three.get()).as("Should produce correct values").isEqualTo((byte) i); } // four should be a copy of the next 8 bytes ByteBuffer four = buffers.get(3); - Assert.assertSame("Should use the same backing array", four.array(), DATA.array()); - Assert.assertEquals(8, four.remaining()); - Assert.assertEquals(24, four.position()); - Assert.assertEquals(32, four.limit()); - Assert.assertEquals(35, four.capacity()); + assertThat(four.array()).as("Should use the same backing array").isSameAs(DATA.array()); + assertThat(four.remaining()).isEqualTo(8); + assertThat(four.position()).isEqualTo(24); + assertThat(four.limit()).isEqualTo(32); + assertThat(four.capacity()).isEqualTo(35); for (; i < 32; i += 1) { - Assert.assertEquals("Should produce correct values", i, four.get()); + assertThat(four.get()).as("Should produce correct values").isEqualTo((byte) i); } // five should be a copy of the next 8 bytes ByteBuffer five = buffers.get(4); - Assert.assertSame("Should use the same backing array", five.array(), DATA.array()); - Assert.assertEquals(3, five.remaining()); - Assert.assertEquals(32, five.position()); - Assert.assertEquals(35, five.limit()); - Assert.assertEquals(35, five.capacity()); + assertThat(five.array()).as("Should use the same backing array").isSameAs(DATA.array()); + assertThat(five.remaining()).isEqualTo(3); + assertThat(five.position()).isEqualTo(32); + assertThat(five.limit()).isEqualTo(35); + assertThat(five.capacity()).isEqualTo(35); for (; i < 35; i += 1) { - Assert.assertEquals("Should produce correct values", i, five.get()); + assertThat(five.get()).as("Should produce correct values").isEqualTo((byte) i); } } @@ -121,9 +121,8 @@ public void testWholeSliceBuffersData() throws Exception { ByteBufferInputStream stream = newStream(); List buffers = stream.sliceBuffers(stream.available()); - Assert.assertEquals( - "Should return duplicates of all non-empty buffers", - Collections.singletonList(DATA), - buffers); + assertThat(buffers) + .as("Should return duplicates of all non-empty buffers") + .isEqualTo(Collections.singletonList(DATA)); } } diff --git a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java index 644b2ff5265d..4634de57073d 100644 --- a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java +++ b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.NullOrder.NULLS_FIRST; import static org.apache.iceberg.SortDirection.ASC; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; @@ -43,13 +44,11 @@ import org.apache.iceberg.DataFiles; import org.apache.iceberg.FileFormat; import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.SortOrder; import org.apache.iceberg.Table; import org.apache.iceberg.TableOperations; -import org.apache.iceberg.TestHelpers; import org.apache.iceberg.Transaction; import org.apache.iceberg.catalog.CatalogTests; import org.apache.iceberg.catalog.Namespace; @@ -71,7 +70,6 @@ import org.apache.iceberg.transforms.Transforms; import org.apache.iceberg.types.Types; import org.assertj.core.api.Assertions; -import org.junit.Assert; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -170,10 +168,9 @@ public void testCreateTableBuilder() { .withProperties(ImmutableMap.of("key2", "value2")) .create(); - Assert.assertEquals(SCHEMA.toString(), table.schema().toString()); - Assert.assertEquals(1, table.spec().fields().size()); - Assert.assertEquals("value1", table.properties().get("key1")); - Assert.assertEquals("value2", table.properties().get("key2")); + assertThat(table.schema()).hasToString(SCHEMA.toString()); + assertThat(table.spec().fields()).hasSize(1); + assertThat(table.properties()).containsEntry("key1", "value1").containsEntry("key2", "value2"); } @Test @@ -188,9 +185,9 @@ public void testCreateTableTxnBuilder() { txn.commitTransaction(); Table table = catalog.loadTable(tableIdent); - Assert.assertEquals(SCHEMA.toString(), table.schema().toString()); - Assert.assertTrue(table.spec().isUnpartitioned()); - Assert.assertEquals("testval1", table.properties().get("key1")); + assertThat(table.schema()).hasToString(SCHEMA.toString()); + assertThat(table.spec().isUnpartitioned()).isTrue(); + assertThat(table.properties()).containsEntry("key1", "testval1"); } @Test @@ -217,23 +214,24 @@ public void testReplaceTxnBuilder() { createTxn.commitTransaction(); Table table = catalog.loadTable(tableIdent); - Assert.assertNotNull(table.currentSnapshot()); + assertThat(table.currentSnapshot()).isNotNull(); Transaction replaceTxn = catalog.buildTable(tableIdent, SCHEMA).withProperty("key2", "value2").replaceTransaction(); replaceTxn.commitTransaction(); table = catalog.loadTable(tableIdent); - Assert.assertNull(table.currentSnapshot()); + assertThat(table.currentSnapshot()).isNull(); PartitionSpec v1Expected = PartitionSpec.builderFor(table.schema()) .alwaysNull("data", "data_bucket") .withSpecId(1) .build(); - Assert.assertEquals("Table should have a spec with one void field", v1Expected, table.spec()); + assertThat(table.spec()) + .as("Table should have a spec with one void field") + .isEqualTo(v1Expected); - Assert.assertEquals("value1", table.properties().get("key1")); - Assert.assertEquals("value2", table.properties().get("key2")); + assertThat(table.properties()).containsEntry("key1", "value1").containsEntry("key2", "value2"); } @Test @@ -242,8 +240,8 @@ public void testCreateTableDefaultSortOrder() { Table table = catalog.createTable(tableIdent, SCHEMA, PARTITION_SPEC); SortOrder sortOrder = table.sortOrder(); - Assert.assertEquals("Order ID must match", 0, sortOrder.orderId()); - Assert.assertTrue("Order must unsorted", sortOrder.isUnsorted()); + assertThat(sortOrder.orderId()).as("Order ID must match").isZero(); + assertThat(sortOrder.isUnsorted()).as("Order must unsorted").isTrue(); } @Test @@ -258,13 +256,16 @@ public void testCreateTableCustomSortOrder() { .create(); SortOrder sortOrder = table.sortOrder(); - Assert.assertEquals("Order ID must match", 1, sortOrder.orderId()); - Assert.assertEquals("Order must have 1 field", 1, sortOrder.fields().size()); - Assert.assertEquals("Direction must match ", ASC, sortOrder.fields().get(0).direction()); - Assert.assertEquals( - "Null order must match ", NULLS_FIRST, sortOrder.fields().get(0).nullOrder()); + assertThat(sortOrder.orderId()).as("Order ID must match").isEqualTo(1); + assertThat(sortOrder.fields()).as("Order must have 1 field").hasSize(1); + assertThat(sortOrder.fields().get(0).direction()).as("Direction must match ").isEqualTo(ASC); + assertThat(sortOrder.fields().get(0).nullOrder()) + .as("Null order must match ") + .isEqualTo(NULLS_FIRST); Transform transform = Transforms.identity(); - Assert.assertEquals("Transform must match", transform, sortOrder.fields().get(0).transform()); + assertThat(sortOrder.fields().get(0).transform()) + .as("Transform must match") + .isEqualTo(transform); } @Test @@ -274,7 +275,7 @@ public void testBasicCatalog() throws Exception { String metaLocation = catalog.defaultWarehouseLocation(testTable); FileSystem fs = Util.getFs(new Path(metaLocation), conf); - Assert.assertTrue(fs.isDirectory(new Path(metaLocation))); + assertThat(fs.isDirectory(new Path(metaLocation))).isTrue(); Assertions.assertThatThrownBy( () -> catalog.createTable(testTable, SCHEMA, PartitionSpec.unpartitioned())) @@ -289,12 +290,12 @@ public void testCreateAndDropTableWithoutNamespace() throws Exception { TableIdentifier testTable = TableIdentifier.of("tbl"); Table table = catalog.createTable(testTable, SCHEMA, PartitionSpec.unpartitioned()); - Assert.assertEquals(table.schema().toString(), SCHEMA.toString()); - Assert.assertEquals(catalog.name() + ".tbl", table.name()); + assertThat(SCHEMA).hasToString(table.schema().toString()); + assertThat(table.name()).isEqualTo(catalog.name() + ".tbl"); String metaLocation = catalog.defaultWarehouseLocation(testTable); FileSystem fs = Util.getFs(new Path(metaLocation), conf); - Assert.assertTrue(fs.isDirectory(new Path(metaLocation))); + assertThat(fs.isDirectory(new Path(metaLocation))).isTrue(); catalog.dropTable(testTable, true); } @@ -303,11 +304,10 @@ public void testCreateAndDropTableWithoutNamespace() throws Exception { public void testDefaultWarehouseLocation() throws Exception { TableIdentifier testTable = TableIdentifier.of("tbl"); TableIdentifier testTable2 = TableIdentifier.of(Namespace.of("ns"), "tbl"); - Assert.assertEquals( - catalog.defaultWarehouseLocation(testTable), warehouseLocation + "/" + testTable.name()); - Assert.assertEquals( - catalog.defaultWarehouseLocation(testTable2), - warehouseLocation + "/" + testTable2.namespace() + "/" + testTable2.name()); + assertThat(warehouseLocation + "/" + testTable.name()) + .isEqualTo(catalog.defaultWarehouseLocation(testTable)); + assertThat(warehouseLocation + "/" + testTable2.namespace() + "/" + testTable2.name()) + .isEqualTo(catalog.defaultWarehouseLocation(testTable2)); } @Test @@ -324,7 +324,7 @@ public void testConcurrentCommit() throws IOException { .withRecordCount(1) .build(); table.newAppend().appendFile(dataFile).commit(); - Assert.assertEquals(1, table.history().size()); + assertThat(table.history()).hasSize(1); catalog.dropTable(tableIdentifier); data = tableDir.resolve("data2.parquet").toAbsolutePath().toString(); Files.write(Paths.get(data), Lists.newArrayList(), StandardCharsets.UTF_8); @@ -356,7 +356,7 @@ public void testCommitHistory() throws IOException { .withRecordCount(1) .build(); table.newAppend().appendFile(dataFile).commit(); - Assert.assertEquals(1, table.history().size()); + assertThat(table.history()).hasSize(1); data = tableDir.resolve("data2.parquet").toAbsolutePath().toString(); Files.write(Paths.get(data), Lists.newArrayList(), StandardCharsets.UTF_8); @@ -367,7 +367,7 @@ public void testCommitHistory() throws IOException { .withRecordCount(1) .build(); table.newAppend().appendFile(dataFile).commit(); - Assert.assertEquals(2, table.history().size()); + assertThat(table.history()).hasSize(2); data = tableDir.resolve("data3.parquet").toAbsolutePath().toString(); Files.write(Paths.get(data), Lists.newArrayList(), StandardCharsets.UTF_8); @@ -378,7 +378,7 @@ public void testCommitHistory() throws IOException { .withRecordCount(1) .build(); table.newAppend().appendFile(dataFile).commit(); - Assert.assertEquals(3, table.history().size()); + assertThat(table.history()).hasSize(3); } @Test @@ -388,14 +388,14 @@ public void testDropTable() { catalog.createTable(testTable, SCHEMA, PartitionSpec.unpartitioned()); catalog.createTable(testTable2, SCHEMA, PartitionSpec.unpartitioned()); catalog.dropTable(testTable); - Assert.assertFalse(catalog.listTables(testTable.namespace()).contains(testTable)); + assertThat(catalog.listTables(testTable.namespace())).doesNotContain(testTable); catalog.dropTable(testTable2); Assertions.assertThatThrownBy(() -> catalog.listTables(testTable2.namespace())) .isInstanceOf(NoSuchNamespaceException.class) .hasMessage("Namespace does not exist: db.ns1.ns2"); - Assert.assertFalse(catalog.dropTable(TableIdentifier.of("db", "tbl-not-exists"))); + assertThat(catalog.dropTable(TableIdentifier.of("db", "tbl-not-exists"))).isFalse(); } @Test @@ -405,7 +405,7 @@ public void testDropTableWithoutMetadataFile() { String metadataFileLocation = catalog.newTableOps(testTable).current().metadataFileLocation(); TableOperations ops = catalog.newTableOps(testTable); ops.io().deleteFile(metadataFileLocation); - Assert.assertTrue(catalog.dropTable(testTable)); + assertThat(catalog.dropTable(testTable)).isTrue(); assertThatThrownBy(() -> catalog.loadTable(testTable)) .isInstanceOf(NoSuchTableException.class) .hasMessageContaining("Table does not exist:"); @@ -417,9 +417,8 @@ public void testRenameTable() { TableIdentifier to = TableIdentifier.of("db", "tbl2-newtable"); catalog.createTable(from, SCHEMA, PartitionSpec.unpartitioned()); catalog.renameTable(from, to); - Assert.assertTrue(catalog.listTables(to.namespace()).contains(to)); - Assert.assertFalse(catalog.listTables(to.namespace()).contains(from)); - Assert.assertTrue(catalog.loadTable(to).name().endsWith(to.name())); + assertThat(catalog.listTables(to.namespace())).contains(to).doesNotContain(from); + assertThat(catalog.loadTable(to).name()).endsWith(to.name()); Assertions.assertThatThrownBy( () -> catalog.renameTable(TableIdentifier.of("db", "tbl-not-exists"), to)) @@ -447,13 +446,11 @@ public void testListTables() { List tbls1 = catalog.listTables(Namespace.of("db")); Set tblSet = Sets.newHashSet(tbls1.stream().map(TableIdentifier::name).iterator()); - Assert.assertEquals(tblSet.size(), 2); - Assert.assertTrue(tblSet.contains("tbl1")); - Assert.assertTrue(tblSet.contains("tbl2")); + assertThat(tblSet).hasSize(2).contains("tbl1", "tbl2"); List tbls2 = catalog.listTables(Namespace.of("db", "ns1")); - Assert.assertEquals(tbls2.size(), 1); - Assert.assertEquals("tbl3", tbls2.get(0).name()); + assertThat(tbls2).hasSize(1); + assertThat(tbls2.get(0).name()).isEqualTo("tbl3"); Assertions.assertThatThrownBy(() -> catalog.listTables(Namespace.of("db", "ns1", "ns2"))) .isInstanceOf(NoSuchNamespaceException.class) @@ -467,8 +464,7 @@ public void testCallingLocationProviderWhenNoCurrentMetadata() { create.table().locationProvider(); // NPE triggered if not handled appropriately create.commitTransaction(); - Assert.assertEquals( - "1 table expected", 1, catalog.listTables(Namespace.of("ns1", "ns2")).size()); + assertThat(catalog.listTables(Namespace.of("ns1", "ns2"))).as("1 table expected").hasSize(1); catalog.dropTable(tableIdent, true); } @@ -483,10 +479,10 @@ public void testExistingTableUpdate() { icebergTable.updateSchema().addColumn("Coll3", Types.LongType.get()).commit(); icebergTable = catalog.loadTable(tableIdent); // Only 2 snapshotFile Should exist and no manifests should exist - Assert.assertEquals(2, metadataVersionFiles(icebergTable.location() + "/metadata/").size()); - Assert.assertEquals(0, manifestFiles(icebergTable.location() + "/metadata/").size()); - Assert.assertNotEquals(SCHEMA.asStruct(), icebergTable.schema().asStruct()); - Assert.assertTrue(icebergTable.schema().asStruct().toString().contains("Coll3")); + assertThat(metadataVersionFiles(icebergTable.location() + "/metadata/")).hasSize(2); + assertThat(manifestFiles(icebergTable.location() + "/metadata/")).isEmpty(); + assertThat(icebergTable.schema().asStruct()).isNotEqualTo(SCHEMA.asStruct()); + assertThat(icebergTable.schema().asStruct().toString()).contains("Coll3"); } @Test @@ -494,13 +490,14 @@ public void testTableName() { TableIdentifier tableIdent = TableIdentifier.of("db", "ns1", "ns2", "tbl"); catalog.buildTable(tableIdent, SCHEMA).withPartitionSpec(PARTITION_SPEC).create(); Table table = catalog.loadTable(tableIdent); - Assert.assertEquals("Name must match", catalog.name() + ".db.ns1.ns2.tbl", table.name()); + assertThat(table.name()).as("Name must match").isEqualTo(catalog.name() + ".db.ns1.ns2.tbl"); TableIdentifier snapshotsTableIdent = TableIdentifier.of("db", "ns1", "ns2", "tbl", "snapshots"); Table snapshotsTable = catalog.loadTable(snapshotsTableIdent); - Assert.assertEquals( - "Name must match", catalog.name() + ".db.ns1.ns2.tbl.snapshots", snapshotsTable.name()); + assertThat(snapshotsTable.name()) + .as("Name must match") + .isEqualTo(catalog.name() + ".db.ns1.ns2.tbl.snapshots"); } @Test @@ -516,30 +513,21 @@ public void testListNamespace() { .forEach(t -> catalog.createTable(t, SCHEMA, PartitionSpec.unpartitioned())); List nsp1 = catalog.listNamespaces(Namespace.of("db")); - Assert.assertEquals(nsp1.size(), 3); + assertThat(nsp1).hasSize(3); Set tblSet = Sets.newHashSet(nsp1.stream().map(Namespace::toString).iterator()); - Assert.assertEquals(tblSet.size(), 3); - Assert.assertTrue(tblSet.contains("db.ns1")); - Assert.assertTrue(tblSet.contains("db.ns2")); - Assert.assertTrue(tblSet.contains("db.ns3")); + assertThat(tblSet).hasSize(3).contains("db.ns1", "db.ns2", "db.ns3"); List nsp2 = catalog.listNamespaces(Namespace.of("db", "ns1")); - Assert.assertEquals(nsp2.size(), 1); - Assert.assertEquals("db.ns1.ns2", nsp2.get(0).toString()); + assertThat(nsp2).hasSize(1); + assertThat(nsp2.get(0)).hasToString("db.ns1.ns2"); List nsp3 = catalog.listNamespaces(); Set tblSet2 = Sets.newHashSet(nsp3.stream().map(Namespace::toString).iterator()); - Assert.assertEquals(tblSet2.size(), 3); - Assert.assertTrue(tblSet2.contains("db")); - Assert.assertTrue(tblSet2.contains("db2")); - Assert.assertTrue(tblSet2.contains("")); + assertThat(tblSet2).hasSize(3).contains("db", "db2", ""); List nsp4 = catalog.listNamespaces(); Set tblSet3 = Sets.newHashSet(nsp4.stream().map(Namespace::toString).iterator()); - Assert.assertEquals(tblSet3.size(), 3); - Assert.assertTrue(tblSet3.contains("db")); - Assert.assertTrue(tblSet3.contains("db2")); - Assert.assertTrue(tblSet3.contains("")); + assertThat(tblSet3).hasSize(3).contains("db", "db2", ""); Assertions.assertThatThrownBy(() -> catalog.listNamespaces(Namespace.of("db", "db2", "ns2"))) .isInstanceOf(NoSuchNamespaceException.class) @@ -556,7 +544,7 @@ public void testLoadNamespaceMeta() { Lists.newArrayList(tbl1, tbl2, tbl3, tbl4) .forEach(t -> catalog.createTable(t, SCHEMA, PartitionSpec.unpartitioned())); - Assert.assertTrue(catalog.loadNamespaceMetadata(Namespace.of("db")).containsKey("location")); + assertThat(catalog.loadNamespaceMetadata(Namespace.of("db"))).containsKey("location"); Assertions.assertThatThrownBy( () -> catalog.loadNamespaceMetadata(Namespace.of("db", "db2", "ns2"))) @@ -573,19 +561,19 @@ public void testNamespaceExists() { Lists.newArrayList(tbl1, tbl2, tbl3, tbl4) .forEach(t -> catalog.createTable(t, SCHEMA, PartitionSpec.unpartitioned())); - Assert.assertTrue( - "Should true to namespace exist", - catalog.namespaceExists(Namespace.of("db", "ns1", "ns2"))); - Assert.assertFalse( - "Should false to namespace doesn't exist", - catalog.namespaceExists(Namespace.of("db", "db2", "not_exist"))); + assertThat(catalog.namespaceExists(Namespace.of("db", "ns1", "ns2"))) + .as("Should true to namespace exist") + .isTrue(); + assertThat(catalog.namespaceExists(Namespace.of("db", "db2", "not_exist"))) + .as("Should false to namespace doesn't exist") + .isFalse(); } @Test public void testDropNamespace() { - Assert.assertFalse( - "Should return false if drop does not modify state", - catalog.dropNamespace(Namespace.of("db", "ns1_not_exitss"))); + assertThat(catalog.dropNamespace(Namespace.of("db", "ns1_not_exitss"))) + .as("Should return false if drop does not modify state") + .isFalse(); TableIdentifier tbl0 = TableIdentifier.of("db", "ns1", "ns2", "tbl2"); TableIdentifier tbl1 = TableIdentifier.of("db", "ns1", "ns2", "tbl1"); @@ -612,10 +600,10 @@ public void testDropNamespace() { @Test public void testCreateNamespace() { Namespace testNamespace = Namespace.of("testDb", "ns1", "ns2"); - Assert.assertFalse(catalog.namespaceExists(testNamespace)); + assertThat(catalog.namespaceExists(testNamespace)).isFalse(); // Test with no metadata catalog.createNamespace(testNamespace); - Assert.assertTrue(catalog.namespaceExists(testNamespace)); + assertThat(catalog.namespaceExists(testNamespace)).isTrue(); } @Test @@ -664,7 +652,7 @@ public void testCreateNamespaceWithMetadata() { Map testMetadata = ImmutableMap.of("key_1", "value_1", "key_2", "value_2", "key_3", "value_3"); catalog.createNamespace(testNamespace, testMetadata); - Assert.assertTrue(catalog.namespaceExists(testNamespace)); + assertThat(catalog.namespaceExists(testNamespace)).isTrue(); } @Test @@ -687,8 +675,8 @@ public void testNamespaceCustomLocation() { Map testMetadata = ImmutableMap.of("location", namespaceLocation); catalog.createNamespace(testNamespace, testMetadata); - Assertions.assertThat(catalog.loadNamespaceMetadata(testNamespace).get("location")) - .isEqualTo(namespaceLocation); + Assertions.assertThat(catalog.loadNamespaceMetadata(testNamespace)) + .containsEntry("location", namespaceLocation); } @Test @@ -711,23 +699,22 @@ public void testSetProperties() { "value_4", "key_2", "new_value_2"); - Assert.assertTrue(catalog.namespaceExists(testNamespace)); - Assert.assertTrue(catalog.setProperties(testNamespace, propertiesToSet)); + assertThat(catalog.namespaceExists(testNamespace)).isTrue(); + assertThat(catalog.setProperties(testNamespace, propertiesToSet)).isTrue(); Map allProperties = catalog.loadNamespaceMetadata(testNamespace); - Assert.assertEquals(6, allProperties.size()); + assertThat(allProperties).hasSize(6); Map namespaceProperties = catalog.loadNamespaceMetadata(testNamespace); - Assert.assertEquals( - "All new keys should be in the namespace properties", - propertiesToSet.keySet(), - Sets.intersection(propertiesToSet.keySet(), namespaceProperties.keySet())); + assertThat(propertiesToSet.keySet()) + .as("All new keys should be in the namespace properties") + .isEqualTo(Sets.intersection(propertiesToSet.keySet(), namespaceProperties.keySet())); + // values should match for (Map.Entry keyValue : propertiesToSet.entrySet()) { - Assert.assertEquals( - "Value for key " + keyValue.getKey() + " should match", - keyValue.getValue(), - namespaceProperties.get(keyValue.getKey())); + assertThat(namespaceProperties) + .as("Value for key " + keyValue.getKey() + " should match") + .containsEntry(keyValue.getKey(), keyValue.getValue()); } } @@ -743,53 +730,23 @@ public void testRemoveProperties() { catalog.removeProperties(testNamespace, propertiesToRemove); Map remainderProperties = catalog.loadNamespaceMetadata(testNamespace); - Assert.assertEquals(3, remainderProperties.size()); - Assert.assertTrue(remainderProperties.containsKey("key_1")); - Assert.assertTrue(remainderProperties.containsKey("key_3")); - Assert.assertTrue(remainderProperties.containsKey("location")); + assertThat(remainderProperties) + .hasSize(3) + .containsKey("key_1") + .containsKey("key_3") + .containsKey("location"); // Remove remaining properties to test if it deletes the namespace Set allProperties = ImmutableSet.of("key_1", "key_3"); catalog.removeProperties(testNamespace, allProperties); - Assert.assertTrue(catalog.namespaceExists(testNamespace)); + assertThat(catalog.namespaceExists(testNamespace)).isTrue(); } @Test public void testConversions() { Namespace ns = Namespace.of("db", "db2", "ns2"); String nsString = JdbcUtil.namespaceToString(ns); - Assert.assertEquals(ns, JdbcUtil.stringToNamespace(nsString)); - } - - @Test - public void testRegisterTable() { - TableIdentifier identifier = TableIdentifier.of("a", "t1"); - catalog.createTable(identifier, SCHEMA); - Table registeringTable = catalog.loadTable(identifier); - catalog.dropTable(identifier, false); - TableOperations ops = ((HasTableOperations) registeringTable).operations(); - String metadataLocation = ((JdbcTableOperations) ops).currentMetadataLocation(); - Table registeredTable = catalog.registerTable(identifier, metadataLocation); - Assertions.assertThat(registeredTable).isNotNull(); - TestHelpers.assertSerializedAndLoadedMetadata(registeringTable, registeredTable); - String expectedMetadataLocation = - ((HasTableOperations) registeredTable).operations().current().metadataFileLocation(); - Assertions.assertThat(metadataLocation).isEqualTo(expectedMetadataLocation); - Assertions.assertThat(catalog.loadTable(identifier)).isNotNull(); - Assertions.assertThat(catalog.dropTable(identifier)).isTrue(); - } - - @Test - public void testRegisterExistingTable() { - TableIdentifier identifier = TableIdentifier.of("a", "t1"); - catalog.createTable(identifier, SCHEMA); - Table registeringTable = catalog.loadTable(identifier); - TableOperations ops = ((HasTableOperations) registeringTable).operations(); - String metadataLocation = ((JdbcTableOperations) ops).currentMetadataLocation(); - Assertions.assertThatThrownBy(() -> catalog.registerTable(identifier, metadataLocation)) - .isInstanceOf(AlreadyExistsException.class) - .hasMessage("Table already exists: a.t1"); - Assertions.assertThat(catalog.dropTable(identifier)).isTrue(); + assertThat(JdbcUtil.stringToNamespace(nsString)).isEqualTo(ns); } @Test diff --git a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcTableConcurrency.java b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcTableConcurrency.java index 7b59e0a4cec8..b4c5677dff80 100644 --- a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcTableConcurrency.java +++ b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcTableConcurrency.java @@ -22,6 +22,7 @@ import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS; import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; @@ -45,10 +46,8 @@ import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.Tasks; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestJdbcTableConcurrency { @@ -57,13 +56,12 @@ public class TestJdbcTableConcurrency { new Schema( required(1, "id", Types.IntegerType.get(), "unique ID"), required(2, "data", Types.StringType.get())); - @Rule public TemporaryFolder temp = new TemporaryFolder(); - File tableDir; + + @TempDir private File tableDir; @Test public synchronized void testConcurrentFastAppends() throws IOException { Map properties = Maps.newHashMap(); - this.tableDir = temp.newFolder(); properties.put(CatalogProperties.WAREHOUSE_LOCATION, tableDir.getAbsolutePath()); String sqliteDb = "jdbc:sqlite:" + tableDir.getAbsolutePath() + "concurentFastAppend.db"; properties.put(CatalogProperties.URI, sqliteDb); @@ -108,13 +106,12 @@ public synchronized void testConcurrentFastAppends() throws IOException { }); icebergTable.refresh(); - Assert.assertEquals(20, icebergTable.currentSnapshot().allManifests(icebergTable.io()).size()); + assertThat(icebergTable.currentSnapshot().allManifests(icebergTable.io())).hasSize(20); } @Test public synchronized void testConcurrentConnections() throws InterruptedException, IOException { Map properties = Maps.newHashMap(); - this.tableDir = temp.newFolder(); properties.put(CatalogProperties.WAREHOUSE_LOCATION, tableDir.getAbsolutePath()); String sqliteDb = "jdbc:sqlite:" + tableDir.getAbsolutePath() + "concurentConnections.db"; properties.put(CatalogProperties.URI, sqliteDb); @@ -149,7 +146,7 @@ public synchronized void testConcurrentConnections() throws InterruptedException } executorService.shutdown(); - Assert.assertTrue("Timeout", executorService.awaitTermination(3, TimeUnit.MINUTES)); - Assert.assertEquals(7, Iterables.size(icebergTable.snapshots())); + assertThat(executorService.awaitTermination(3, TimeUnit.MINUTES)).as("Timeout").isTrue(); + assertThat(Iterables.size(icebergTable.snapshots())).isEqualTo(7); } } diff --git a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcUtil.java b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcUtil.java index 3577aba3ebf7..0dec6eb83ebb 100644 --- a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcUtil.java +++ b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcUtil.java @@ -22,7 +22,7 @@ import java.util.Properties; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestJdbcUtil { diff --git a/core/src/test/java/org/apache/iceberg/metrics/TestCommitMetricsResultParser.java b/core/src/test/java/org/apache/iceberg/metrics/TestCommitMetricsResultParser.java index a3d52a394347..7598555c6b25 100644 --- a/core/src/test/java/org/apache/iceberg/metrics/TestCommitMetricsResultParser.java +++ b/core/src/test/java/org/apache/iceberg/metrics/TestCommitMetricsResultParser.java @@ -27,7 +27,7 @@ import java.util.concurrent.TimeUnit; import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestCommitMetricsResultParser { diff --git a/core/src/test/java/org/apache/iceberg/metrics/TestCommitReportParser.java b/core/src/test/java/org/apache/iceberg/metrics/TestCommitReportParser.java index 6d0da871a063..8c018e1b52bc 100644 --- a/core/src/test/java/org/apache/iceberg/metrics/TestCommitReportParser.java +++ b/core/src/test/java/org/apache/iceberg/metrics/TestCommitReportParser.java @@ -24,7 +24,7 @@ import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestCommitReportParser { diff --git a/core/src/test/java/org/apache/iceberg/metrics/TestCounterResultParser.java b/core/src/test/java/org/apache/iceberg/metrics/TestCounterResultParser.java index 609e8c825f5d..ea936a0265d8 100644 --- a/core/src/test/java/org/apache/iceberg/metrics/TestCounterResultParser.java +++ b/core/src/test/java/org/apache/iceberg/metrics/TestCounterResultParser.java @@ -21,7 +21,7 @@ import com.fasterxml.jackson.databind.JsonNode; import org.apache.iceberg.metrics.MetricsContext.Unit; import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestCounterResultParser { diff --git a/core/src/test/java/org/apache/iceberg/metrics/TestScanMetricsResultParser.java b/core/src/test/java/org/apache/iceberg/metrics/TestScanMetricsResultParser.java index 2c2f946e8c31..47b67cd96104 100644 --- a/core/src/test/java/org/apache/iceberg/metrics/TestScanMetricsResultParser.java +++ b/core/src/test/java/org/apache/iceberg/metrics/TestScanMetricsResultParser.java @@ -23,7 +23,7 @@ import java.util.concurrent.TimeUnit; import org.apache.iceberg.metrics.MetricsContext.Unit; import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestScanMetricsResultParser { diff --git a/core/src/test/java/org/apache/iceberg/metrics/TestScanReport.java b/core/src/test/java/org/apache/iceberg/metrics/TestScanReport.java index ab299b921d9a..caaf1bcaed21 100644 --- a/core/src/test/java/org/apache/iceberg/metrics/TestScanReport.java +++ b/core/src/test/java/org/apache/iceberg/metrics/TestScanReport.java @@ -24,7 +24,7 @@ import java.util.concurrent.TimeUnit; import org.apache.iceberg.expressions.Expressions; import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestScanReport { diff --git a/core/src/test/java/org/apache/iceberg/metrics/TestScanReportParser.java b/core/src/test/java/org/apache/iceberg/metrics/TestScanReportParser.java index 17848deeedec..df126601c8f1 100644 --- a/core/src/test/java/org/apache/iceberg/metrics/TestScanReportParser.java +++ b/core/src/test/java/org/apache/iceberg/metrics/TestScanReportParser.java @@ -23,7 +23,7 @@ import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestScanReportParser { diff --git a/core/src/test/java/org/apache/iceberg/metrics/TestTimerResultParser.java b/core/src/test/java/org/apache/iceberg/metrics/TestTimerResultParser.java index 58d2b3c58e12..3bbf2b88829b 100644 --- a/core/src/test/java/org/apache/iceberg/metrics/TestTimerResultParser.java +++ b/core/src/test/java/org/apache/iceberg/metrics/TestTimerResultParser.java @@ -23,7 +23,7 @@ import java.time.temporal.ChronoUnit; import java.util.concurrent.TimeUnit; import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestTimerResultParser { diff --git a/core/src/test/java/org/apache/iceberg/puffin/TestFileMetadataParser.java b/core/src/test/java/org/apache/iceberg/puffin/TestFileMetadataParser.java index c105283753dd..4d49580fd84b 100644 --- a/core/src/test/java/org/apache/iceberg/puffin/TestFileMetadataParser.java +++ b/core/src/test/java/org/apache/iceberg/puffin/TestFileMetadataParser.java @@ -24,7 +24,7 @@ import java.io.UncheckedIOException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestFileMetadataParser { @Test diff --git a/core/src/test/java/org/apache/iceberg/puffin/TestPuffinFormat.java b/core/src/test/java/org/apache/iceberg/puffin/TestPuffinFormat.java index 191658072911..896c8fe5d10c 100644 --- a/core/src/test/java/org/apache/iceberg/puffin/TestPuffinFormat.java +++ b/core/src/test/java/org/apache/iceberg/puffin/TestPuffinFormat.java @@ -28,7 +28,7 @@ import java.nio.ByteOrder; import java.util.Arrays; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestPuffinFormat { @Test diff --git a/core/src/test/java/org/apache/iceberg/puffin/TestPuffinReader.java b/core/src/test/java/org/apache/iceberg/puffin/TestPuffinReader.java index a63bfd55cbc9..094fee618dfa 100644 --- a/core/src/test/java/org/apache/iceberg/puffin/TestPuffinReader.java +++ b/core/src/test/java/org/apache/iceberg/puffin/TestPuffinReader.java @@ -36,7 +36,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Streams; import org.apache.iceberg.util.ByteBuffers; import org.apache.iceberg.util.Pair; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestPuffinReader { @Test diff --git a/core/src/test/java/org/apache/iceberg/puffin/TestPuffinWriter.java b/core/src/test/java/org/apache/iceberg/puffin/TestPuffinWriter.java index 7457a8c8017e..2a1184987130 100644 --- a/core/src/test/java/org/apache/iceberg/puffin/TestPuffinWriter.java +++ b/core/src/test/java/org/apache/iceberg/puffin/TestPuffinWriter.java @@ -30,7 +30,7 @@ import org.apache.iceberg.inmemory.InMemoryOutputFile; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestPuffinWriter { @Test diff --git a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java index c6d41818441c..0772601b77df 100644 --- a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java +++ b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java @@ -22,6 +22,11 @@ import java.util.List; import java.util.Map; import java.util.function.Consumer; +import org.apache.iceberg.BaseTable; +import org.apache.iceberg.BaseTransaction; +import org.apache.iceberg.Table; +import org.apache.iceberg.Transaction; +import org.apache.iceberg.Transactions; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.SupportsNamespaces; @@ -40,8 +45,11 @@ import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.base.Splitter; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.rest.requests.CommitTransactionRequest; import org.apache.iceberg.rest.requests.CreateNamespaceRequest; import org.apache.iceberg.rest.requests.CreateTableRequest; +import org.apache.iceberg.rest.requests.RegisterTableRequest; import org.apache.iceberg.rest.requests.RenameTableRequest; import org.apache.iceberg.rest.requests.ReportMetricsRequest; import org.apache.iceberg.rest.requests.UpdateNamespacePropertiesRequest; @@ -119,6 +127,11 @@ enum Route { LoadTableResponse.class), LOAD_TABLE( HTTPMethod.GET, "v1/namespaces/{namespace}/tables/{table}", null, LoadTableResponse.class), + REGISTER_TABLE( + HTTPMethod.POST, + "v1/namespaces/{namespace}/register", + RegisterTableRequest.class, + LoadTableResponse.class), UPDATE_TABLE( HTTPMethod.POST, "v1/namespaces/{namespace}/tables/{table}", @@ -130,7 +143,9 @@ enum Route { HTTPMethod.POST, "v1/namespaces/{namespace}/tables/{table}/metrics", ReportMetricsRequest.class, - null); + null), + COMMIT_TRANSACTION( + HTTPMethod.POST, "v1/transactions/commit", CommitTransactionRequest.class, null); private final HTTPMethod method; private final int requiredLength; @@ -336,6 +351,14 @@ public T handleRequest( return castResponse(responseType, CatalogHandlers.loadTable(catalog, ident)); } + case REGISTER_TABLE: + { + Namespace namespace = namespaceFromPathVars(vars); + RegisterTableRequest request = castRequest(RegisterTableRequest.class, body); + return castResponse( + responseType, CatalogHandlers.registerTable(catalog, namespace, request)); + } + case UPDATE_TABLE: { TableIdentifier ident = identFromPathVars(vars); @@ -357,12 +380,49 @@ public T handleRequest( return null; } + case COMMIT_TRANSACTION: + { + CommitTransactionRequest request = castRequest(CommitTransactionRequest.class, body); + commitTransaction(catalog, request); + return null; + } + default: } return null; } + /** + * This is a very simplistic approach that only validates the requirements for each table and does + * not do any other conflict detection. Therefore, it does not guarantee true transactional + * atomicity, which is left to the implementation details of a REST server. + */ + private static void commitTransaction(Catalog catalog, CommitTransactionRequest request) { + List transactions = Lists.newArrayList(); + + for (UpdateTableRequest tableChange : request.tableChanges()) { + Table table = catalog.loadTable(tableChange.identifier()); + if (table instanceof BaseTable) { + Transaction transaction = + Transactions.newTransaction( + tableChange.identifier().toString(), ((BaseTable) table).operations()); + transactions.add(transaction); + + BaseTransaction.TransactionTable txTable = + (BaseTransaction.TransactionTable) transaction.table(); + + // this performs validations and makes temporary commits that are in-memory + CatalogHandlers.commit(txTable.operations(), tableChange); + } else { + throw new IllegalStateException("Cannot wrap catalog that does not produce BaseTable"); + } + } + + // only commit if validations passed previously + transactions.forEach(Transaction::commitTransaction); + } + public T execute( HTTPMethod method, String path, diff --git a/core/src/test/java/org/apache/iceberg/rest/RequestResponseTestBase.java b/core/src/test/java/org/apache/iceberg/rest/RequestResponseTestBase.java index cf5879f8ea29..4855e9a6c779 100644 --- a/core/src/test/java/org/apache/iceberg/rest/RequestResponseTestBase.java +++ b/core/src/test/java/org/apache/iceberg/rest/RequestResponseTestBase.java @@ -25,8 +25,7 @@ import java.util.Set; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public abstract class RequestResponseTestBase { @@ -76,11 +75,13 @@ public void testHasOnlyKnownFields() { try { JsonNode node = mapper().readValue(serialize(createExampleInstance()), JsonNode.class); for (String field : fieldsFromSpec) { - Assert.assertTrue("Should have field: " + field, node.has(field)); + Assertions.assertThat(node.has(field)).as("Should have field: %s", field).isTrue(); } for (String field : ((Iterable) node::fieldNames)) { - Assert.assertTrue("Should not have field: " + field, fieldsFromSpec.contains(field)); + Assertions.assertThat(fieldsFromSpec) + .as("Should not have field: %s", field) + .contains(field); } } catch (JsonProcessingException e) { throw new RuntimeException(e); diff --git a/core/src/test/java/org/apache/iceberg/rest/TestHTTPClient.java b/core/src/test/java/org/apache/iceberg/rest/TestHTTPClient.java index 63f16cb2ead9..e596df43e6f5 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestHTTPClient.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestHTTPClient.java @@ -44,10 +44,9 @@ import org.apache.iceberg.rest.responses.ErrorResponse; import org.apache.iceberg.rest.responses.ErrorResponseParser; import org.assertj.core.api.Assertions; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; import org.mockserver.integration.ClientAndServer; import org.mockserver.model.HttpRequest; import org.mockserver.model.HttpResponse; @@ -68,7 +67,7 @@ public class TestHTTPClient { private static ClientAndServer mockServer; private static RESTClient restClient; - @BeforeClass + @BeforeAll public static void beforeClass() { mockServer = startClientAndServer(PORT); restClient = HTTPClient.builder(ImmutableMap.of()).uri(URI).build(); @@ -76,7 +75,7 @@ public static void beforeClass() { icebergBuildFullVersion = IcebergBuild.fullVersion(); } - @AfterClass + @AfterAll public static void stopServer() throws IOException { mockServer.stop(); restClient.close(); @@ -147,10 +146,9 @@ public static void testHttpMethodOnSuccess(HttpMethod method) throws JsonProcess doExecuteRequest(method, path, body, onError, h -> assertThat(h).isNotEmpty()); if (method.usesRequestBody()) { - Assert.assertEquals( - "On a successful " + method + ", the correct response body should be returned", - successResponse, - body); + Assertions.assertThat(body) + .as("On a successful " + method + ", the correct response body should be returned") + .isEqualTo(successResponse); } verify(onError, never()).accept(any()); diff --git a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java index 02468f3d9b8a..4695ac1b0f1a 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java @@ -20,6 +20,7 @@ import static org.apache.iceberg.types.Types.NestedField.required; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.AssertionsForClassTypes.assertThatThrownBy; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.times; @@ -38,6 +39,7 @@ import java.util.function.Consumer; import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.BaseTransaction; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.DataFiles; import org.apache.iceberg.FileScanTask; @@ -46,14 +48,22 @@ import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.Transaction; +import org.apache.iceberg.UpdatePartitionSpec; +import org.apache.iceberg.UpdateSchema; import org.apache.iceberg.catalog.CatalogTests; +import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.SessionCatalog; +import org.apache.iceberg.catalog.TableCommit; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.jdbc.JdbcCatalog; import org.apache.iceberg.metrics.MetricsReport; import org.apache.iceberg.metrics.MetricsReporter; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.rest.RESTCatalogAdapter.HTTPMethod; import org.apache.iceberg.rest.RESTSessionCatalog.SnapshotMode; @@ -65,17 +75,18 @@ import org.apache.iceberg.rest.responses.LoadTableResponse; import org.apache.iceberg.rest.responses.OAuthTokenResponse; import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.handler.gzip.GzipHandler; import org.eclipse.jetty.servlet.ServletContextHandler; import org.eclipse.jetty.servlet.ServletHolder; -import org.junit.Assert; import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import org.mockito.Mockito; import org.mockito.stubbing.Answer; @@ -267,20 +278,17 @@ public T get( restCat.setConf(new Configuration()); restCat.initialize("prod", initialConfig); - Assert.assertEquals( - "Catalog properties after initialize should use the server's override properties", - "false", - restCat.properties().get(CatalogProperties.CACHE_ENABLED)); + Assertions.assertThat(restCat.properties().get(CatalogProperties.CACHE_ENABLED)) + .as("Catalog properties after initialize should use the server's override properties") + .isEqualTo("false"); - Assert.assertEquals( - "Catalog after initialize should use the server's default properties if not specified", - "1", - restCat.properties().get(CatalogProperties.CLIENT_POOL_SIZE)); + Assertions.assertThat(restCat.properties().get(CatalogProperties.CLIENT_POOL_SIZE)) + .as("Catalog after initialize should use the server's default properties if not specified") + .isEqualTo("1"); - Assert.assertEquals( - "Catalog should return final warehouse location", - "s3://bucket/warehouse", - restCat.properties().get(CatalogProperties.WAREHOUSE_LOCATION)); + Assertions.assertThat(restCat.properties().get(CatalogProperties.WAREHOUSE_LOCATION)) + .as("Catalog should return final warehouse location") + .isEqualTo("s3://bucket/warehouse"); restCat.close(); } @@ -311,7 +319,7 @@ public void testCatalogBasicBearerToken() { catalog.initialize( "prod", ImmutableMap.of(CatalogProperties.URI, "ignored", "token", "bearer-token")); - Assertions.assertFalse(catalog.tableExists(TableIdentifier.of("ns", "table"))); + Assertions.assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))).isFalse(); // the bearer token should be used for all interactions Mockito.verify(adapter) @@ -347,7 +355,7 @@ public void testCatalogCredential() { catalog.initialize( "prod", ImmutableMap.of(CatalogProperties.URI, "ignored", "credential", "catalog:secret")); - Assertions.assertFalse(catalog.tableExists(TableIdentifier.of("ns", "table"))); + Assertions.assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))).isFalse(); // no token or credential for catalog token exchange Mockito.verify(adapter) @@ -400,7 +408,7 @@ public void testCatalogBearerTokenWithClientCredential() { catalog.initialize( "prod", ImmutableMap.of(CatalogProperties.URI, "ignored", "token", "bearer-token")); - Assertions.assertFalse(catalog.tableExists(TableIdentifier.of("ns", "table"))); + Assertions.assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))).isFalse(); // use the bearer token for config Mockito.verify(adapter) @@ -455,7 +463,7 @@ public void testCatalogCredentialWithClientCredential() { catalog.initialize( "prod", ImmutableMap.of(CatalogProperties.URI, "ignored", "credential", "catalog:secret")); - Assertions.assertFalse(catalog.tableExists(TableIdentifier.of("ns", "table"))); + Assertions.assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))).isFalse(); // call client credentials with no initial auth Mockito.verify(adapter) @@ -527,7 +535,7 @@ public void testCatalogBearerTokenAndCredentialWithClientCredential() { "token", "bearer-token")); - Assertions.assertFalse(catalog.tableExists(TableIdentifier.of("ns", "table"))); + Assertions.assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))).isFalse(); // use the bearer token for client credentials Mockito.verify(adapter) @@ -673,7 +681,7 @@ private void testClientAuth( catalog.initialize( "prod", ImmutableMap.of(CatalogProperties.URI, "ignored", "token", catalogToken)); - Assertions.assertFalse(catalog.tableExists(TableIdentifier.of("ns", "table"))); + Assertions.assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))).isFalse(); Mockito.verify(adapter) .execute( @@ -849,6 +857,133 @@ public void testTableSnapshotLoading() { any()); } + @ParameterizedTest + @ValueSource(strings = {"1", "2"}) + public void testTableSnapshotLoadingWithDivergedBranches(String formatVersion) { + RESTCatalogAdapter adapter = Mockito.spy(new RESTCatalogAdapter(backendCatalog)); + + RESTCatalog catalog = + new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) -> adapter); + catalog.initialize( + "test", + ImmutableMap.of( + CatalogProperties.URI, + "ignored", + CatalogProperties.FILE_IO_IMPL, + "org.apache.iceberg.inmemory.InMemoryFileIO", + "snapshot-loading-mode", + "refs")); + + Table table = + catalog.createTable( + TABLE, + SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of("format-version", formatVersion)); + + table + .newFastAppend() + .appendFile( + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/data-a.parquet") + .withFileSizeInBytes(10) + .withRecordCount(2) + .build()) + .commit(); + + String branch = "divergedBranch"; + table.manageSnapshots().createBranch(branch, table.currentSnapshot().snapshotId()).commit(); + + // branch and main are diverged now + table + .newFastAppend() + .appendFile( + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/data-b.parquet") + .withFileSizeInBytes(10) + .withRecordCount(2) + .build()) + .toBranch(branch) + .commit(); + + ResourcePaths paths = ResourcePaths.forCatalogProperties(Maps.newHashMap()); + + // Respond with only referenced snapshots + Answer refsAnswer = + invocation -> { + LoadTableResponse originalResponse = (LoadTableResponse) invocation.callRealMethod(); + TableMetadata fullTableMetadata = originalResponse.tableMetadata(); + + Set referencedSnapshotIds = + fullTableMetadata.refs().values().stream() + .map(SnapshotRef::snapshotId) + .collect(Collectors.toSet()); + + TableMetadata refsMetadata = + fullTableMetadata.removeSnapshotsIf( + s -> !referencedSnapshotIds.contains(s.snapshotId())); + + return LoadTableResponse.builder() + .withTableMetadata(refsMetadata) + .addAllConfig(originalResponse.config()) + .build(); + }; + + Mockito.doAnswer(refsAnswer) + .when(adapter) + .execute( + eq(HTTPMethod.GET), + eq(paths.table(TABLE)), + eq(ImmutableMap.of("snapshots", "refs")), + any(), + eq(LoadTableResponse.class), + any(), + any()); + + Table refsTables = catalog.loadTable(TABLE); + assertThat(refsTables.currentSnapshot()).isEqualTo(table.currentSnapshot()); + + // verify that the table was loaded with the refs argument + verify(adapter, times(1)) + .execute( + eq(HTTPMethod.GET), + eq(paths.table(TABLE)), + eq(ImmutableMap.of("snapshots", "refs")), + any(), + eq(LoadTableResponse.class), + any(), + any()); + + // verify that all snapshots are loaded when referenced + assertThat(catalog.loadTable(TABLE).snapshots()) + .containsExactlyInAnyOrderElementsOf(table.snapshots()); + verify(adapter, times(1)) + .execute( + eq(HTTPMethod.GET), + eq(paths.table(TABLE)), + eq(ImmutableMap.of("snapshots", "all")), + any(), + eq(LoadTableResponse.class), + any(), + any()); + + // verify that committing to branch is possible + catalog + .loadTable(TABLE) + .newAppend() + .appendFile( + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/data-c.parquet") + .withFileSizeInBytes(10) + .withRecordCount(2) + .build()) + .toBranch(branch) + .commit(); + + assertThat(catalog.loadTable(TABLE).snapshots()) + .hasSizeGreaterThan(Lists.newArrayList(table.snapshots()).size()); + } + public void testTableAuth( String catalogToken, Map credentials, @@ -907,12 +1042,14 @@ public void testTableAuth( required(2, "data", Types.StringType.get())); Table table = catalog.createTable(ident, expectedSchema); - Assertions.assertEquals( - expectedSchema.asStruct(), table.schema().asStruct(), "Schema should match"); + Assertions.assertThat(table.schema().asStruct()) + .as("Schema should match") + .isEqualTo(expectedSchema.asStruct()); Table loaded = catalog.loadTable(ident); // the first load will send the token - Assertions.assertEquals( - expectedSchema.asStruct(), loaded.schema().asStruct(), "Schema should match"); + Assertions.assertThat(loaded.schema().asStruct()) + .as("Schema should match") + .isEqualTo(expectedSchema.asStruct()); loaded.refresh(); // refresh to force reload @@ -1147,7 +1284,8 @@ public void testCatalogRefreshedTokenIsUsed() { .untilAsserted( () -> { // use the exchanged catalog token - Assertions.assertFalse(catalog.tableExists(TableIdentifier.of("ns", "table"))); + Assertions.assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))) + .isFalse(); // call client credentials with no initial auth Mockito.verify(adapter) @@ -1296,7 +1434,7 @@ public void testCatalogExpiredBearerTokenIsRefreshedWithCredential() { catalog.initialize( "prod", ImmutableMap.of(CatalogProperties.URI, "ignored", "credential", credential)); - Assertions.assertFalse(catalog.tableExists(TableIdentifier.of("ns", "table"))); + Assertions.assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))).isFalse(); // call client credentials with no initial auth Mockito.verify(adapter) @@ -1382,7 +1520,7 @@ public void testCatalogValidBearerTokenIsNotRefreshed() { RESTCatalog catalog = new RESTCatalog(context, (config) -> adapter); catalog.initialize("prod", ImmutableMap.of(CatalogProperties.URI, "ignored", "token", token)); - Assertions.assertFalse(catalog.tableExists(TableIdentifier.of("ns", "table"))); + Assertions.assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))).isFalse(); Mockito.verify(adapter) .execute( @@ -1475,7 +1613,8 @@ public void testCatalogTokenRefreshFailsAndUsesCredentialForRefresh() { .untilAsserted( () -> { // use the exchanged catalog token - Assertions.assertFalse(catalog.tableExists(TableIdentifier.of("ns", "table"))); + Assertions.assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))) + .isFalse(); // call client credentials with no initial auth Mockito.verify(adapter) @@ -1742,4 +1881,126 @@ public void testCatalogTokenRefreshDisabledWithCredential() { eq(catalogHeaders), any()); } + + @Test + public void diffAgainstSingleTable() { + Namespace namespace = Namespace.of("namespace"); + TableIdentifier identifier = TableIdentifier.of(namespace, "multipleDiffsAgainstSingleTable"); + + Table table = catalog().buildTable(identifier, SCHEMA).create(); + Transaction transaction = table.newTransaction(); + + UpdateSchema updateSchema = + transaction.updateSchema().addColumn("new_col", Types.LongType.get()); + Schema expectedSchema = updateSchema.apply(); + updateSchema.commit(); + + UpdatePartitionSpec updateSpec = + transaction.updateSpec().addField("shard", Expressions.bucket("id", 16)); + PartitionSpec expectedSpec = updateSpec.apply(); + updateSpec.commit(); + + TableCommit tableCommit = + TableCommit.create( + identifier, + ((BaseTransaction) transaction).startMetadata(), + ((BaseTransaction) transaction).currentMetadata()); + + restCatalog.commitTransaction(tableCommit); + + Table loaded = catalog().loadTable(identifier); + assertThat(loaded.schema().asStruct()).isEqualTo(expectedSchema.asStruct()); + assertThat(loaded.spec().fields()).isEqualTo(expectedSpec.fields()); + } + + @Test + public void multipleDiffsAgainstMultipleTables() { + Namespace namespace = Namespace.of("multiDiffNamespace"); + TableIdentifier identifier1 = TableIdentifier.of(namespace, "multiDiffTable1"); + TableIdentifier identifier2 = TableIdentifier.of(namespace, "multiDiffTable2"); + + Table table1 = catalog().buildTable(identifier1, SCHEMA).create(); + Table table2 = catalog().buildTable(identifier2, SCHEMA).create(); + Transaction t1Transaction = table1.newTransaction(); + Transaction t2Transaction = table2.newTransaction(); + + UpdateSchema updateSchema = + t1Transaction.updateSchema().addColumn("new_col", Types.LongType.get()); + Schema expectedSchema = updateSchema.apply(); + updateSchema.commit(); + + UpdateSchema updateSchema2 = + t2Transaction.updateSchema().addColumn("new_col2", Types.LongType.get()); + Schema expectedSchema2 = updateSchema2.apply(); + updateSchema2.commit(); + + TableCommit tableCommit1 = + TableCommit.create( + identifier1, + ((BaseTransaction) t1Transaction).startMetadata(), + ((BaseTransaction) t1Transaction).currentMetadata()); + + TableCommit tableCommit2 = + TableCommit.create( + identifier2, + ((BaseTransaction) t2Transaction).startMetadata(), + ((BaseTransaction) t2Transaction).currentMetadata()); + + restCatalog.commitTransaction(tableCommit1, tableCommit2); + + assertThat(catalog().loadTable(identifier1).schema().asStruct()) + .isEqualTo(expectedSchema.asStruct()); + + assertThat(catalog().loadTable(identifier2).schema().asStruct()) + .isEqualTo(expectedSchema2.asStruct()); + } + + @Test + public void multipleDiffsAgainstMultipleTablesLastFails() { + Namespace namespace = Namespace.of("multiDiffNamespace"); + TableIdentifier identifier1 = TableIdentifier.of(namespace, "multiDiffTable1"); + TableIdentifier identifier2 = TableIdentifier.of(namespace, "multiDiffTable2"); + + catalog().createTable(identifier1, SCHEMA); + catalog().createTable(identifier2, SCHEMA); + + Table table1 = catalog().loadTable(identifier1); + Table table2 = catalog().loadTable(identifier2); + Schema originalSchemaOne = table1.schema(); + + Transaction t1Transaction = catalog().loadTable(identifier1).newTransaction(); + t1Transaction.updateSchema().addColumn("new_col1", Types.LongType.get()).commit(); + + Transaction t2Transaction = catalog().loadTable(identifier2).newTransaction(); + t2Transaction.updateSchema().renameColumn("data", "new-column").commit(); + + // delete the colum that is being renamed in the above TX to cause a conflict + table2.updateSchema().deleteColumn("data").commit(); + Schema updatedSchemaTwo = table2.schema(); + + TableCommit tableCommit1 = + TableCommit.create( + identifier1, + ((BaseTransaction) t1Transaction).startMetadata(), + ((BaseTransaction) t1Transaction).currentMetadata()); + + TableCommit tableCommit2 = + TableCommit.create( + identifier2, + ((BaseTransaction) t2Transaction).startMetadata(), + ((BaseTransaction) t2Transaction).currentMetadata()); + + assertThatThrownBy(() -> restCatalog.commitTransaction(tableCommit1, tableCommit2)) + .isInstanceOf(CommitFailedException.class) + .hasMessageContaining("Requirement failed: current schema changed: expected id 0 != 1"); + + Schema schema1 = catalog().loadTable(identifier1).schema(); + assertThat(schema1.asStruct()).isEqualTo(originalSchemaOne.asStruct()); + + Schema schema2 = catalog().loadTable(identifier2).schema(); + assertThat(schema2.asStruct()).isEqualTo(updatedSchemaTwo.asStruct()); + assertThat(schema2.findField("data")).isNull(); + assertThat(schema2.findField("new-column")).isNull(); + assertThat(schema2.columns()).hasSize(1); + } } diff --git a/core/src/test/java/org/apache/iceberg/rest/TestRESTUtil.java b/core/src/test/java/org/apache/iceberg/rest/TestRESTUtil.java index bf312e67df43..680a8bcaa377 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTUtil.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTUtil.java @@ -22,7 +22,7 @@ import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestRESTUtil { diff --git a/core/src/test/java/org/apache/iceberg/rest/TestResourcePaths.java b/core/src/test/java/org/apache/iceberg/rest/TestResourcePaths.java index dc1fbb0c9c95..e0e61a594e7d 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestResourcePaths.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestResourcePaths.java @@ -21,8 +21,8 @@ import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Assert; -import org.junit.Test; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; public class TestResourcePaths { private final String prefix = "ws/catalog"; @@ -33,99 +33,114 @@ public class TestResourcePaths { @Test public void testConfigPath() { // prefix does not affect the config route because config is merged into catalog properties - Assert.assertEquals(ResourcePaths.config(), "v1/config"); + Assertions.assertThat(ResourcePaths.config()).isEqualTo("v1/config"); } @Test public void testNamespaces() { - Assert.assertEquals("v1/ws/catalog/namespaces", withPrefix.namespaces()); - Assert.assertEquals("v1/namespaces", withoutPrefix.namespaces()); + Assertions.assertThat(withPrefix.namespaces()).isEqualTo("v1/ws/catalog/namespaces"); + Assertions.assertThat(withoutPrefix.namespaces()).isEqualTo("v1/namespaces"); } @Test public void testNamespace() { Namespace ns = Namespace.of("ns"); - Assert.assertEquals("v1/ws/catalog/namespaces/ns", withPrefix.namespace(ns)); - Assert.assertEquals("v1/namespaces/ns", withoutPrefix.namespace(ns)); + Assertions.assertThat(withPrefix.namespace(ns)).isEqualTo("v1/ws/catalog/namespaces/ns"); + Assertions.assertThat(withoutPrefix.namespace(ns)).isEqualTo("v1/namespaces/ns"); } @Test public void testNamespaceWithSlash() { Namespace ns = Namespace.of("n/s"); - Assert.assertEquals("v1/ws/catalog/namespaces/n%2Fs", withPrefix.namespace(ns)); - Assert.assertEquals("v1/namespaces/n%2Fs", withoutPrefix.namespace(ns)); + Assertions.assertThat(withPrefix.namespace(ns)).isEqualTo("v1/ws/catalog/namespaces/n%2Fs"); + Assertions.assertThat(withoutPrefix.namespace(ns)).isEqualTo("v1/namespaces/n%2Fs"); } @Test public void testNamespaceWithMultipartNamespace() { Namespace ns = Namespace.of("n", "s"); - Assert.assertEquals("v1/ws/catalog/namespaces/n%1Fs", withPrefix.namespace(ns)); - Assert.assertEquals("v1/namespaces/n%1Fs", withoutPrefix.namespace(ns)); + Assertions.assertThat(withPrefix.namespace(ns)).isEqualTo("v1/ws/catalog/namespaces/n%1Fs"); + Assertions.assertThat(withoutPrefix.namespace(ns)).isEqualTo("v1/namespaces/n%1Fs"); } @Test public void testNamespaceProperties() { Namespace ns = Namespace.of("ns"); - Assert.assertEquals( - "v1/ws/catalog/namespaces/ns/properties", withPrefix.namespaceProperties(ns)); - Assert.assertEquals("v1/namespaces/ns/properties", withoutPrefix.namespaceProperties(ns)); + Assertions.assertThat(withPrefix.namespaceProperties(ns)) + .isEqualTo("v1/ws/catalog/namespaces/ns/properties"); + Assertions.assertThat(withoutPrefix.namespaceProperties(ns)) + .isEqualTo("v1/namespaces/ns/properties"); } @Test public void testNamespacePropertiesWithSlash() { Namespace ns = Namespace.of("n/s"); - Assert.assertEquals( - "v1/ws/catalog/namespaces/n%2Fs/properties", withPrefix.namespaceProperties(ns)); - Assert.assertEquals("v1/namespaces/n%2Fs/properties", withoutPrefix.namespaceProperties(ns)); + Assertions.assertThat(withPrefix.namespaceProperties(ns)) + .isEqualTo("v1/ws/catalog/namespaces/n%2Fs/properties"); + Assertions.assertThat(withoutPrefix.namespaceProperties(ns)) + .isEqualTo("v1/namespaces/n%2Fs/properties"); } @Test public void testNamespacePropertiesWithMultipartNamespace() { Namespace ns = Namespace.of("n", "s"); - Assert.assertEquals( - "v1/ws/catalog/namespaces/n%1Fs/properties", withPrefix.namespaceProperties(ns)); - Assert.assertEquals("v1/namespaces/n%1Fs/properties", withoutPrefix.namespaceProperties(ns)); + Assertions.assertThat(withPrefix.namespaceProperties(ns)) + .isEqualTo("v1/ws/catalog/namespaces/n%1Fs/properties"); + Assertions.assertThat(withoutPrefix.namespaceProperties(ns)) + .isEqualTo("v1/namespaces/n%1Fs/properties"); } @Test public void testTables() { Namespace ns = Namespace.of("ns"); - Assert.assertEquals("v1/ws/catalog/namespaces/ns/tables", withPrefix.tables(ns)); - Assert.assertEquals("v1/namespaces/ns/tables", withoutPrefix.tables(ns)); + Assertions.assertThat(withPrefix.tables(ns)).isEqualTo("v1/ws/catalog/namespaces/ns/tables"); + Assertions.assertThat(withoutPrefix.tables(ns)).isEqualTo("v1/namespaces/ns/tables"); } @Test public void testTablesWithSlash() { Namespace ns = Namespace.of("n/s"); - Assert.assertEquals("v1/ws/catalog/namespaces/n%2Fs/tables", withPrefix.tables(ns)); - Assert.assertEquals("v1/namespaces/n%2Fs/tables", withoutPrefix.tables(ns)); + Assertions.assertThat(withPrefix.tables(ns)).isEqualTo("v1/ws/catalog/namespaces/n%2Fs/tables"); + Assertions.assertThat(withoutPrefix.tables(ns)).isEqualTo("v1/namespaces/n%2Fs/tables"); } @Test public void testTablesWithMultipartNamespace() { Namespace ns = Namespace.of("n", "s"); - Assert.assertEquals("v1/ws/catalog/namespaces/n%1Fs/tables", withPrefix.tables(ns)); - Assert.assertEquals("v1/namespaces/n%1Fs/tables", withoutPrefix.tables(ns)); + Assertions.assertThat(withPrefix.tables(ns)).isEqualTo("v1/ws/catalog/namespaces/n%1Fs/tables"); + Assertions.assertThat(withoutPrefix.tables(ns)).isEqualTo("v1/namespaces/n%1Fs/tables"); } @Test public void testTable() { TableIdentifier ident = TableIdentifier.of("ns", "table"); - Assert.assertEquals("v1/ws/catalog/namespaces/ns/tables/table", withPrefix.table(ident)); - Assert.assertEquals("v1/namespaces/ns/tables/table", withoutPrefix.table(ident)); + Assertions.assertThat(withPrefix.table(ident)) + .isEqualTo("v1/ws/catalog/namespaces/ns/tables/table"); + Assertions.assertThat(withoutPrefix.table(ident)).isEqualTo("v1/namespaces/ns/tables/table"); } @Test public void testTableWithSlash() { TableIdentifier ident = TableIdentifier.of("n/s", "tab/le"); - Assert.assertEquals("v1/ws/catalog/namespaces/n%2Fs/tables/tab%2Fle", withPrefix.table(ident)); - Assert.assertEquals("v1/namespaces/n%2Fs/tables/tab%2Fle", withoutPrefix.table(ident)); + Assertions.assertThat(withPrefix.table(ident)) + .isEqualTo("v1/ws/catalog/namespaces/n%2Fs/tables/tab%2Fle"); + Assertions.assertThat(withoutPrefix.table(ident)) + .isEqualTo("v1/namespaces/n%2Fs/tables/tab%2Fle"); } @Test public void testTableWithMultipartNamespace() { TableIdentifier ident = TableIdentifier.of("n", "s", "table"); - Assert.assertEquals("v1/ws/catalog/namespaces/n%1Fs/tables/table", withPrefix.table(ident)); - Assert.assertEquals("v1/namespaces/n%1Fs/tables/table", withoutPrefix.table(ident)); + Assertions.assertThat(withPrefix.table(ident)) + .isEqualTo("v1/ws/catalog/namespaces/n%1Fs/tables/table"); + Assertions.assertThat(withoutPrefix.table(ident)).isEqualTo("v1/namespaces/n%1Fs/tables/table"); + } + + @Test + public void testRegister() { + Namespace ns = Namespace.of("ns"); + Assertions.assertThat(withPrefix.register(ns)) + .isEqualTo("v1/ws/catalog/namespaces/ns/register"); + Assertions.assertThat(withoutPrefix.register(ns)).isEqualTo("v1/namespaces/ns/register"); } } diff --git a/core/src/test/java/org/apache/iceberg/rest/requests/TestCommitTransactionRequestParser.java b/core/src/test/java/org/apache/iceberg/rest/requests/TestCommitTransactionRequestParser.java new file mode 100644 index 000000000000..c10e25d6e631 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/rest/requests/TestCommitTransactionRequestParser.java @@ -0,0 +1,175 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest.requests; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import com.fasterxml.jackson.databind.JsonNode; +import org.apache.iceberg.MetadataUpdate; +import org.apache.iceberg.UpdateRequirement; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.junit.jupiter.api.Test; + +public class TestCommitTransactionRequestParser { + + @Test + public void nullAndEmptyCheck() { + assertThatThrownBy(() -> CommitTransactionRequestParser.toJson(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid commit transaction request: null"); + + assertThatThrownBy(() -> CommitTransactionRequestParser.fromJson((JsonNode) null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse commit transaction request from null object"); + + assertThatThrownBy(() -> CommitTransactionRequestParser.fromJson("{}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse missing field: table-changes"); + + assertThatThrownBy(() -> CommitTransactionRequestParser.fromJson("{\"table-changes\":{}}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse commit transaction request from non-array: {}"); + + assertThatThrownBy(() -> CommitTransactionRequestParser.fromJson("{\"table-changes\":[]}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid table changes: empty"); + } + + @Test + public void invalidTableIdentifier() { + assertThatThrownBy( + () -> + CommitTransactionRequestParser.fromJson( + "{\"table-changes\":[{\"ns1.table1\" : \"ns1.table1\"}]}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid table changes: table identifier is required"); + + assertThatThrownBy( + () -> + CommitTransactionRequestParser.fromJson( + "{\"table-changes\":[{\"identifier\" : {}}]}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse missing string: name"); + + assertThatThrownBy( + () -> + CommitTransactionRequestParser.fromJson( + "{\"table-changes\":[{\"identifier\" : { \"name\": 23}}]}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse to a string value: name: 23"); + } + + @Test + public void roundTripSerde() { + String uuid = "2cc52516-5e73-41f2-b139-545d41a4e151"; + UpdateTableRequest commitTableRequestOne = + UpdateTableRequest.create( + TableIdentifier.of("ns1", "table1"), + ImmutableList.of( + new UpdateRequirement.AssertTableUUID(uuid), + new UpdateRequirement.AssertTableDoesNotExist()), + ImmutableList.of( + new MetadataUpdate.AssignUUID(uuid), new MetadataUpdate.SetCurrentSchema(23))); + + UpdateTableRequest commitTableRequestTwo = + UpdateTableRequest.create( + TableIdentifier.of("ns1", "table2"), + ImmutableList.of( + new UpdateRequirement.AssertDefaultSpecID(4), + new UpdateRequirement.AssertCurrentSchemaID(24)), + ImmutableList.of( + new MetadataUpdate.RemoveSnapshot(101L), new MetadataUpdate.SetCurrentSchema(25))); + + CommitTransactionRequest request = + new CommitTransactionRequest( + ImmutableList.of(commitTableRequestOne, commitTableRequestTwo)); + + String expectedJson = + "{\n" + + " \"table-changes\" : [ {\n" + + " \"identifier\" : {\n" + + " \"namespace\" : [ \"ns1\" ],\n" + + " \"name\" : \"table1\"\n" + + " },\n" + + " \"requirements\" : [ {\n" + + " \"type\" : \"assert-table-uuid\",\n" + + " \"uuid\" : \"2cc52516-5e73-41f2-b139-545d41a4e151\"\n" + + " }, {\n" + + " \"type\" : \"assert-create\"\n" + + " } ],\n" + + " \"updates\" : [ {\n" + + " \"action\" : \"assign-uuid\",\n" + + " \"uuid\" : \"2cc52516-5e73-41f2-b139-545d41a4e151\"\n" + + " }, {\n" + + " \"action\" : \"set-current-schema\",\n" + + " \"schema-id\" : 23\n" + + " } ]\n" + + " }, {\n" + + " \"identifier\" : {\n" + + " \"namespace\" : [ \"ns1\" ],\n" + + " \"name\" : \"table2\"\n" + + " },\n" + + " \"requirements\" : [ {\n" + + " \"type\" : \"assert-default-spec-id\",\n" + + " \"default-spec-id\" : 4\n" + + " }, {\n" + + " \"type\" : \"assert-current-schema-id\",\n" + + " \"current-schema-id\" : 24\n" + + " } ],\n" + + " \"updates\" : [ {\n" + + " \"action\" : \"remove-snapshots\",\n" + + " \"snapshot-ids\" : [ 101 ]\n" + + " }, {\n" + + " \"action\" : \"set-current-schema\",\n" + + " \"schema-id\" : 25\n" + + " } ]\n" + + " } ]\n" + + "}"; + + String json = CommitTransactionRequestParser.toJson(request, true); + assertThat(json).isEqualTo(expectedJson); + + // can't do an equality comparison on CommitTransactionRequest because updates/requirements + // don't implement equals/hashcode + assertThat( + CommitTransactionRequestParser.toJson( + CommitTransactionRequestParser.fromJson(json), true)) + .isEqualTo(expectedJson); + } + + @Test + public void emptyRequirementsAndUpdates() { + CommitTransactionRequest commitTxRequest = + new CommitTransactionRequest( + ImmutableList.of( + UpdateTableRequest.create( + TableIdentifier.of("ns1", "table1"), ImmutableList.of(), ImmutableList.of()))); + + String json = + "{\"table-changes\":[{\"identifier\":{\"namespace\":[\"ns1\"],\"name\":\"table1\"},\"requirements\":[],\"updates\":[]}]}"; + + assertThat(CommitTransactionRequestParser.toJson(commitTxRequest)).isEqualTo(json); + // can't do an equality comparison on CommitTransactionRequest because updates/requirements + // don't implement equals/hashcode + assertThat(CommitTransactionRequestParser.toJson(CommitTransactionRequestParser.fromJson(json))) + .isEqualTo(json); + } +} diff --git a/core/src/test/java/org/apache/iceberg/rest/requests/TestCreateNamespaceRequest.java b/core/src/test/java/org/apache/iceberg/rest/requests/TestCreateNamespaceRequest.java index f78e8c35bbd1..8b849b2cfb44 100644 --- a/core/src/test/java/org/apache/iceberg/rest/requests/TestCreateNamespaceRequest.java +++ b/core/src/test/java/org/apache/iceberg/rest/requests/TestCreateNamespaceRequest.java @@ -25,8 +25,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.rest.RequestResponseTestBase; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestCreateNamespaceRequest extends RequestResponseTestBase { @@ -148,8 +147,8 @@ public CreateNamespaceRequest createExampleInstance() { @Override public void assertEquals(CreateNamespaceRequest actual, CreateNamespaceRequest expected) { - Assert.assertEquals("Namespaces should be equal", actual.namespace(), expected.namespace()); - Assert.assertEquals("Properties should be equal", actual.properties(), expected.properties()); + Assertions.assertThat(actual.namespace()).isEqualTo(expected.namespace()); + Assertions.assertThat(actual.properties()).isEqualTo(expected.properties()); } @Override diff --git a/core/src/test/java/org/apache/iceberg/rest/requests/TestCreateTableRequest.java b/core/src/test/java/org/apache/iceberg/rest/requests/TestCreateTableRequest.java index 0df4f374b905..a5895d380708 100644 --- a/core/src/test/java/org/apache/iceberg/rest/requests/TestCreateTableRequest.java +++ b/core/src/test/java/org/apache/iceberg/rest/requests/TestCreateTableRequest.java @@ -35,8 +35,7 @@ import org.apache.iceberg.rest.RequestResponseTestBase; import org.apache.iceberg.types.Types; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestCreateTableRequest extends RequestResponseTestBase { @@ -259,20 +258,27 @@ public CreateTableRequest createExampleInstance() { @Override public void assertEquals(CreateTableRequest actual, CreateTableRequest expected) { - Assert.assertEquals("Name should be the same", expected.name(), actual.name()); - Assert.assertEquals( - "Location should be the same if provided", expected.location(), actual.location()); - Assert.assertTrue( - "Schemas should be equivalent and have same schema id", - expected.schema().sameSchema(actual.schema()) - && expected.schema().schemaId() == actual.schema().schemaId()); - Assert.assertEquals("Partition spec should be equal", expected.spec(), actual.spec()); - Assert.assertEquals( - "Write [sort] order should be the same", expected.writeOrder(), actual.writeOrder()); - Assert.assertEquals( - "Properties should be the same", expected.properties(), actual.properties()); - Assert.assertEquals( - "Stage create should be equal", expected.stageCreate(), actual.stageCreate()); + Assertions.assertThat(actual.name()).as("Name should be the same").isEqualTo(expected.name()); + Assertions.assertThat(actual.location()) + .as("Location should be the same if provided") + .isEqualTo(expected.location()); + Assertions.assertThat( + expected.schema().sameSchema(actual.schema()) + && expected.schema().schemaId() == actual.schema().schemaId()) + .as("Schemas should be equivalent and have same schema id") + .isTrue(); + Assertions.assertThat(actual.spec()) + .as("Partition spec should be equal") + .isEqualTo(expected.spec()); + Assertions.assertThat(actual.writeOrder()) + .as("Write [sort] order should be the same") + .isEqualTo(expected.writeOrder()); + Assertions.assertThat(actual.properties()) + .as("Properties should be the same") + .isEqualTo(expected.properties()); + Assertions.assertThat(actual.stageCreate()) + .as("Stage create should be equal") + .isEqualTo(expected.stageCreate()); } @Override diff --git a/core/src/test/java/org/apache/iceberg/rest/requests/TestRegisterTableRequestParser.java b/core/src/test/java/org/apache/iceberg/rest/requests/TestRegisterTableRequestParser.java new file mode 100644 index 000000000000..9b479d89d7d7 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/rest/requests/TestRegisterTableRequestParser.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest.requests; + +import static org.assertj.core.api.Assertions.assertThat; + +import com.fasterxml.jackson.databind.JsonNode; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +public class TestRegisterTableRequestParser { + + @Test + public void nullCheck() { + Assertions.assertThatThrownBy(() -> RegisterTableRequestParser.toJson(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid register table request: null"); + + Assertions.assertThatThrownBy(() -> RegisterTableRequestParser.fromJson((JsonNode) null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse register table request from null object"); + } + + @Test + public void missingFields() { + Assertions.assertThatThrownBy(() -> RegisterTableRequestParser.fromJson("{}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse missing string: name"); + + Assertions.assertThatThrownBy( + () -> RegisterTableRequestParser.fromJson("{\"name\" : \"test_tbl\"}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse missing string: metadata-location"); + + Assertions.assertThatThrownBy( + () -> + RegisterTableRequestParser.fromJson( + "{\"metadata-location\" : \"file://tmp/NS/test_tbl/metadata/00000-d4f60d2f-2ad2-408b-8832-0ed7fbd851ee.metadata.json\"}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse missing string: name"); + } + + @Test + public void roundTripSerde() { + RegisterTableRequest request = + ImmutableRegisterTableRequest.builder() + .name("table_1") + .metadataLocation( + "file://tmp/NS/test_tbl/metadata/00000-d4f60d2f-2ad2-408b-8832-0ed7fbd851ee.metadata.json") + .build(); + + String expectedJson = + "{\n" + + " \"name\" : \"table_1\",\n" + + " \"metadata-location\" : \"file://tmp/NS/test_tbl/metadata/00000-d4f60d2f-2ad2-408b-8832-0ed7fbd851ee.metadata.json\"\n" + + "}"; + + String json = RegisterTableRequestParser.toJson(request, true); + assertThat(json).isEqualTo(expectedJson); + + assertThat(RegisterTableRequestParser.toJson(RegisterTableRequestParser.fromJson(json), true)) + .isEqualTo(expectedJson); + } +} diff --git a/core/src/test/java/org/apache/iceberg/rest/requests/TestRenameTableRequest.java b/core/src/test/java/org/apache/iceberg/rest/requests/TestRenameTableRequest.java index d95e9454a21a..aeff8755aab3 100644 --- a/core/src/test/java/org/apache/iceberg/rest/requests/TestRenameTableRequest.java +++ b/core/src/test/java/org/apache/iceberg/rest/requests/TestRenameTableRequest.java @@ -24,8 +24,7 @@ import org.apache.iceberg.catalog.TableIdentifierParser; import org.apache.iceberg.rest.RequestResponseTestBase; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestRenameTableRequest extends RequestResponseTestBase { @@ -119,12 +118,12 @@ public RenameTableRequest createExampleInstance() { @Override public void assertEquals(RenameTableRequest actual, RenameTableRequest expected) { - Assert.assertEquals( - "Source table identifier should be equal", expected.source(), actual.source()); - Assert.assertEquals( - "Destination table identifier should be equal", - expected.destination(), - actual.destination()); + Assertions.assertThat(actual.source()) + .as("Source table identifier should be equal") + .isEqualTo(expected.source()); + Assertions.assertThat(actual.destination()) + .as("Destination table identifier should be equal") + .isEqualTo(expected.destination()); } @Override diff --git a/core/src/test/java/org/apache/iceberg/rest/requests/TestReportMetricsRequestParser.java b/core/src/test/java/org/apache/iceberg/rest/requests/TestReportMetricsRequestParser.java index 465714b173d9..f04938908b64 100644 --- a/core/src/test/java/org/apache/iceberg/rest/requests/TestReportMetricsRequestParser.java +++ b/core/src/test/java/org/apache/iceberg/rest/requests/TestReportMetricsRequestParser.java @@ -31,7 +31,7 @@ import org.apache.iceberg.metrics.ScanReport; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestReportMetricsRequestParser { diff --git a/core/src/test/java/org/apache/iceberg/rest/requests/TestUpdateNamespacePropertiesRequest.java b/core/src/test/java/org/apache/iceberg/rest/requests/TestUpdateNamespacePropertiesRequest.java index b8a823afb4de..daef718e1510 100644 --- a/core/src/test/java/org/apache/iceberg/rest/requests/TestUpdateNamespacePropertiesRequest.java +++ b/core/src/test/java/org/apache/iceberg/rest/requests/TestUpdateNamespacePropertiesRequest.java @@ -28,8 +28,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.rest.RequestResponseTestBase; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestUpdateNamespacePropertiesRequest extends RequestResponseTestBase { @@ -212,12 +211,12 @@ public UpdateNamespacePropertiesRequest createExampleInstance() { @Override public void assertEquals( UpdateNamespacePropertiesRequest actual, UpdateNamespacePropertiesRequest expected) { - Assert.assertEquals( - "Properties to update should be equal", actual.updates(), expected.updates()); - Assert.assertEquals( - "Properties to remove should be equal", - Sets.newHashSet(actual.removals()), - Sets.newHashSet(expected.removals())); + Assertions.assertThat(actual.updates()) + .as("Properties to update should be equal") + .isEqualTo(expected.updates()); + Assertions.assertThat(Sets.newHashSet(actual.removals())) + .as("Properties to remove should be equal") + .containsExactlyInAnyOrderElementsOf(Sets.newHashSet(expected.removals())); } @Override diff --git a/core/src/test/java/org/apache/iceberg/rest/requests/TestUpdateTableRequestParser.java b/core/src/test/java/org/apache/iceberg/rest/requests/TestUpdateTableRequestParser.java new file mode 100644 index 000000000000..9522801cadd2 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/rest/requests/TestUpdateTableRequestParser.java @@ -0,0 +1,222 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest.requests; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import com.fasterxml.jackson.databind.JsonNode; +import org.apache.iceberg.MetadataUpdate; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.junit.jupiter.api.Test; + +public class TestUpdateTableRequestParser { + + @Test + public void nullAndEmptyCheck() { + assertThatThrownBy(() -> UpdateTableRequestParser.toJson(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid update table request: null"); + + assertThatThrownBy(() -> UpdateTableRequestParser.fromJson((JsonNode) null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse update table request from null object"); + + UpdateTableRequest request = UpdateTableRequestParser.fromJson("{}"); + assertThat(request.identifier()).isNull(); + assertThat(request.updates()).isEmpty(); + assertThat(request.requirements()).isEmpty(); + } + + @Test + public void invalidTableIdentifier() { + // table identifier is optional + UpdateTableRequest request = + UpdateTableRequestParser.fromJson("{\"requirements\" : [], \"updates\" : []}"); + assertThat(request.identifier()).isNull(); + + assertThatThrownBy(() -> UpdateTableRequestParser.fromJson("{\"identifier\" : {}}}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse missing string: name"); + + assertThatThrownBy( + () -> UpdateTableRequestParser.fromJson("{\"identifier\" : { \"name\": 23}}}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse to a string value: name: 23"); + } + + @Test + public void invalidRequirements() { + assertThatThrownBy( + () -> + UpdateTableRequestParser.fromJson( + "{\"identifier\":{\"namespace\":[\"ns1\"],\"name\":\"table1\"}," + + "\"requirements\":[23],\"updates\":[]}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse update requirement from non-object value: 23"); + + assertThatThrownBy( + () -> + UpdateTableRequestParser.fromJson( + "{\"identifier\":{\"namespace\":[\"ns1\"],\"name\":\"table1\"}," + + "\"requirements\":[{}],\"updates\":[]}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse update requirement. Missing field: type"); + + assertThatThrownBy( + () -> + UpdateTableRequestParser.fromJson( + "{\"identifier\":{\"namespace\":[\"ns1\"],\"name\":\"table1\"}," + + "\"requirements\":[{\"type\":\"assert-table-uuid\"}],\"updates\":[]}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse missing string: uuid"); + } + + @Test + public void invalidMetadataUpdates() { + assertThatThrownBy( + () -> + UpdateTableRequestParser.fromJson( + "{\"identifier\":{\"namespace\":[\"ns1\"],\"name\":\"table1\"}," + + "\"requirements\":[],\"updates\":[23]}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse metadata update from non-object value: 23"); + + assertThatThrownBy( + () -> + UpdateTableRequestParser.fromJson( + "{\"identifier\":{\"namespace\":[\"ns1\"],\"name\":\"table1\"}," + + "\"requirements\":[],\"updates\":[{}]}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse metadata update. Missing field: action"); + + assertThatThrownBy( + () -> + UpdateTableRequestParser.fromJson( + "{\"identifier\":{\"namespace\":[\"ns1\"],\"name\":\"table1\"}," + + "\"requirements\":[],\"updates\":[{\"action\":\"assign-uuid\"}]}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse missing string: uuid"); + } + + @Test + public void roundTripSerde() { + String uuid = "2cc52516-5e73-41f2-b139-545d41a4e151"; + UpdateTableRequest request = + UpdateTableRequest.create( + TableIdentifier.of("ns1", "table1"), + ImmutableList.of( + new org.apache.iceberg.UpdateRequirement.AssertTableUUID(uuid), + new org.apache.iceberg.UpdateRequirement.AssertTableDoesNotExist()), + ImmutableList.of( + new MetadataUpdate.AssignUUID(uuid), new MetadataUpdate.SetCurrentSchema(23))); + + String expectedJson = + "{\n" + + " \"identifier\" : {\n" + + " \"namespace\" : [ \"ns1\" ],\n" + + " \"name\" : \"table1\"\n" + + " },\n" + + " \"requirements\" : [ {\n" + + " \"type\" : \"assert-table-uuid\",\n" + + " \"uuid\" : \"2cc52516-5e73-41f2-b139-545d41a4e151\"\n" + + " }, {\n" + + " \"type\" : \"assert-create\"\n" + + " } ],\n" + + " \"updates\" : [ {\n" + + " \"action\" : \"assign-uuid\",\n" + + " \"uuid\" : \"2cc52516-5e73-41f2-b139-545d41a4e151\"\n" + + " }, {\n" + + " \"action\" : \"set-current-schema\",\n" + + " \"schema-id\" : 23\n" + + " } ]\n" + + "}"; + + String json = UpdateTableRequestParser.toJson(request, true); + assertThat(json).isEqualTo(expectedJson); + + // can't do an equality comparison on UpdateTableRequest because updates/requirements + // don't implement equals/hashcode + assertThat(UpdateTableRequestParser.toJson(UpdateTableRequestParser.fromJson(json), true)) + .isEqualTo(expectedJson); + } + + @Test + public void roundTripSerdeWithoutIdentifier() { + String uuid = "2cc52516-5e73-41f2-b139-545d41a4e151"; + UpdateTableRequest request = + new UpdateTableRequest( + ImmutableList.of( + new org.apache.iceberg.UpdateRequirement.AssertTableUUID(uuid), + new org.apache.iceberg.UpdateRequirement.AssertTableDoesNotExist()), + ImmutableList.of( + new MetadataUpdate.AssignUUID(uuid), new MetadataUpdate.SetCurrentSchema(23))); + + String expectedJson = + "{\n" + + " \"requirements\" : [ {\n" + + " \"type\" : \"assert-table-uuid\",\n" + + " \"uuid\" : \"2cc52516-5e73-41f2-b139-545d41a4e151\"\n" + + " }, {\n" + + " \"type\" : \"assert-create\"\n" + + " } ],\n" + + " \"updates\" : [ {\n" + + " \"action\" : \"assign-uuid\",\n" + + " \"uuid\" : \"2cc52516-5e73-41f2-b139-545d41a4e151\"\n" + + " }, {\n" + + " \"action\" : \"set-current-schema\",\n" + + " \"schema-id\" : 23\n" + + " } ]\n" + + "}"; + + String json = UpdateTableRequestParser.toJson(request, true); + assertThat(json).isEqualTo(expectedJson); + + // can't do an equality comparison on UpdateTableRequest because updates/requirements + // don't implement equals/hashcode + assertThat(UpdateTableRequestParser.toJson(UpdateTableRequestParser.fromJson(json), true)) + .isEqualTo(expectedJson); + } + + @Test + public void emptyRequirementsAndUpdates() { + UpdateTableRequest request = + UpdateTableRequest.create( + TableIdentifier.of("ns1", "table1"), ImmutableList.of(), ImmutableList.of()); + + String json = + "{\"identifier\":{\"namespace\":[\"ns1\"],\"name\":\"table1\"},\"requirements\":[],\"updates\":[]}"; + + assertThat(UpdateTableRequestParser.toJson(request)).isEqualTo(json); + // can't do an equality comparison on UpdateTableRequest because updates/requirements + // don't implement equals/hashcode + assertThat(UpdateTableRequestParser.toJson(UpdateTableRequestParser.fromJson(json))) + .isEqualTo(json); + + assertThat(UpdateTableRequestParser.toJson(request)).isEqualTo(json); + // can't do an equality comparison on UpdateTableRequest because updates/requirements + // don't implement equals/hashcode + assertThat( + UpdateTableRequestParser.toJson( + UpdateTableRequestParser.fromJson( + "{\"identifier\":{\"namespace\":[\"ns1\"],\"name\":\"table1\"}}"))) + .isEqualTo(json); + } +} diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestCatalogErrorResponseParser.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestCatalogErrorResponseParser.java index 994b76f7d42a..e4fb413bed98 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestCatalogErrorResponseParser.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestCatalogErrorResponseParser.java @@ -21,8 +21,7 @@ import java.util.Arrays; import java.util.List; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestCatalogErrorResponseParser { @@ -36,10 +35,9 @@ public void testErrorResponseToJson() { String json = "{\"error\":" + errorModelJson + "}"; ErrorResponse response = ErrorResponse.builder().withMessage(message).withType(type).responseCode(code).build(); - Assert.assertEquals( - "Should be able to serialize an error response as json", - ErrorResponseParser.toJson(response), - json); + Assertions.assertThat(ErrorResponseParser.toJson(response)) + .as("Should be able to serialize an error response as json") + .isEqualTo(json); } @Test @@ -60,10 +58,9 @@ public void testErrorResponseToJsonWithStack() { .responseCode(code) .withStackTrace(stack) .build(); - Assert.assertEquals( - "Should be able to serialize an error response as json", - json, - ErrorResponseParser.toJson(response)); + Assertions.assertThat(ErrorResponseParser.toJson(response)) + .as("Should be able to serialize an error response as json") + .isEqualTo(json); } @Test diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestConfigResponse.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestConfigResponse.java index 223ee360cf34..298ebc3cf5bb 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestConfigResponse.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestConfigResponse.java @@ -20,13 +20,12 @@ import com.fasterxml.jackson.core.JsonProcessingException; import java.util.Map; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.rest.RequestResponseTestBase; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; public class TestConfigResponse extends RequestResponseTestBase { @@ -37,7 +36,7 @@ public class TestConfigResponse extends RequestResponseTestBase private static final Map DEFAULTS_WITH_NULL_VALUE = Maps.newHashMap(); private static final Map OVERRIDES_WITH_NULL_VALUE = Maps.newHashMap(); - @BeforeClass + @BeforeAll public static void beforeAllForRestCatalogConfig() { DEFAULTS_WITH_NULL_VALUE.put("warehouse", null); OVERRIDES_WITH_NULL_VALUE.put("clients", null); @@ -145,64 +144,52 @@ public void testCanUseNullAsPropertyValue() throws JsonProcessingException { public void testDeserializeInvalidResponse() { String jsonDefaultsHasWrongType = "{\"defaults\":[\"warehouse\",\"s3://bucket/warehouse\"],\"overrides\":{\"clients\":\"5\"}}"; - AssertHelpers.assertThrows( - "A JSON response with the wrong type for the defaults field should fail to deserialize", - JsonProcessingException.class, - () -> deserialize(jsonDefaultsHasWrongType)); + Assertions.assertThatThrownBy(() -> deserialize(jsonDefaultsHasWrongType)) + .isInstanceOf(JsonProcessingException.class) + .hasMessageContaining( + "Cannot deserialize value of type `java.util.LinkedHashMap`"); String jsonOverridesHasWrongType = "{\"defaults\":{\"warehouse\":\"s3://bucket/warehouse\"},\"overrides\":\"clients\"}"; - AssertHelpers.assertThrows( - "A JSON response with the wrong type for the overrides field should fail to deserialize", - JsonProcessingException.class, - () -> deserialize(jsonOverridesHasWrongType)); - - AssertHelpers.assertThrows( - "A null JSON response body should fail to deserialize", - IllegalArgumentException.class, - () -> deserialize(null)); + Assertions.assertThatThrownBy(() -> deserialize(jsonOverridesHasWrongType)) + .isInstanceOf(JsonProcessingException.class) + .hasMessageContaining("Cannot construct instance of `java.util.LinkedHashMap`"); + + Assertions.assertThatThrownBy(() -> deserialize(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("argument \"content\" is null"); } @Test public void testBuilderDoesNotCreateInvalidObjects() { - AssertHelpers.assertThrows( - "The builder should not allow using null as a key in the properties to override", - NullPointerException.class, - "Invalid override property: null", - () -> ConfigResponse.builder().withOverride(null, "100").build()); - - AssertHelpers.assertThrows( - "The builder should not allow using null as a key in the default properties", - NullPointerException.class, - "Invalid default property: null", - () -> ConfigResponse.builder().withDefault(null, "100").build()); - - AssertHelpers.assertThrows( - "The builder should not allow passing a null map of config properties to override", - NullPointerException.class, - "Invalid override properties map: null", - () -> ConfigResponse.builder().withOverrides(null).build()); - - AssertHelpers.assertThrows( - "The builder should not allow passing a null map of default config properties", - NullPointerException.class, - "Invalid default properties map: null", - () -> ConfigResponse.builder().withDefaults(null).build()); + Assertions.assertThatThrownBy(() -> ConfigResponse.builder().withOverride(null, "100").build()) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid override property: null"); + + Assertions.assertThatThrownBy(() -> ConfigResponse.builder().withDefault(null, "100").build()) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid default property: null"); + + Assertions.assertThatThrownBy(() -> ConfigResponse.builder().withOverrides(null).build()) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid override properties map: null"); + + Assertions.assertThatThrownBy(() -> ConfigResponse.builder().withDefaults(null).build()) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid default properties map: null"); Map mapWithNullKey = Maps.newHashMap(); mapWithNullKey.put(null, "a"); mapWithNullKey.put("b", "b"); - AssertHelpers.assertThrows( - "The builder should not allow passing a map of default config properties with a null key", - IllegalArgumentException.class, - "Invalid default property: null", - () -> ConfigResponse.builder().withDefaults(mapWithNullKey).build()); - - AssertHelpers.assertThrows( - "The builder should not allow passing a map of properties to override with a null key", - IllegalArgumentException.class, - "Invalid override property: null", - () -> ConfigResponse.builder().withOverrides(mapWithNullKey).build()); + Assertions.assertThatThrownBy( + () -> ConfigResponse.builder().withDefaults(mapWithNullKey).build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid default property: null"); + + Assertions.assertThatThrownBy( + () -> ConfigResponse.builder().withOverrides(mapWithNullKey).build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid override property: null"); } @Test @@ -226,12 +213,13 @@ public void testMergeStripsNullValuedEntries() { "b", "from_overrides", "c", "from_client"); - Assert.assertEquals( - "The merged properties map should use values from defaults, then client config, and finally overrides", - expected, - merged); - Assert.assertFalse( - "The merged properties map should omit keys with null values", merged.containsValue(null)); + Assertions.assertThat(merged) + .as( + "The merged properties map should use values from defaults, then client config, and finally overrides") + .isEqualTo(expected); + Assertions.assertThat(merged) + .as("The merged properties map should omit keys with null values") + .doesNotContainValue(null); } @Override @@ -246,14 +234,12 @@ public ConfigResponse createExampleInstance() { @Override public void assertEquals(ConfigResponse actual, ConfigResponse expected) { - Assert.assertEquals( - "Config properties to use as defaults should be equal", - actual.defaults(), - expected.defaults()); - Assert.assertEquals( - "Config properties to use as overrides should be equal", - actual.overrides(), - expected.overrides()); + Assertions.assertThat(actual.defaults()) + .as("Config properties to use as defaults should be equal") + .isEqualTo(expected.defaults()); + Assertions.assertThat(actual.overrides()) + .as("Config properties to use as overrides should be equal") + .isEqualTo(expected.overrides()); } @Override diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestCreateNamespaceResponse.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestCreateNamespaceResponse.java index df392563065a..ece9f1e688d1 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestCreateNamespaceResponse.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestCreateNamespaceResponse.java @@ -20,13 +20,12 @@ import com.fasterxml.jackson.core.JsonProcessingException; import java.util.Map; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.rest.RequestResponseTestBase; -import org.junit.Assert; -import org.junit.Test; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; public class TestCreateNamespaceResponse extends RequestResponseTestBase { @@ -85,69 +84,59 @@ public void testCanDeserializeWithoutDefaultValues() throws JsonProcessingExcept public void testDeserializeInvalidResponse() { String jsonResponseMalformedNamespaceValue = "{\"namespace\":\"accounting%1Ftax\",\"properties\":null}"; - AssertHelpers.assertThrows( - "A JSON response with the wrong type for the namespace field should fail to deserialize", - JsonProcessingException.class, - "Cannot parse string array from non-array", - () -> deserialize(jsonResponseMalformedNamespaceValue)); + Assertions.assertThatThrownBy(() -> deserialize(jsonResponseMalformedNamespaceValue)) + .isInstanceOf(JsonProcessingException.class) + .hasMessageContaining("Cannot parse string array from non-array"); String jsonResponsePropertiesHasWrongType = "{\"namespace\":[\"accounting\",\"tax\"],\"properties\":[]}"; - AssertHelpers.assertThrows( - "A JSON response with the wrong type for the properties field should fail to deserialize", - JsonProcessingException.class, - () -> deserialize(jsonResponsePropertiesHasWrongType)); + Assertions.assertThatThrownBy(() -> deserialize(jsonResponsePropertiesHasWrongType)) + .isInstanceOf(JsonProcessingException.class) + .hasMessageContaining( + "Cannot deserialize value of type `java.util.LinkedHashMap`"); - AssertHelpers.assertThrows( - "An empty JSON response should fail to deserialize", - IllegalArgumentException.class, - "Invalid namespace: null", - () -> deserialize("{}")); + Assertions.assertThatThrownBy(() -> deserialize("{}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid namespace: null"); String jsonMisspelledKeys = "{\"namepsace\":[\"accounting\",\"tax\"],\"propertiezzzz\":{\"owner\":\"Hank\"}}"; - AssertHelpers.assertThrows( - "A JSON response with the keys spelled incorrectly should fail to deserialize and validate", - IllegalArgumentException.class, - "Invalid namespace: null", - () -> deserialize(jsonMisspelledKeys)); - - AssertHelpers.assertThrows( - "A null JSON response body should fail to deserialize", - IllegalArgumentException.class, - () -> deserialize(null)); + Assertions.assertThatThrownBy(() -> deserialize(jsonMisspelledKeys)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid namespace: null"); + + Assertions.assertThatThrownBy(() -> deserialize(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("argument \"content\" is null"); } @Test public void testBuilderDoesNotBuildInvalidRequests() { - AssertHelpers.assertThrows( - "The builder should not allow using null for the namespace", - NullPointerException.class, - "Invalid namespace: null", - () -> CreateNamespaceResponse.builder().withNamespace(null).build()); - - AssertHelpers.assertThrows( - "The builder should not allow passing a null collection of properties", - NullPointerException.class, - "Invalid collection of properties: null", - () -> CreateNamespaceResponse.builder().setProperties(null).build()); + Assertions.assertThatThrownBy( + () -> CreateNamespaceResponse.builder().withNamespace(null).build()) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid namespace: null"); + + Assertions.assertThatThrownBy( + () -> CreateNamespaceResponse.builder().setProperties(null).build()) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid collection of properties: null"); Map mapWithNullKey = Maps.newHashMap(); mapWithNullKey.put(null, "hello"); - AssertHelpers.assertThrows( - "The builder should not allow using null as a key in the properties to set", - IllegalArgumentException.class, - "Invalid property to set: null", - () -> CreateNamespaceResponse.builder().setProperties(mapWithNullKey).build()); + Assertions.assertThatThrownBy( + () -> CreateNamespaceResponse.builder().setProperties(mapWithNullKey).build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid property to set: null"); Map mapWithMultipleNullValues = Maps.newHashMap(); mapWithMultipleNullValues.put("a", null); mapWithMultipleNullValues.put("b", "b"); - AssertHelpers.assertThrows( - "The builder should not allow using null as a value in the properties to set", - IllegalArgumentException.class, - "Invalid value to set for properties [a]: null", - () -> CreateNamespaceResponse.builder().setProperties(mapWithMultipleNullValues).build()); + Assertions.assertThatThrownBy( + () -> + CreateNamespaceResponse.builder().setProperties(mapWithMultipleNullValues).build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid value to set for properties [a]: null"); } @Override @@ -165,8 +154,8 @@ public CreateNamespaceResponse createExampleInstance() { @Override public void assertEquals(CreateNamespaceResponse actual, CreateNamespaceResponse expected) { - Assert.assertEquals("Namespaces should be equal", actual.namespace(), expected.namespace()); - Assert.assertEquals("Properties should be equal", actual.properties(), expected.properties()); + Assertions.assertThat(actual.namespace()).isEqualTo(expected.namespace()); + Assertions.assertThat(actual.properties()).isEqualTo(expected.properties()); } @Override diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestGetNamespaceResponse.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestGetNamespaceResponse.java index 2095ceb9a9f7..d228e9c45b2a 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestGetNamespaceResponse.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestGetNamespaceResponse.java @@ -20,13 +20,12 @@ import com.fasterxml.jackson.core.JsonProcessingException; import java.util.Map; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.rest.RequestResponseTestBase; -import org.junit.Assert; -import org.junit.Test; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; public class TestGetNamespaceResponse extends RequestResponseTestBase { @@ -67,71 +66,59 @@ public void testCanDeserializeWithoutDefaultValues() throws JsonProcessingExcept @Test public void testDeserializeInvalidResponse() { String jsonNamespaceHasWrongType = "{\"namespace\":\"accounting%1Ftax\",\"properties\":null}"; - AssertHelpers.assertThrows( - "A JSON response with the wrong type for a field should fail to deserialize", - JsonProcessingException.class, - "Cannot parse string array from non-array", - () -> deserialize(jsonNamespaceHasWrongType)); + Assertions.assertThatThrownBy(() -> deserialize(jsonNamespaceHasWrongType)) + .as("A JSON response with the wrong type for a field should fail to deserialize") + .isInstanceOf(JsonProcessingException.class) + .hasMessageContaining("Cannot parse string array from non-array"); String jsonPropertiesHasWrongType = "{\"namespace\":[\"accounting\",\"tax\"],\"properties\":[]}"; - AssertHelpers.assertThrows( - "A JSON response with the wrong type for a field should fail to deserialize", - JsonProcessingException.class, - () -> deserialize(jsonPropertiesHasWrongType)); + Assertions.assertThatThrownBy(() -> deserialize(jsonPropertiesHasWrongType)) + .isInstanceOf(JsonProcessingException.class) + .hasMessageContaining( + "Cannot deserialize value of type `java.util.LinkedHashMap`"); String emptyJson = "{}"; - AssertHelpers.assertThrows( - "An empty JSON request should fail to deserialize after validation", - IllegalArgumentException.class, - "Invalid namespace: null", - () -> deserialize(emptyJson)); + Assertions.assertThatThrownBy(() -> deserialize(emptyJson)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid namespace: null"); String jsonWithKeysSpelledIncorrectly = "{\"namepsace\":[\"accounting\",\"tax\"],\"propertiezzzz\":{\"owner\":\"Hank\"}}"; - AssertHelpers.assertThrows( - "A JSON response with the keys spelled incorrectly should fail to deserialize", - IllegalArgumentException.class, - "Invalid namespace: null", - () -> deserialize(jsonWithKeysSpelledIncorrectly)); + Assertions.assertThatThrownBy(() -> deserialize(jsonWithKeysSpelledIncorrectly)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid namespace: null"); String nullJson = null; - AssertHelpers.assertThrows( - "An empty JSON request should fail to deserialize", - IllegalArgumentException.class, - () -> deserialize(nullJson)); + Assertions.assertThatThrownBy(() -> deserialize(nullJson)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("argument \"content\" is null"); } @Test public void testBuilderDoesNotBuildInvalidRequests() { - AssertHelpers.assertThrows( - "The builder should not allow using null for the namespace", - NullPointerException.class, - "Invalid namespace: null", - () -> GetNamespaceResponse.builder().withNamespace(null).build()); - - AssertHelpers.assertThrows( - "The builder should not allow passing a null collection of properties", - NullPointerException.class, - "Invalid properties map: null", - () -> GetNamespaceResponse.builder().setProperties(null).build()); + Assertions.assertThatThrownBy(() -> GetNamespaceResponse.builder().withNamespace(null).build()) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid namespace: null"); + + Assertions.assertThatThrownBy(() -> GetNamespaceResponse.builder().setProperties(null).build()) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid properties map: null"); Map mapWithNullKey = Maps.newHashMap(); mapWithNullKey.put(null, "hello"); - AssertHelpers.assertThrows( - "The builder should not allow using null as a key in the properties to set", - IllegalArgumentException.class, - "Invalid property: null", - () -> GetNamespaceResponse.builder().setProperties(mapWithNullKey).build()); + Assertions.assertThatThrownBy( + () -> GetNamespaceResponse.builder().setProperties(mapWithNullKey).build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid property: null"); Map mapWithMultipleNullValues = Maps.newHashMap(); mapWithMultipleNullValues.put("a", null); mapWithMultipleNullValues.put("b", "b"); - AssertHelpers.assertThrows( - "The builder should not allow using null as a value in the properties to set", - IllegalArgumentException.class, - "Invalid value for properties [a]: null", - () -> GetNamespaceResponse.builder().setProperties(mapWithMultipleNullValues).build()); + Assertions.assertThatThrownBy( + () -> GetNamespaceResponse.builder().setProperties(mapWithMultipleNullValues).build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid value for properties [a]: null"); } @Override @@ -149,8 +136,8 @@ public GetNamespaceResponse createExampleInstance() { @Override public void assertEquals(GetNamespaceResponse actual, GetNamespaceResponse expected) { - Assert.assertEquals("Namespace should be equal", actual.namespace(), expected.namespace()); - Assert.assertEquals("Properties should be equal", actual.properties(), expected.properties()); + Assertions.assertThat(actual.namespace()).isEqualTo(expected.namespace()); + Assertions.assertThat(actual.properties()).isEqualTo(expected.properties()); } @Override diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestListNamespacesResponse.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestListNamespacesResponse.java index 2c2c7bbfeb0f..bfe5a662b219 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestListNamespacesResponse.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestListNamespacesResponse.java @@ -20,14 +20,12 @@ import com.fasterxml.jackson.core.JsonProcessingException; import java.util.List; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.rest.RequestResponseTestBase; -import org.junit.Assert; -import org.junit.Test; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; public class TestListNamespacesResponse extends RequestResponseTestBase { @@ -47,52 +45,42 @@ public void testRoundTripSerDe() throws JsonProcessingException { @Test public void testDeserializeInvalidResponseThrows() { String jsonNamespacesHasWrongType = "{\"namespaces\":\"accounting\"}"; - AssertHelpers.assertThrows( - "A malformed JSON response with the wrong type for a field should fail to deserialize", - JsonProcessingException.class, - () -> deserialize(jsonNamespacesHasWrongType)); + Assertions.assertThatThrownBy(() -> deserialize(jsonNamespacesHasWrongType)) + .isInstanceOf(JsonProcessingException.class) + .hasMessageContaining( + "Cannot deserialize value of type `java.util.ArrayList`"); String emptyJson = "{}"; - AssertHelpers.assertThrows( - "An empty JSON response will deserialize, but not into a valid object", - IllegalArgumentException.class, - "Invalid namespace: null", - () -> deserialize(emptyJson)); + Assertions.assertThatThrownBy(() -> deserialize(emptyJson)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid namespace: null"); String jsonWithKeysSpelledIncorrectly = "{\"namepsacezz\":[\"accounting\",\"tax\"]}"; - AssertHelpers.assertThrows( - "A JSON response with the keys spelled incorrectly should fail to deserialize", - IllegalArgumentException.class, - "Invalid namespace: null", - () -> deserialize(jsonWithKeysSpelledIncorrectly)); + Assertions.assertThatThrownBy(() -> deserialize(jsonWithKeysSpelledIncorrectly)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid namespace: null"); String nullJson = null; - AssertHelpers.assertThrows( - "A null JSON response should fail to deserialize", - IllegalArgumentException.class, - () -> deserialize(nullJson)); + Assertions.assertThatThrownBy(() -> deserialize(nullJson)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("argument \"content\" is null"); } @Test public void testBuilderDoesNotCreateInvalidObjects() { - AssertHelpers.assertThrows( - "The builder should not allow using null as a namespace to add to the list", - NullPointerException.class, - "Invalid namespace: null", - () -> ListNamespacesResponse.builder().add(null).build()); - - AssertHelpers.assertThrows( - "The builder should not allow passing a null list of namespaces to add", - NullPointerException.class, - "Invalid namespace list: null", - () -> ListNamespacesResponse.builder().addAll(null).build()); + Assertions.assertThatThrownBy(() -> ListNamespacesResponse.builder().add(null).build()) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid namespace: null"); + + Assertions.assertThatThrownBy(() -> ListNamespacesResponse.builder().addAll(null).build()) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid namespace list: null"); List listWithNullElement = Lists.newArrayList(Namespace.of("a"), null); - AssertHelpers.assertThrows( - "The builder should not allow passing a collection of namespaces with a null element in it", - IllegalArgumentException.class, - "Invalid namespace: null", - () -> ListNamespacesResponse.builder().addAll(listWithNullElement).build()); + Assertions.assertThatThrownBy( + () -> ListNamespacesResponse.builder().addAll(listWithNullElement).build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid namespace: null"); } @Override @@ -107,10 +95,10 @@ public ListNamespacesResponse createExampleInstance() { @Override public void assertEquals(ListNamespacesResponse actual, ListNamespacesResponse expected) { - Assert.assertTrue( - "Namespaces list should be equal", - actual.namespaces().size() == expected.namespaces().size() - && Sets.newHashSet(actual.namespaces()).equals(Sets.newHashSet(expected.namespaces()))); + Assertions.assertThat(actual.namespaces()) + .as("Namespaces list should be equal") + .hasSize(expected.namespaces().size()) + .containsExactlyInAnyOrderElementsOf(expected.namespaces()); } @Override diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestListTablesResponse.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestListTablesResponse.java index 144a967d3070..116d43a6d147 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestListTablesResponse.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestListTablesResponse.java @@ -20,15 +20,13 @@ import com.fasterxml.jackson.core.JsonProcessingException; import java.util.List; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.rest.RequestResponseTestBase; -import org.junit.Assert; -import org.junit.Test; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; public class TestListTablesResponse extends RequestResponseTestBase { @@ -49,75 +47,62 @@ public void testRoundTripSerDe() throws JsonProcessingException { @Test public void testDeserializeInvalidResponsesThrows() { String identifiersHasWrongType = "{\"identifiers\":\"accounting%1Ftax\"}"; - AssertHelpers.assertThrows( - "A JSON response with the incorrect type for the field identifiers should fail to parse", - JsonProcessingException.class, - () -> deserialize(identifiersHasWrongType)); + Assertions.assertThatThrownBy(() -> deserialize(identifiersHasWrongType)) + .isInstanceOf(JsonProcessingException.class) + .hasMessageContaining( + "Cannot deserialize value of type `java.util.ArrayList`"); String emptyJson = "{}"; - AssertHelpers.assertThrows( - "An empty JSON response should fail to deserialize", - IllegalArgumentException.class, - "Invalid identifier list: null", - () -> deserialize(emptyJson)); + Assertions.assertThatThrownBy(() -> deserialize(emptyJson)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid identifier list: null"); String jsonWithKeysSpelledIncorrectly = "{\"identifyrezzzz\":[{\"namespace\":[\"accounting\",\"tax\"],\"name\":\"paid\"}]}"; - AssertHelpers.assertThrows( - "A JSON response with the keys spelled incorrectly should fail to deserialize", - IllegalArgumentException.class, - "Invalid identifier list: null", - () -> deserialize(jsonWithKeysSpelledIncorrectly)); + Assertions.assertThatThrownBy(() -> deserialize(jsonWithKeysSpelledIncorrectly)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid identifier list: null"); String jsonWithInvalidIdentifiersInList = "{\"identifiers\":[{\"namespace\":\"accounting.tax\",\"name\":\"paid\"}]}"; - AssertHelpers.assertThrows( - "A JSON response with an invalid identifier in the list of identifiers should fail to parse", - JsonProcessingException.class, - () -> deserialize(jsonWithInvalidIdentifiersInList)); + Assertions.assertThatThrownBy(() -> deserialize(jsonWithInvalidIdentifiersInList)) + .isInstanceOf(JsonProcessingException.class) + .hasMessageContaining( + "Cannot parse JSON array from non-array value: namespace: \"accounting.tax\""); String jsonWithInvalidIdentifiersInList2 = "{\"identifiers\":[{\"namespace\":[\"accounting\",\"tax\"],\"name\":\"paid\"},\"accounting.tax.paid\"]}"; - AssertHelpers.assertThrows( - "A JSON response with an invalid identifier in the list of identifiers should fail to parse", - JsonProcessingException.class, - () -> deserialize(jsonWithInvalidIdentifiersInList2)); + Assertions.assertThatThrownBy(() -> deserialize(jsonWithInvalidIdentifiersInList2)) + .isInstanceOf(JsonProcessingException.class) + .hasMessageContaining("Cannot parse missing or non-object table identifier"); String jsonWithInvalidTypeForNamePartOfIdentifier = "{\"identifiers\":[{\"namespace\":[\"accounting\",\"tax\"],\"name\":true}]}"; - AssertHelpers.assertThrows( - "A JSON response with an invalid identifier in the list of identifiers should fail to parse", - JsonProcessingException.class, - () -> deserialize(jsonWithInvalidTypeForNamePartOfIdentifier)); + Assertions.assertThatThrownBy(() -> deserialize(jsonWithInvalidTypeForNamePartOfIdentifier)) + .isInstanceOf(JsonProcessingException.class) + .hasMessageContaining("Cannot parse to a string value"); String nullJson = null; - AssertHelpers.assertThrows( - "A null JSON response should fail to deserialize", - IllegalArgumentException.class, - () -> deserialize(nullJson)); + Assertions.assertThatThrownBy(() -> deserialize(nullJson)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("argument \"content\" is null"); } @Test public void testBuilderDoesNotCreateInvalidObjects() { - AssertHelpers.assertThrows( - "The builder should not allow using null as a table identifier to add to the list", - NullPointerException.class, - "Invalid table identifier: null", - () -> ListTablesResponse.builder().add(null).build()); - - AssertHelpers.assertThrows( - "The builder should not allow passing a null list of table identifiers to add", - NullPointerException.class, - "Invalid table identifier list: null", - () -> ListTablesResponse.builder().addAll(null).build()); + Assertions.assertThatThrownBy(() -> ListTablesResponse.builder().add(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid table identifier: null"); + + Assertions.assertThatThrownBy(() -> ListTablesResponse.builder().addAll(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid table identifier list: null"); List listWithNullElement = Lists.newArrayList(TableIdentifier.of(Namespace.of("foo"), "bar"), null); - AssertHelpers.assertThrows( - "The builder should not allow passing a collection of table identifiers with a null element in it", - IllegalArgumentException.class, - "Invalid table identifier: null", - () -> ListTablesResponse.builder().addAll(listWithNullElement).build()); + Assertions.assertThatThrownBy(() -> ListTablesResponse.builder().addAll(listWithNullElement)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid table identifier: null"); } @Override @@ -132,11 +117,10 @@ public ListTablesResponse createExampleInstance() { @Override public void assertEquals(ListTablesResponse actual, ListTablesResponse expected) { - Assert.assertTrue( - "Identifiers should be equal", - actual.identifiers().size() == expected.identifiers().size() - && Sets.newHashSet(actual.identifiers()) - .equals(Sets.newHashSet(expected.identifiers()))); + Assertions.assertThat(actual.identifiers()) + .as("Identifiers should be equal") + .hasSameSizeAs(expected.identifiers()) + .containsExactlyInAnyOrderElementsOf(expected.identifiers()); } @Override diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java index 48417173f7c3..0b65541fa415 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java @@ -24,7 +24,6 @@ import java.nio.file.Path; import java.nio.file.Paths; import java.util.Map; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.NullOrder; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; @@ -37,8 +36,8 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.rest.RequestResponseTestBase; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Test; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; public class TestLoadTableResponse extends RequestResponseTestBase { @@ -98,11 +97,9 @@ public LoadTableResponse deserialize(String json) throws JsonProcessingException @Test public void testFailures() { - AssertHelpers.assertThrows( - "Table metadata should be required", - NullPointerException.class, - "Invalid metadata: null", - () -> LoadTableResponse.builder().build()); + Assertions.assertThatThrownBy(() -> LoadTableResponse.builder().build()) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid metadata: null"); } @Test @@ -126,11 +123,10 @@ public void testRoundTripSerdeWithV1TableMetadata() throws Exception { public void testMissingSchemaType() throws Exception { // When the schema type (struct) is missing String tableMetadataJson = readTableMetadataInputFile("TableMetadataV1MissingSchemaType.json"); - AssertHelpers.assertThrows( - "Cannot parse type from json when there is no type", - IllegalArgumentException.class, - "Cannot parse type from json:", - () -> TableMetadataParser.fromJson(TEST_METADATA_LOCATION, tableMetadataJson)); + Assertions.assertThatThrownBy( + () -> TableMetadataParser.fromJson(TEST_METADATA_LOCATION, tableMetadataJson)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot parse type from json:"); } @Test @@ -161,86 +157,105 @@ public void testCanDeserializeWithoutDefaultValues() throws Exception { LoadTableResponse actual = deserialize(json); LoadTableResponse expected = LoadTableResponse.builder().withTableMetadata(metadata).build(); assertEquals(actual, expected); - Assert.assertEquals( - "Deserialized JSON with missing fields should have the default values", - ImmutableMap.of(), - actual.config()); + Assertions.assertThat(actual.config()) + .as("Deserialized JSON with missing fields should have the default values") + .isEqualTo(ImmutableMap.of()); } @Override public void assertEquals(LoadTableResponse actual, LoadTableResponse expected) { - Assert.assertEquals("Should have the same configuration", expected.config(), actual.config()); + Assertions.assertThat(actual.config()) + .as("Should have the same configuration") + .isEqualTo(expected.config()); assertEqualTableMetadata(actual.tableMetadata(), expected.tableMetadata()); - Assert.assertEquals( - "Should have the same metadata location", - expected.metadataLocation(), - actual.metadataLocation()); + Assertions.assertThat(actual.metadataLocation()) + .as("Should have the same metadata location") + .isEqualTo(expected.metadataLocation()); } private void assertEqualTableMetadata(TableMetadata actual, TableMetadata expected) { - Assert.assertEquals( - "Format version should match", expected.formatVersion(), actual.formatVersion()); - Assert.assertEquals("Table UUID should match", expected.uuid(), actual.uuid()); - Assert.assertEquals("Table location should match", expected.location(), actual.location()); - Assert.assertEquals("Last column id", expected.lastColumnId(), actual.lastColumnId()); - Assert.assertEquals( - "Schema should match", expected.schema().asStruct(), actual.schema().asStruct()); + Assertions.assertThat(actual.formatVersion()) + .as("Format version should match") + .isEqualTo(expected.formatVersion()); + Assertions.assertThat(actual.uuid()).as("Table UUID should match").isEqualTo(expected.uuid()); + Assertions.assertThat(actual.location()) + .as("Table location should match") + .isEqualTo(expected.location()); + Assertions.assertThat(actual.lastColumnId()) + .as("Last column id") + .isEqualTo(expected.lastColumnId()); + Assertions.assertThat(actual.schema().asStruct()) + .as("Schema should match") + .isEqualTo(expected.schema().asStruct()); assertSameSchemaList(expected.schemas(), actual.schemas()); - Assert.assertEquals( - "Current schema id should match", expected.currentSchemaId(), actual.currentSchemaId()); - Assert.assertEquals( - "Schema should match", expected.schema().asStruct(), actual.schema().asStruct()); - Assert.assertEquals( - "Last sequence number should match", - expected.lastSequenceNumber(), - actual.lastSequenceNumber()); - Assert.assertEquals( - "Partition spec should match", expected.spec().toString(), actual.spec().toString()); - Assert.assertEquals( - "Default spec ID should match", expected.defaultSpecId(), actual.defaultSpecId()); - Assert.assertEquals("PartitionSpec map should match", expected.specs(), actual.specs()); - Assert.assertEquals( - "Default Sort ID should match", expected.defaultSortOrderId(), actual.defaultSortOrderId()); - Assert.assertEquals("Sort order should match", expected.sortOrder(), actual.sortOrder()); - Assert.assertEquals("Sort order map should match", expected.sortOrders(), actual.sortOrders()); - Assert.assertEquals("Properties should match", expected.properties(), actual.properties()); - Assert.assertEquals( - "Snapshots should match", - Lists.transform(expected.snapshots(), Snapshot::snapshotId), - Lists.transform(actual.snapshots(), Snapshot::snapshotId)); - Assert.assertEquals("History should match", expected.snapshotLog(), actual.snapshotLog()); + Assertions.assertThat(actual.currentSchemaId()) + .as("Current schema id should match") + .isEqualTo(expected.currentSchemaId()); + Assertions.assertThat(actual.schema().asStruct()) + .as("Schema should match") + .isEqualTo(expected.schema().asStruct()); + Assertions.assertThat(actual.lastSequenceNumber()) + .as("Last sequence number should match") + .isEqualTo(expected.lastSequenceNumber()); + Assertions.assertThat(actual.spec().toString()) + .as("Partition spec should match") + .isEqualTo(expected.spec().toString()); + Assertions.assertThat(actual.defaultSpecId()) + .as("Default spec ID should match") + .isEqualTo(expected.defaultSpecId()); + Assertions.assertThat(actual.specs()) + .as("PartitionSpec map should match") + .isEqualTo(expected.specs()); + Assertions.assertThat(actual.defaultSortOrderId()) + .as("Default Sort ID should match") + .isEqualTo(expected.defaultSortOrderId()); + Assertions.assertThat(actual.sortOrder()) + .as("Sort order should match") + .isEqualTo(expected.sortOrder()); + Assertions.assertThat(actual.sortOrders()) + .as("Sort order map should match") + .isEqualTo(expected.sortOrders()); + Assertions.assertThat(actual.properties()) + .as("Properties should match") + .isEqualTo(expected.properties()); + Assertions.assertThat(Lists.transform(actual.snapshots(), Snapshot::snapshotId)) + .as("Snapshots should match") + .isEqualTo(Lists.transform(expected.snapshots(), Snapshot::snapshotId)); + Assertions.assertThat(actual.snapshotLog()) + .as("History should match") + .isEqualTo(expected.snapshotLog()); Snapshot expectedCurrentSnapshot = expected.currentSnapshot(); Snapshot actualCurrentSnapshot = actual.currentSnapshot(); - Assert.assertTrue( - "Both expected and actual current snapshot should either be null or non-null", - (expectedCurrentSnapshot != null && actualCurrentSnapshot != null) - || (expectedCurrentSnapshot == null && actualCurrentSnapshot == null)); + Assertions.assertThat( + expectedCurrentSnapshot != null && actualCurrentSnapshot != null + || expectedCurrentSnapshot == null && actualCurrentSnapshot == null) + .as("Both expected and actual current snapshot should either be null or non-null") + .isTrue(); if (expectedCurrentSnapshot != null) { - Assert.assertEquals( - "Current snapshot ID should match", - expected.currentSnapshot().snapshotId(), - actual.currentSnapshot().snapshotId()); - Assert.assertEquals( - "Parent snapshot ID should match", - expected.currentSnapshot().parentId(), - actual.currentSnapshot().parentId()); - Assert.assertEquals( - "Schema ID for current snapshot should match", - expected.currentSnapshot().schemaId(), - actual.currentSnapshot().schemaId()); + Assertions.assertThat(actual.currentSnapshot().snapshotId()) + .as("Current snapshot ID should match") + .isEqualTo(expected.currentSnapshot().snapshotId()); + Assertions.assertThat(actual.currentSnapshot().parentId()) + .as("Parent snapshot ID should match") + .isEqualTo(expected.currentSnapshot().parentId()); + Assertions.assertThat(actual.currentSnapshot().schemaId()) + .as("Schema ID for current snapshot should match") + .isEqualTo(expected.currentSnapshot().schemaId()); } - Assert.assertEquals( - "Metadata file location should match", - expected.metadataFileLocation(), - actual.metadataFileLocation()); - Assert.assertEquals( - "Last column id should match", expected.lastColumnId(), actual.lastColumnId()); - Assert.assertEquals( - "Schema should match", expected.schema().asStruct(), actual.schema().asStruct()); + Assertions.assertThat(actual.metadataFileLocation()) + .as("Metadata file location should match") + .isEqualTo(expected.metadataFileLocation()); + Assertions.assertThat(actual.lastColumnId()) + .as("Last column id should match") + .isEqualTo(expected.lastColumnId()); + Assertions.assertThat(actual.schema().asStruct()) + .as("Schema should match") + .isEqualTo(expected.schema().asStruct()); assertSameSchemaList(expected.schemas(), actual.schemas()); - Assert.assertEquals( - "Current schema id should match", expected.currentSchemaId(), actual.currentSchemaId()); - Assert.assertEquals("Refs map should match", expected.refs(), actual.refs()); + Assertions.assertThat(actual.currentSchemaId()) + .as("Current schema id should match") + .isEqualTo(expected.currentSchemaId()); + Assertions.assertThat(actual.refs()).as("Refs map should match").isEqualTo(expected.refs()); } private String readTableMetadataInputFile(String fileName) throws Exception { diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestOAuthErrorResponseParser.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestOAuthErrorResponseParser.java index 1624bc08f476..96953d3e6069 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestOAuthErrorResponseParser.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestOAuthErrorResponseParser.java @@ -20,7 +20,7 @@ import org.apache.iceberg.rest.auth.OAuth2Properties; import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestOAuthErrorResponseParser { diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestOAuthTokenResponse.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestOAuthTokenResponse.java index ac52cd24cebc..cf568ac0c6a1 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestOAuthTokenResponse.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestOAuthTokenResponse.java @@ -19,11 +19,10 @@ package org.apache.iceberg.rest.responses; import com.fasterxml.jackson.core.JsonProcessingException; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.rest.RequestResponseTestBase; import org.apache.iceberg.rest.auth.OAuth2Util; -import org.junit.Assert; -import org.junit.Test; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; public class TestOAuthTokenResponse extends RequestResponseTestBase { @Override @@ -44,13 +43,17 @@ public OAuthTokenResponse createExampleInstance() { @Override public void assertEquals(OAuthTokenResponse actual, OAuthTokenResponse expected) { - Assert.assertEquals("Token should match", expected.token(), actual.token()); - Assert.assertEquals("Token type should match", expected.tokenType(), actual.tokenType()); - Assert.assertEquals( - "Issued token type should match", expected.issuedTokenType(), actual.issuedTokenType()); - Assert.assertEquals( - "Expiration should match", expected.expiresInSeconds(), actual.expiresInSeconds()); - Assert.assertEquals("Scope should match", expected.scopes(), actual.scopes()); + Assertions.assertThat(actual.token()).as("Token should match").isEqualTo(expected.token()); + Assertions.assertThat(actual.tokenType()) + .as("Token type should match") + .isEqualTo(expected.tokenType()); + Assertions.assertThat(actual.issuedTokenType()) + .as("Issued token type should match") + .isEqualTo(expected.issuedTokenType()); + Assertions.assertThat(actual.expiresInSeconds()) + .as("Expiration should match") + .isEqualTo(expected.expiresInSeconds()); + Assertions.assertThat(actual.scopes()).as("Scope should match").isEqualTo(expected.scopes()); } @Override @@ -111,28 +114,22 @@ public void testRoundTrip() throws Exception { @Test public void testFailures() { - AssertHelpers.assertThrows( - "Token should be required", - IllegalArgumentException.class, - "missing string: access_token", - () -> deserialize("{\"token_type\":\"bearer\"}")); + Assertions.assertThatThrownBy(() -> deserialize("{\"token_type\":\"bearer\"}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("missing string: access_token"); - AssertHelpers.assertThrows( - "Token should be string", - IllegalArgumentException.class, - "Cannot parse to a string value: access_token: 34", - () -> deserialize("{\"access_token\":34,\"token_type\":\"bearer\"}")); + Assertions.assertThatThrownBy( + () -> deserialize("{\"access_token\":34,\"token_type\":\"bearer\"}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot parse to a string value: access_token: 34"); - AssertHelpers.assertThrows( - "Token type should be required", - IllegalArgumentException.class, - "missing string: token_type", - () -> deserialize("{\"access_token\":\"bearer-token\"}")); + Assertions.assertThatThrownBy(() -> deserialize("{\"access_token\":\"bearer-token\"}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("missing string: token_type"); - AssertHelpers.assertThrows( - "Token type should be string", - IllegalArgumentException.class, - "Cannot parse to a string value: token_type: 34", - () -> deserialize("{\"access_token\":\"bearer-token\",\"token_type\":34}")); + Assertions.assertThatThrownBy( + () -> deserialize("{\"access_token\":\"bearer-token\",\"token_type\":34}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot parse to a string value: token_type: 34"); } } diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestUpdateNamespacePropertiesResponse.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestUpdateNamespacePropertiesResponse.java index 4fbf386c02a0..17176eff717d 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestUpdateNamespacePropertiesResponse.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestUpdateNamespacePropertiesResponse.java @@ -20,13 +20,11 @@ import com.fasterxml.jackson.core.JsonProcessingException; import java.util.List; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.rest.RequestResponseTestBase; -import org.junit.Assert; -import org.junit.Test; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; public class TestUpdateNamespacePropertiesResponse extends RequestResponseTestBase { @@ -145,30 +143,27 @@ public void testDeserializeInvalidResponse() { // Invalid top-level types String jsonInvalidTypeOnRemovedField = "{\"removed\":{\"foo\":true},\"updated\":[\"owner\"],\"missing\":[\"bar\"]}"; - AssertHelpers.assertThrows( - "A JSON response with an invalid type for one of the fields should fail to parse", - JsonProcessingException.class, - () -> deserialize(jsonInvalidTypeOnRemovedField)); + Assertions.assertThatThrownBy(() -> deserialize(jsonInvalidTypeOnRemovedField)) + .isInstanceOf(JsonProcessingException.class) + .hasMessageContaining( + "Cannot deserialize value of type `java.util.ArrayList`"); String jsonInvalidTypeOnUpdatedField = "{\"updated\":\"owner\",\"missing\":[\"bar\"]}"; - AssertHelpers.assertThrows( - "A JSON response with an invalid type for one of the fields should fail to parse", - JsonProcessingException.class, - () -> deserialize(jsonInvalidTypeOnUpdatedField)); + Assertions.assertThatThrownBy(() -> deserialize(jsonInvalidTypeOnUpdatedField)) + .isInstanceOf(JsonProcessingException.class) + .hasMessageContaining("Cannot construct instance of `java.util.ArrayList`"); // Valid top-level (array) types, but at least one entry in the list is not the expected type String jsonInvalidValueOfTypeIntNestedInRemovedList = "{\"removed\":[\"foo\", \"bar\", 123456], ,\"updated\":[\"owner\"],\"missing\":[\"bar\"]}"; - AssertHelpers.assertThrows( - "A JSON response with an invalid type inside one of the list fields should fail to deserialize", - JsonProcessingException.class, - () -> deserialize(jsonInvalidValueOfTypeIntNestedInRemovedList)); + Assertions.assertThatThrownBy(() -> deserialize(jsonInvalidValueOfTypeIntNestedInRemovedList)) + .isInstanceOf(JsonProcessingException.class) + .hasMessageContaining("Unexpected character (',' (code 44))"); // Exception comes from Jackson - AssertHelpers.assertThrows( - "A null JSON response body should fail to deserialize", - IllegalArgumentException.class, - () -> deserialize(null)); + Assertions.assertThatThrownBy(() -> deserialize(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("argument \"content\" is null"); } @Test @@ -176,61 +171,58 @@ public void testBuilderDoesNotCreateInvalidObjects() { List listContainingNull = Lists.newArrayList("a", null, null); // updated - AssertHelpers.assertThrows( - "The builder should not allow using null as a property that was updated", - NullPointerException.class, - "Invalid updated property: null", - () -> UpdateNamespacePropertiesResponse.builder().addUpdated((String) null).build()); - - AssertHelpers.assertThrows( - "The builder should not allow passing a null list of properties that were removed", - NullPointerException.class, - "Invalid updated property list: null", - () -> UpdateNamespacePropertiesResponse.builder().addUpdated((List) null).build()); - - AssertHelpers.assertThrows( - "The builder should not allow passing a list of properties that were removed with a null element", - IllegalArgumentException.class, - "Invalid updated property: null", - () -> UpdateNamespacePropertiesResponse.builder().addUpdated(listContainingNull).build()); + Assertions.assertThatThrownBy( + () -> UpdateNamespacePropertiesResponse.builder().addUpdated((String) null).build()) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid updated property: null"); + + Assertions.assertThatThrownBy( + () -> + UpdateNamespacePropertiesResponse.builder().addUpdated((List) null).build()) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid updated property list: null"); + + Assertions.assertThatThrownBy( + () -> + UpdateNamespacePropertiesResponse.builder().addUpdated(listContainingNull).build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid updated property: null"); // removed - AssertHelpers.assertThrows( - "The builder should not allow using null as a property that was removed", - NullPointerException.class, - "Invalid removed property: null", - () -> UpdateNamespacePropertiesResponse.builder().addRemoved((String) null).build()); - - AssertHelpers.assertThrows( - "The builder should not allow passing a null list of properties that were removed", - NullPointerException.class, - "Invalid removed property list: null", - () -> UpdateNamespacePropertiesResponse.builder().addRemoved((List) null).build()); - - AssertHelpers.assertThrows( - "The builder should not allow passing a list of properties that were removed with a null element", - IllegalArgumentException.class, - "Invalid removed property: null", - () -> UpdateNamespacePropertiesResponse.builder().addRemoved(listContainingNull).build()); + Assertions.assertThatThrownBy( + () -> UpdateNamespacePropertiesResponse.builder().addRemoved((String) null).build()) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid removed property: null"); + + Assertions.assertThatThrownBy( + () -> + UpdateNamespacePropertiesResponse.builder().addRemoved((List) null).build()) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid removed property list: null"); + + Assertions.assertThatThrownBy( + () -> + UpdateNamespacePropertiesResponse.builder().addRemoved(listContainingNull).build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid removed property: null"); // missing - AssertHelpers.assertThrows( - "The builder should not allow using null as a property that was missing", - NullPointerException.class, - "Invalid missing property: null", - () -> UpdateNamespacePropertiesResponse.builder().addMissing((String) null).build()); - - AssertHelpers.assertThrows( - "The builder should not allow passing a null list of properties that were missing", - NullPointerException.class, - "Invalid missing property list: null", - () -> UpdateNamespacePropertiesResponse.builder().addMissing((List) null).build()); - - AssertHelpers.assertThrows( - "The builder should not allow passing a list of properties that were missing with a null element", - IllegalArgumentException.class, - "Invalid missing property: null", - () -> UpdateNamespacePropertiesResponse.builder().addMissing(listContainingNull).build()); + Assertions.assertThatThrownBy( + () -> UpdateNamespacePropertiesResponse.builder().addMissing((String) null).build()) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid missing property: null"); + + Assertions.assertThatThrownBy( + () -> + UpdateNamespacePropertiesResponse.builder().addMissing((List) null).build()) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid missing property list: null"); + + Assertions.assertThatThrownBy( + () -> + UpdateNamespacePropertiesResponse.builder().addMissing(listContainingNull).build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid missing property: null"); } @Override @@ -250,18 +242,15 @@ public UpdateNamespacePropertiesResponse createExampleInstance() { @Override public void assertEquals( UpdateNamespacePropertiesResponse actual, UpdateNamespacePropertiesResponse expected) { - Assert.assertEquals( - "Properties updated should be equal", - Sets.newHashSet(actual.updated()), - Sets.newHashSet(expected.updated())); - Assert.assertEquals( - "Properties removed should be equal", - Sets.newHashSet(actual.removed()), - Sets.newHashSet(expected.removed())); - Assert.assertEquals( - "Properties missing should be equal", - Sets.newHashSet(actual.missing()), - Sets.newHashSet(expected.missing())); + Assertions.assertThat(actual.updated()) + .as("Properties updated should be equal") + .containsExactlyInAnyOrderElementsOf(expected.updated()); + Assertions.assertThat(actual.removed()) + .as("Properties removed should be equal") + .containsExactlyInAnyOrderElementsOf(expected.removed()); + Assertions.assertThat(actual.missing()) + .as("Properties missing should be equal") + .containsExactlyInAnyOrderElementsOf(expected.missing()); } @Override diff --git a/core/src/test/java/org/apache/iceberg/util/TestBinPacking.java b/core/src/test/java/org/apache/iceberg/util/TestBinPacking.java index 59d64d542ba7..f0cc6db1ad6e 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestBinPacking.java +++ b/core/src/test/java/org/apache/iceberg/util/TestBinPacking.java @@ -18,178 +18,149 @@ */ package org.apache.iceberg.util; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.List; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.BinPacking.ListPacker; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestBinPacking { @Test public void testBasicBinPacking() { - Assert.assertEquals( - "Should pack the first 2 values", - list(list(1, 2), list(3), list(4), list(5)), - pack(list(1, 2, 3, 4, 5), 3)); - - Assert.assertEquals( - "Should pack the first 2 values", - list(list(1, 2), list(3), list(4), list(5)), - pack(list(1, 2, 3, 4, 5), 5)); - - Assert.assertEquals( - "Should pack the first 3 values", - list(list(1, 2, 3), list(4), list(5)), - pack(list(1, 2, 3, 4, 5), 6)); - - Assert.assertEquals( - "Should pack the first 3 values", - list(list(1, 2, 3), list(4), list(5)), - pack(list(1, 2, 3, 4, 5), 8)); - - Assert.assertEquals( - "Should pack the first 3 values, last 2 values", - list(list(1, 2, 3), list(4, 5)), - pack(list(1, 2, 3, 4, 5), 9)); - - Assert.assertEquals( - "Should pack the first 4 values", - list(list(1, 2, 3, 4), list(5)), - pack(list(1, 2, 3, 4, 5), 10)); - - Assert.assertEquals( - "Should pack the first 4 values", - list(list(1, 2, 3, 4), list(5)), - pack(list(1, 2, 3, 4, 5), 14)); - - Assert.assertEquals( - "Should pack the first 5 values", list(list(1, 2, 3, 4, 5)), pack(list(1, 2, 3, 4, 5), 15)); + assertThat(pack(list(1, 2, 3, 4, 5), 3)) + .as("Should pack the first 2 values") + .isEqualTo(list(list(1, 2), list(3), list(4), list(5))); + + assertThat(pack(list(1, 2, 3, 4, 5), 5)) + .as("Should pack the first 2 values") + .isEqualTo(list(list(1, 2), list(3), list(4), list(5))); + + assertThat(pack(list(1, 2, 3, 4, 5), 6)) + .as("Should pack the first 3 values") + .isEqualTo(list(list(1, 2, 3), list(4), list(5))); + + assertThat(pack(list(1, 2, 3, 4, 5), 8)) + .as("Should pack the first 3 values") + .isEqualTo(list(list(1, 2, 3), list(4), list(5))); + + assertThat(pack(list(1, 2, 3, 4, 5), 9)) + .as("Should pack the first 3 values, last 2 values") + .isEqualTo(list(list(1, 2, 3), list(4, 5))); + + assertThat(pack(list(1, 2, 3, 4, 5), 10)) + .as("Should pack the first 4 values") + .isEqualTo(list(list(1, 2, 3, 4), list(5))); + + assertThat(pack(list(1, 2, 3, 4, 5), 14)) + .as("Should pack the first 4 values") + .isEqualTo(list(list(1, 2, 3, 4), list(5))); + + assertThat(pack(list(1, 2, 3, 4, 5), 15)) + .as("Should pack the first 5 values") + .isEqualTo(list(list(1, 2, 3, 4, 5))); } @Test public void testReverseBinPackingSingleLookback() { - Assert.assertEquals( - "Should pack the first 2 values", - list(list(1, 2), list(3), list(4), list(5)), - packEnd(list(1, 2, 3, 4, 5), 3, 1)); - - Assert.assertEquals( - "Should pack the first 2 values", - list(list(1, 2), list(3), list(4), list(5)), - packEnd(list(1, 2, 3, 4, 5), 4, 1)); - - Assert.assertEquals( - "Should pack the second and third values", - list(list(1), list(2, 3), list(4), list(5)), - packEnd(list(1, 2, 3, 4, 5), 5, 1)); - - Assert.assertEquals( - "Should pack the first 3 values", - list(list(1, 2, 3), list(4), list(5)), - packEnd(list(1, 2, 3, 4, 5), 6, 1)); - - Assert.assertEquals( - "Should pack the first two pairs of values", - list(list(1, 2), list(3, 4), list(5)), - packEnd(list(1, 2, 3, 4, 5), 7, 1)); - - Assert.assertEquals( - "Should pack the first two pairs of values", - list(list(1, 2), list(3, 4), list(5)), - packEnd(list(1, 2, 3, 4, 5), 8, 1)); - - Assert.assertEquals( - "Should pack the first 3 values, last 2 values", - list(list(1, 2, 3), list(4, 5)), - packEnd(list(1, 2, 3, 4, 5), 9, 1)); - - Assert.assertEquals( - "Should pack the first 3 values, last 2 values", - list(list(1, 2, 3), list(4, 5)), - packEnd(list(1, 2, 3, 4, 5), 11, 1)); - - Assert.assertEquals( - "Should pack the first 3 values, last 2 values", - list(list(1, 2), list(3, 4, 5)), - packEnd(list(1, 2, 3, 4, 5), 12, 1)); - - Assert.assertEquals( - "Should pack the last 4 values", - list(list(1), list(2, 3, 4, 5)), - packEnd(list(1, 2, 3, 4, 5), 14, 1)); - - Assert.assertEquals( - "Should pack the first 5 values", - list(list(1, 2, 3, 4, 5)), - packEnd(list(1, 2, 3, 4, 5), 15, 1)); + assertThat(packEnd(list(1, 2, 3, 4, 5), 3, 1)) + .as("Should pack the first 2 values") + .isEqualTo(list(list(1, 2), list(3), list(4), list(5))); + + assertThat(packEnd(list(1, 2, 3, 4, 5), 4, 1)) + .as("Should pack the first 2 values") + .isEqualTo(list(list(1, 2), list(3), list(4), list(5))); + + assertThat(packEnd(list(1, 2, 3, 4, 5), 5, 1)) + .as("Should pack the second and third values") + .isEqualTo(list(list(1), list(2, 3), list(4), list(5))); + + assertThat(packEnd(list(1, 2, 3, 4, 5), 6, 1)) + .as("Should pack the first 3 values") + .isEqualTo(list(list(1, 2, 3), list(4), list(5))); + + assertThat(packEnd(list(1, 2, 3, 4, 5), 7, 1)) + .as("Should pack the first two pairs of values") + .isEqualTo(list(list(1, 2), list(3, 4), list(5))); + + assertThat(packEnd(list(1, 2, 3, 4, 5), 8, 1)) + .as("Should pack the first two pairs of values") + .isEqualTo(list(list(1, 2), list(3, 4), list(5))); + + assertThat(packEnd(list(1, 2, 3, 4, 5), 9, 1)) + .as("Should pack the first 3 values, last 2 values") + .isEqualTo(list(list(1, 2, 3), list(4, 5))); + + assertThat(packEnd(list(1, 2, 3, 4, 5), 11, 1)) + .as("Should pack the first 3 values, last 2 values") + .isEqualTo(list(list(1, 2, 3), list(4, 5))); + + assertThat(packEnd(list(1, 2, 3, 4, 5), 12, 1)) + .as("Should pack the first 3 values, last 2 values") + .isEqualTo(list(list(1, 2), list(3, 4, 5))); + + assertThat(packEnd(list(1, 2, 3, 4, 5), 14, 1)) + .as("Should pack the last 4 values") + .isEqualTo(list(list(1), list(2, 3, 4, 5))); + + assertThat(packEnd(list(1, 2, 3, 4, 5), 15, 1)) + .as("Should pack the first 5 values") + .isEqualTo(list(list(1, 2, 3, 4, 5))); } @Test public void testReverseBinPackingUnlimitedLookback() { - Assert.assertEquals( - "Should pack the first 2 values", - list(list(1, 2), list(3), list(4), list(5)), - packEnd(list(1, 2, 3, 4, 5), 3)); - - Assert.assertEquals( - "Should pack 1 with 3", - list(list(2), list(1, 3), list(4), list(5)), - packEnd(list(1, 2, 3, 4, 5), 4)); - - Assert.assertEquals( - "Should pack 2,3 and 1,4", - list(list(2, 3), list(1, 4), list(5)), - packEnd(list(1, 2, 3, 4, 5), 5)); - - Assert.assertEquals( - "Should pack 2,4 and 1,5", - list(list(3), list(2, 4), list(1, 5)), - packEnd(list(1, 2, 3, 4, 5), 6)); - - Assert.assertEquals( - "Should pack 3,4 and 2,5", - list(list(1), list(3, 4), list(2, 5)), - packEnd(list(1, 2, 3, 4, 5), 7)); - - Assert.assertEquals( - "Should pack 1,2,3 and 3,5", - list(list(1, 2, 4), list(3, 5)), - packEnd(list(1, 2, 3, 4, 5), 8)); - - Assert.assertEquals( - "Should pack the first 3 values, last 2 values", - list(list(1, 2, 3), list(4, 5)), - packEnd(list(1, 2, 3, 4, 5), 9)); - - Assert.assertEquals( - "Should pack 2,3 and 1,4,5", - list(list(2, 3), list(1, 4, 5)), - packEnd(list(1, 2, 3, 4, 5), 10)); - - Assert.assertEquals( - "Should pack 1,3 and 2,4,5", - list(list(1, 3), list(2, 4, 5)), - packEnd(list(1, 2, 3, 4, 5), 11)); - - Assert.assertEquals( - "Should pack 1,2 and 3,4,5", - list(list(1, 2), list(3, 4, 5)), - packEnd(list(1, 2, 3, 4, 5), 12)); - - Assert.assertEquals( - "Should pack 1,2 and 3,4,5", - list(list(2), list(1, 3, 4, 5)), - packEnd(list(1, 2, 3, 4, 5), 13)); - - Assert.assertEquals( - "Should pack the last 4 values", - list(list(1), list(2, 3, 4, 5)), - packEnd(list(1, 2, 3, 4, 5), 14)); - - Assert.assertEquals( - "Should pack the first 5 values", - list(list(1, 2, 3, 4, 5)), - packEnd(list(1, 2, 3, 4, 5), 15)); + assertThat(packEnd(list(1, 2, 3, 4, 5), 3)) + .as("Should pack the first 2 values") + .isEqualTo(list(list(1, 2), list(3), list(4), list(5))); + + assertThat(packEnd(list(1, 2, 3, 4, 5), 4)) + .as("Should pack 1 with 3") + .isEqualTo(list(list(2), list(1, 3), list(4), list(5))); + + assertThat(packEnd(list(1, 2, 3, 4, 5), 5)) + .as("Should pack 2,3 and 1,4") + .isEqualTo(list(list(2, 3), list(1, 4), list(5))); + + assertThat(packEnd(list(1, 2, 3, 4, 5), 6)) + .as("Should pack 2,4 and 1,5") + .isEqualTo(list(list(3), list(2, 4), list(1, 5))); + + assertThat(packEnd(list(1, 2, 3, 4, 5), 7)) + .as("Should pack 3,4 and 2,5") + .isEqualTo(list(list(1), list(3, 4), list(2, 5))); + + assertThat(packEnd(list(1, 2, 3, 4, 5), 8)) + .as("Should pack 1,2,3 and 3,5") + .isEqualTo(list(list(1, 2, 4), list(3, 5))); + + assertThat(packEnd(list(1, 2, 3, 4, 5), 9)) + .as("Should pack the first 3 values, last 2 values") + .isEqualTo(list(list(1, 2, 3), list(4, 5))); + + assertThat(packEnd(list(1, 2, 3, 4, 5), 10)) + .as("Should pack 2,3 and 1,4,5") + .isEqualTo(list(list(2, 3), list(1, 4, 5))); + + assertThat(packEnd(list(1, 2, 3, 4, 5), 11)) + .as("Should pack 1,3 and 2,4,5") + .isEqualTo(list(list(1, 3), list(2, 4, 5))); + + assertThat(packEnd(list(1, 2, 3, 4, 5), 12)) + .as("Should pack 1,2 and 3,4,5") + .isEqualTo(list(list(1, 2), list(3, 4, 5))); + + assertThat(packEnd(list(1, 2, 3, 4, 5), 13)) + .as("Should pack 1,2 and 3,4,5") + .isEqualTo(list(list(2), list(1, 3, 4, 5))); + + assertThat(packEnd(list(1, 2, 3, 4, 5), 14)) + .as("Should pack the last 4 values") + .isEqualTo(list(list(1), list(2, 3, 4, 5))); + + assertThat(packEnd(list(1, 2, 3, 4, 5), 15)) + .as("Should pack the first 5 values") + .isEqualTo(list(list(1, 2, 3, 4, 5))); } @Test @@ -201,10 +172,9 @@ public void testBinPackingLookBack() { // 4. [5, 1, 1], [5] // 5. [5, 1, 1], [5], [5] // 6. [5, 1, 1, 1], [5], [5] - Assert.assertEquals( - "Unlimited look-back: should merge ones into first bin", - list(list(5, 1, 1, 1), list(5), list(5)), - pack(list(5, 1, 5, 1, 5, 1), 8)); + assertThat(pack(list(5, 1, 5, 1, 5, 1), 8)) + .as("Unlimited look-back: should merge ones into first bin") + .isEqualTo(list(list(5, 1, 1, 1), list(5), list(5))); // lookback state: // 1. [5] @@ -213,10 +183,9 @@ public void testBinPackingLookBack() { // 4. [5, 1, 1], [5] // 5. [5], [5] ([5, 1, 1] drops out of look-back) // 6. [5, 1], [5] - Assert.assertEquals( - "2 bin look-back: should merge two ones into first bin", - list(list(5, 1, 1), list(5, 1), list(5)), - pack(list(5, 1, 5, 1, 5, 1), 8, 2)); + assertThat(pack(list(5, 1, 5, 1, 5, 1), 8, 2)) + .as("2 bin look-back: should merge two ones into first bin") + .isEqualTo(list(list(5, 1, 1), list(5, 1), list(5))); // lookback state: // 1. [5] @@ -225,20 +194,17 @@ public void testBinPackingLookBack() { // 4. [5, 1] // 5. [5] ([5, 1] #2 drops out of look-back) // 6. [5, 1] - Assert.assertEquals( - "1 bin look-back: should merge ones with fives", - list(list(5, 1), list(5, 1), list(5, 1)), - pack(list(5, 1, 5, 1, 5, 1), 8, 1)); - - Assert.assertEquals( - "2 bin look-back: should merge until targetWeight when largestBinFirst is enabled", - list(list(36, 36, 36), list(128), list(36, 65), list(65)), - pack(list(36, 36, 36, 36, 65, 65, 128), 128, 2, true)); - - Assert.assertEquals( - "1 bin look-back: should merge until targetWeight when largestBinFirst is enabled", - list(list(64, 64), list(128), list(32, 32, 32, 32)), - pack(list(64, 64, 128, 32, 32, 32, 32), 128, 1, true)); + assertThat(pack(list(5, 1, 5, 1, 5, 1), 8, 1)) + .as("1 bin look-back: should merge ones with fives") + .isEqualTo(list(list(5, 1), list(5, 1), list(5, 1))); + + assertThat(pack(list(36, 36, 36, 36, 65, 65, 128), 128, 2, true)) + .as("2 bin look-back: should merge until targetWeight when largestBinFirst is enabled") + .isEqualTo(list(list(36, 36, 36), list(128), list(36, 65), list(65))); + + assertThat(pack(list(64, 64, 128, 32, 32, 32, 32), 128, 1, true)) + .as("1 bin look-back: should merge until targetWeight when largestBinFirst is enabled") + .isEqualTo(list(list(64, 64), list(128), list(32, 32, 32, 32))); } private List> pack(List items, long targetWeight) { diff --git a/core/src/test/java/org/apache/iceberg/util/TestEnvironmentUtil.java b/core/src/test/java/org/apache/iceberg/util/TestEnvironmentUtil.java index 7fdb93edde00..d9a573b80941 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestEnvironmentUtil.java +++ b/core/src/test/java/org/apache/iceberg/util/TestEnvironmentUtil.java @@ -18,10 +18,11 @@ */ package org.apache.iceberg.util; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.Map; import java.util.Optional; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Assumptions; import org.junit.jupiter.api.Test; @@ -33,10 +34,9 @@ public void testEnvironmentSubstitution() { envEntry.isPresent(), "Expecting at least one env. variable to be present"); Map resolvedProps = EnvironmentUtil.resolveAll(ImmutableMap.of("env-test", "env:" + envEntry.get().getKey())); - Assertions.assertEquals( - ImmutableMap.of("env-test", envEntry.get().getValue()), - resolvedProps, - "Should get the user from the environment"); + assertThat(resolvedProps) + .as("Should get the user from the environment") + .isEqualTo(ImmutableMap.of("env-test", envEntry.get().getValue())); } @Test @@ -46,10 +46,9 @@ public void testMultipleEnvironmentSubstitutions() { ImmutableMap.of("USER", "u", "VAR", "value"), ImmutableMap.of("user-test", "env:USER", "other", "left-alone", "var", "env:VAR")); - Assertions.assertEquals( - ImmutableMap.of("user-test", "u", "other", "left-alone", "var", "value"), - result, - "Should resolve all values starting with env:"); + assertThat(result) + .as("Should resolve all values starting with env:") + .isEqualTo(ImmutableMap.of("user-test", "u", "other", "left-alone", "var", "value")); } @Test @@ -57,7 +56,8 @@ public void testEnvironmentSubstitutionWithMissingVar() { Map result = EnvironmentUtil.resolveAll(ImmutableMap.of(), ImmutableMap.of("user-test", "env:USER")); - Assertions.assertEquals( - ImmutableMap.of(), result, "Should not contain values with missing environment variables"); + assertThat(result) + .as("Should not contain values with missing environment variables") + .isEqualTo(ImmutableMap.of()); } } diff --git a/core/src/test/java/org/apache/iceberg/util/TestInMemoryLockManager.java b/core/src/test/java/org/apache/iceberg/util/TestInMemoryLockManager.java index 48fc4a2cb3c1..7f7a39706517 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestInMemoryLockManager.java +++ b/core/src/test/java/org/apache/iceberg/util/TestInMemoryLockManager.java @@ -18,39 +18,37 @@ */ package org.apache.iceberg.util; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.List; import java.util.UUID; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import java.util.stream.IntStream; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.Timeout; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +@Timeout(value = 5) public class TestInMemoryLockManager { private LockManagers.InMemoryLockManager lockManager; private String lockEntityId; private String ownerId; - @Rule public Timeout timeout = new Timeout(5, TimeUnit.SECONDS); - - @Before + @BeforeEach public void before() { lockEntityId = UUID.randomUUID().toString(); ownerId = UUID.randomUUID().toString(); lockManager = new LockManagers.InMemoryLockManager(Maps.newHashMap()); } - @After + @AfterEach public void after() { lockManager.close(); } @@ -80,24 +78,26 @@ public void testAcquireOnceMultiProcesses() { } }) .collect(Collectors.toList()); - Assert.assertEquals( - "only 1 thread should have acquired the lock", 1, results.stream().filter(s -> s).count()); + assertThat(results.stream().filter(s -> s).count()) + .as("only 1 thread should have acquired the lock") + .isOne(); } @Test public void testReleaseAndAcquire() { - Assert.assertTrue(lockManager.acquire(lockEntityId, ownerId)); - Assert.assertTrue(lockManager.release(lockEntityId, ownerId)); - Assert.assertTrue( - "acquire after release should succeed", lockManager.acquire(lockEntityId, ownerId)); + assertThat(lockManager.acquire(lockEntityId, ownerId)).isTrue(); + assertThat(lockManager.release(lockEntityId, ownerId)).isTrue(); + assertThat(lockManager.acquire(lockEntityId, ownerId)) + .as("acquire after release should succeed") + .isTrue(); } @Test public void testReleaseWithWrongOwner() { - Assert.assertTrue(lockManager.acquire(lockEntityId, ownerId)); - Assert.assertFalse( - "should return false if ownerId is wrong", - lockManager.release(lockEntityId, UUID.randomUUID().toString())); + assertThat(lockManager.acquire(lockEntityId, ownerId)).isTrue(); + assertThat(lockManager.release(lockEntityId, UUID.randomUUID().toString())) + .as("should return false if ownerId is wrong") + .isFalse(); } @Test @@ -106,7 +106,7 @@ public void testAcquireSingleProcess() throws Exception { ImmutableMap.of( CatalogProperties.LOCK_ACQUIRE_INTERVAL_MS, "500", CatalogProperties.LOCK_ACQUIRE_TIMEOUT_MS, "2000")); - Assert.assertTrue(lockManager.acquire(lockEntityId, ownerId)); + assertThat(lockManager.acquire(lockEntityId, ownerId)).isTrue(); String oldOwner = ownerId; CompletableFuture.supplyAsync( @@ -116,14 +116,16 @@ public void testAcquireSingleProcess() throws Exception { } catch (InterruptedException e) { throw new RuntimeException(e); } - Assert.assertTrue(lockManager.release(lockEntityId, oldOwner)); + assertThat(lockManager.release(lockEntityId, oldOwner)).isTrue(); return null; }); ownerId = UUID.randomUUID().toString(); long start = System.currentTimeMillis(); - Assert.assertTrue(lockManager.acquire(lockEntityId, ownerId)); - Assert.assertTrue("should succeed after 200ms", System.currentTimeMillis() - start >= 200); + assertThat(lockManager.acquire(lockEntityId, ownerId)).isTrue(); + assertThat(System.currentTimeMillis() - start) + .as("should succeed after 200ms") + .isGreaterThanOrEqualTo(200); } @Test @@ -143,14 +145,17 @@ public void testAcquireMultiProcessAllSucceed() { } catch (InterruptedException e) { throw new RuntimeException(e); } - Assert.assertTrue(lockManager.release(lockEntityId, owner)); + assertThat(lockManager.release(lockEntityId, owner)).isTrue(); } return succeeded; }) .collect(Collectors.toList()); - Assert.assertEquals( - "all lock acquire should succeed sequentially", 3, results.stream().filter(s -> s).count()); - Assert.assertTrue("must take more than 3 seconds", System.currentTimeMillis() - start >= 3000); + assertThat(results.stream().filter(s -> s).count()) + .as("all lock acquire should succeed sequentially") + .isEqualTo(3); + assertThat(System.currentTimeMillis() - start) + .as("must take more than 3 seconds") + .isGreaterThanOrEqualTo(3000); } @Test @@ -166,7 +171,8 @@ public void testAcquireMultiProcessOnlyOneSucceed() { .parallel() .mapToObj(i -> lockManager.acquire(lockEntityId, ownerId)) .collect(Collectors.toList()); - Assert.assertEquals( - "only 1 thread should have acquired the lock", 1, results.stream().filter(s -> s).count()); + assertThat(results.stream().filter(s -> s).count()) + .as("only 1 thread should have acquired the lock") + .isOne(); } } diff --git a/core/src/test/java/org/apache/iceberg/util/TestJsonUtil.java b/core/src/test/java/org/apache/iceberg/util/TestJsonUtil.java index 8a769fccb9e2..6b558599ac46 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestJsonUtil.java +++ b/core/src/test/java/org/apache/iceberg/util/TestJsonUtil.java @@ -19,12 +19,14 @@ package org.apache.iceberg.util; import com.fasterxml.jackson.core.JsonProcessingException; +import java.nio.ByteBuffer; import java.util.Arrays; import java.util.List; import java.util.Map; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.io.BaseEncoding; import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestJsonUtil { @@ -167,6 +169,26 @@ public void getStringOrNull() throws JsonProcessingException { .hasMessage("Cannot parse to a string value: x: 23"); } + @Test + public void getByteBufferOrNull() throws JsonProcessingException { + Assertions.assertThat(JsonUtil.getByteBufferOrNull("x", JsonUtil.mapper().readTree("{}"))) + .isNull(); + Assertions.assertThat( + JsonUtil.getByteBufferOrNull("x", JsonUtil.mapper().readTree("{\"x\": null}"))) + .isNull(); + + byte[] bytes = new byte[] {1, 2, 3, 4}; + String base16Str = BaseEncoding.base16().encode(bytes); + String json = String.format("{\"x\": \"%s\"}", base16Str); + ByteBuffer byteBuffer = JsonUtil.getByteBufferOrNull("x", JsonUtil.mapper().readTree(json)); + Assertions.assertThat(byteBuffer.array()).isEqualTo(bytes); + + Assertions.assertThatThrownBy( + () -> JsonUtil.getByteBufferOrNull("x", JsonUtil.mapper().readTree("{\"x\": 23}"))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse byte buffer from non-text value: x: 23"); + } + @Test public void getBool() throws JsonProcessingException { Assertions.assertThatThrownBy(() -> JsonUtil.getBool("x", JsonUtil.mapper().readTree("{}"))) @@ -194,6 +216,28 @@ public void getBool() throws JsonProcessingException { .isFalse(); } + @Test + public void getIntArrayOrNull() throws JsonProcessingException { + Assertions.assertThat(JsonUtil.getIntArrayOrNull("items", JsonUtil.mapper().readTree("{}"))) + .isNull(); + + Assertions.assertThat( + JsonUtil.getIntArrayOrNull("items", JsonUtil.mapper().readTree("{\"items\": null}"))) + .isNull(); + + Assertions.assertThatThrownBy( + () -> + JsonUtil.getIntArrayOrNull( + "items", JsonUtil.mapper().readTree("{\"items\": [13, \"23\"]}"))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse integer from non-int value in items: \"23\""); + + Assertions.assertThat( + JsonUtil.getIntArrayOrNull( + "items", JsonUtil.mapper().readTree("{\"items\": [23, 45]}"))) + .isEqualTo(new int[] {23, 45}); + } + @Test public void getIntegerList() throws JsonProcessingException { Assertions.assertThatThrownBy( @@ -204,7 +248,7 @@ public void getIntegerList() throws JsonProcessingException { Assertions.assertThatThrownBy( () -> JsonUtil.getIntegerList("items", JsonUtil.mapper().readTree("{\"items\": null}"))) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot parse from non-array value: items: null"); + .hasMessage("Cannot parse JSON array from non-array value: items: null"); Assertions.assertThatThrownBy( () -> @@ -240,7 +284,7 @@ public void getIntegerSet() throws JsonProcessingException { Assertions.assertThatThrownBy( () -> JsonUtil.getIntegerSet("items", JsonUtil.mapper().readTree("{\"items\": null}"))) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot parse from non-array value: items: null"); + .hasMessage("Cannot parse JSON array from non-array value: items: null"); Assertions.assertThatThrownBy( () -> @@ -286,7 +330,7 @@ public void getLongList() throws JsonProcessingException { Assertions.assertThatThrownBy( () -> JsonUtil.getLongList("items", JsonUtil.mapper().readTree("{\"items\": null}"))) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot parse from non-array value: items: null"); + .hasMessage("Cannot parse JSON array from non-array value: items: null"); Assertions.assertThatThrownBy( () -> @@ -312,6 +356,28 @@ public void getLongList() throws JsonProcessingException { .isEqualTo(items); } + @Test + public void getLongListOrNull() throws JsonProcessingException { + Assertions.assertThat(JsonUtil.getLongListOrNull("items", JsonUtil.mapper().readTree("{}"))) + .isNull(); + + Assertions.assertThat( + JsonUtil.getLongListOrNull("items", JsonUtil.mapper().readTree("{\"items\": null}"))) + .isNull(); + + Assertions.assertThatThrownBy( + () -> + JsonUtil.getLongListOrNull( + "items", JsonUtil.mapper().readTree("{\"items\": [13, \"23\"]}"))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse long from non-long value in items: \"23\""); + + Assertions.assertThat( + JsonUtil.getLongListOrNull( + "items", JsonUtil.mapper().readTree("{\"items\": [23, 45]}"))) + .containsExactlyElementsOf(Arrays.asList(23L, 45L)); + } + @Test public void getLongSet() throws JsonProcessingException { Assertions.assertThatThrownBy( @@ -322,7 +388,7 @@ public void getLongSet() throws JsonProcessingException { Assertions.assertThatThrownBy( () -> JsonUtil.getLongSet("items", JsonUtil.mapper().readTree("{\"items\": null}"))) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot parse from non-array value: items: null"); + .hasMessage("Cannot parse JSON array from non-array value: items: null"); Assertions.assertThatThrownBy( () -> @@ -367,7 +433,7 @@ public void getStringList() throws JsonProcessingException { Assertions.assertThatThrownBy( () -> JsonUtil.getStringList("items", JsonUtil.mapper().readTree("{\"items\": null}"))) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot parse from non-array value: items: null"); + .hasMessage("Cannot parse JSON array from non-array value: items: null"); Assertions.assertThatThrownBy( () -> @@ -426,7 +492,7 @@ public void getStringSet() throws JsonProcessingException { Assertions.assertThatThrownBy( () -> JsonUtil.getStringSet("items", JsonUtil.mapper().readTree("{\"items\": null}"))) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot parse from non-array value: items: null"); + .hasMessage("Cannot parse JSON array from non-array value: items: null"); Assertions.assertThatThrownBy( () -> @@ -451,7 +517,7 @@ public void getStringMap() throws JsonProcessingException { Assertions.assertThatThrownBy( () -> JsonUtil.getStringMap("items", JsonUtil.mapper().readTree("{\"items\": null}"))) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot parse from non-object value: items: null"); + .hasMessage("Cannot parse string map from non-object value: items: null"); Assertions.assertThatThrownBy( () -> diff --git a/core/src/test/java/org/apache/iceberg/util/TestLocationUtil.java b/core/src/test/java/org/apache/iceberg/util/TestLocationUtil.java index d0b2b98243eb..f557352299b7 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestLocationUtil.java +++ b/core/src/test/java/org/apache/iceberg/util/TestLocationUtil.java @@ -18,35 +18,34 @@ */ package org.apache.iceberg.util; +import static org.assertj.core.api.Assertions.assertThat; + import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestLocationUtil { @Test public void testStripTrailingSlash() { String pathWithoutTrailingSlash = "s3://bucket/db/tbl"; - Assert.assertEquals( - "Should have no trailing slashes", - pathWithoutTrailingSlash, - LocationUtil.stripTrailingSlash(pathWithoutTrailingSlash)); + assertThat(LocationUtil.stripTrailingSlash(pathWithoutTrailingSlash)) + .as("Should have no trailing slashes") + .isEqualTo(pathWithoutTrailingSlash); String pathWithSingleTrailingSlash = pathWithoutTrailingSlash + "/"; - Assert.assertEquals( - "Should have no trailing slashes", - pathWithoutTrailingSlash, - LocationUtil.stripTrailingSlash(pathWithSingleTrailingSlash)); + assertThat(LocationUtil.stripTrailingSlash(pathWithSingleTrailingSlash)) + .as("Should have no trailing slashes") + .isEqualTo(pathWithoutTrailingSlash); String pathWithMultipleTrailingSlash = pathWithoutTrailingSlash + "////"; - Assert.assertEquals( - "Should have no trailing slashes", - pathWithoutTrailingSlash, - LocationUtil.stripTrailingSlash(pathWithMultipleTrailingSlash)); + assertThat(LocationUtil.stripTrailingSlash(pathWithMultipleTrailingSlash)) + .as("Should have no trailing slashes") + .isEqualTo(pathWithoutTrailingSlash); String pathWithOnlySlash = "////"; - Assert.assertEquals( - "Should have no trailing slashes", "", LocationUtil.stripTrailingSlash(pathWithOnlySlash)); + assertThat(LocationUtil.stripTrailingSlash(pathWithOnlySlash)) + .as("Should have no trailing slashes") + .isEmpty(); } @Test diff --git a/core/src/test/java/org/apache/iceberg/util/TestLockManagers.java b/core/src/test/java/org/apache/iceberg/util/TestLockManagers.java index a1a5c0cb9166..5dd23c3a459d 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestLockManagers.java +++ b/core/src/test/java/org/apache/iceberg/util/TestLockManagers.java @@ -23,7 +23,7 @@ import org.apache.iceberg.LockManager; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestLockManagers { diff --git a/core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java b/core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java index 960f7d454e30..68685614d3da 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java +++ b/core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java @@ -33,7 +33,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.awaitility.Awaitility; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestParallelIterable { @Test diff --git a/core/src/test/java/org/apache/iceberg/util/TestReachableFileUtil.java b/core/src/test/java/org/apache/iceberg/util/TestReachableFileUtil.java index 6f6b9ed40fde..b8d686246b0f 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestReachableFileUtil.java +++ b/core/src/test/java/org/apache/iceberg/util/TestReachableFileUtil.java @@ -19,6 +19,7 @@ package org.apache.iceberg.util; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -42,11 +43,9 @@ import org.apache.iceberg.hadoop.Util; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestReachableFileUtil { private static final HadoopTables TABLES = new HadoopTables(new Configuration()); @@ -68,14 +67,11 @@ public class TestReachableFileUtil { .withFileSizeInBytes(10) .withRecordCount(1) .build(); - - @Rule public TemporaryFolder temp = new TemporaryFolder(); - + @TempDir private File tableDir; private Table table; - @Before + @BeforeEach public void setupTableLocation() throws Exception { - File tableDir = temp.newFolder(); String tableLocation = tableDir.toURI().toString(); this.table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation); } @@ -87,7 +83,7 @@ public void testManifestListLocations() { table.newAppend().appendFile(FILE_B).commit(); List manifestListPaths = ReachableFileUtil.manifestListLocations(table); - Assert.assertEquals(manifestListPaths.size(), 2); + assertThat(manifestListPaths).hasSize(2); } @Test @@ -99,10 +95,10 @@ public void testMetadataFileLocations() { table.newAppend().appendFile(FILE_B).commit(); Set metadataFileLocations = ReachableFileUtil.metadataFileLocations(table, true); - Assert.assertEquals(metadataFileLocations.size(), 4); + assertThat(metadataFileLocations).hasSize(4); metadataFileLocations = ReachableFileUtil.metadataFileLocations(table, false); - Assert.assertEquals(metadataFileLocations.size(), 2); + assertThat(metadataFileLocations).hasSize(2); } @Test @@ -119,7 +115,7 @@ public void testMetadataFileLocationsWithMissingFiles() { table.io().deleteFile(location); Set metadataFileLocations = ReachableFileUtil.metadataFileLocations(table, true); - Assert.assertEquals(metadataFileLocations.size(), 2); + assertThat(metadataFileLocations).hasSize(2); } @Test @@ -133,7 +129,7 @@ public void testVersionHintWithStaticTables() { String reportedVersionHintLocation = ReachableFileUtil.versionHintLocation(staticTable); String expectedVersionHintLocation = ops.metadataFileLocation(Util.VERSION_HINT_FILENAME); - Assert.assertEquals(expectedVersionHintLocation, reportedVersionHintLocation); + assertThat(reportedVersionHintLocation).isEqualTo(expectedVersionHintLocation); } @Test @@ -142,6 +138,6 @@ public void testVersionHintWithBucketNameAsLocation() { when(mockTable.location()).thenReturn("s3://bucket1"); String reportedVersionHintLocation = ReachableFileUtil.versionHintLocation(mockTable); String expectedVersionHintLocation = "s3://bucket1/metadata/" + Util.VERSION_HINT_FILENAME; - Assert.assertEquals(expectedVersionHintLocation, reportedVersionHintLocation); + assertThat(reportedVersionHintLocation).isEqualTo(expectedVersionHintLocation); } } diff --git a/core/src/test/java/org/apache/iceberg/util/TestSnapshotUtil.java b/core/src/test/java/org/apache/iceberg/util/TestSnapshotUtil.java index 05c74c301336..576df47e7b2f 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestSnapshotUtil.java +++ b/core/src/test/java/org/apache/iceberg/util/TestSnapshotUtil.java @@ -19,8 +19,10 @@ package org.apache.iceberg.util; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; +import java.util.Iterator; import java.util.List; import java.util.stream.StreamSupport; import org.apache.iceberg.DataFile; @@ -28,19 +30,18 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TestHelpers; import org.apache.iceberg.TestTables; import org.apache.iceberg.types.Types; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestSnapshotUtil { - @Rule public TemporaryFolder temp = new TemporaryFolder(); - + @TempDir private File tableDir; // Schema passed to create tables public static final Schema SCHEMA = new Schema( @@ -49,7 +50,6 @@ public class TestSnapshotUtil { // Partition spec used to create tables protected static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).build(); - protected File tableDir = null; protected File metadataDir = null; public TestTables.TestTable table = null; @@ -60,95 +60,124 @@ public class TestSnapshotUtil { .withRecordCount(1) .build(); - private long snapshotAId; + private long snapshotBaseTimestamp; + private long snapshotBaseId; + private long snapshotBranchId; + private long snapshotMain1Id; + private long snapshotMain2Id; + private long snapshotFork0Id; + private long snapshotFork1Id; + private long snapshotFork2Id; + + private Snapshot appendFileTo(String branch) { + table.newFastAppend().appendFile(FILE_A).toBranch(branch).commit(); + return table.snapshot(branch); + } - private long snapshotATimestamp; - private long snapshotBId; - private long snapshotCId; - private long snapshotDId; + private Snapshot appendFileToMain() { + return appendFileTo(SnapshotRef.MAIN_BRANCH); + } - @Before + @BeforeEach public void before() throws Exception { - this.tableDir = temp.newFolder(); tableDir.delete(); // created by table create this.metadataDir = new File(tableDir, "metadata"); this.table = TestTables.create(tableDir, "test", SCHEMA, SPEC, 2); - table.newFastAppend().appendFile(FILE_A).commit(); - Snapshot snapshotA = table.currentSnapshot(); - this.snapshotAId = snapshotA.snapshotId(); - this.snapshotATimestamp = snapshotA.timestampMillis(); - - TestHelpers.waitUntilAfter(snapshotATimestamp); - - table.newFastAppend().appendFile(FILE_A).commit(); - this.snapshotBId = table.currentSnapshot().snapshotId(); + Snapshot snapshotBase = appendFileToMain(); + this.snapshotBaseId = snapshotBase.snapshotId(); + this.snapshotBaseTimestamp = snapshotBase.timestampMillis(); + TestHelpers.waitUntilAfter(snapshotBaseTimestamp); - table.newFastAppend().appendFile(FILE_A).commit(); - this.snapshotDId = table.currentSnapshot().snapshotId(); + this.snapshotMain1Id = appendFileToMain().snapshotId(); + this.snapshotMain2Id = appendFileToMain().snapshotId(); String branchName = "b1"; - table.manageSnapshots().createBranch(branchName, snapshotAId).commit(); - table.newFastAppend().appendFile(FILE_A).toBranch(branchName).commit(); - this.snapshotCId = table.snapshot(branchName).snapshotId(); + table.manageSnapshots().createBranch(branchName, snapshotBaseId).commit(); + this.snapshotBranchId = appendFileTo(branchName).snapshotId(); + + // Create a branch that leads back to an expired snapshot + String forkBranch = "fork"; + table.manageSnapshots().createBranch(forkBranch, snapshotBaseId).commit(); + this.snapshotFork0Id = appendFileTo(forkBranch).snapshotId(); + this.snapshotFork1Id = appendFileTo(forkBranch).snapshotId(); + this.snapshotFork2Id = appendFileTo(forkBranch).snapshotId(); + table.expireSnapshots().expireSnapshotId(snapshotFork0Id).commit(); } - @After + @AfterEach public void cleanupTables() { TestTables.clearTables(); } @Test public void isParentAncestorOf() { - Assert.assertTrue(SnapshotUtil.isParentAncestorOf(table, snapshotBId, snapshotAId)); - Assert.assertFalse(SnapshotUtil.isParentAncestorOf(table, snapshotCId, snapshotBId)); + assertThat(SnapshotUtil.isParentAncestorOf(table, snapshotMain1Id, snapshotBaseId)).isTrue(); + assertThat(SnapshotUtil.isParentAncestorOf(table, snapshotBranchId, snapshotMain1Id)).isFalse(); + assertThat(SnapshotUtil.isParentAncestorOf(table, snapshotFork2Id, snapshotFork0Id)).isTrue(); } @Test public void isAncestorOf() { - Assert.assertTrue(SnapshotUtil.isAncestorOf(table, snapshotBId, snapshotAId)); - Assert.assertFalse(SnapshotUtil.isAncestorOf(table, snapshotCId, snapshotBId)); + assertThat(SnapshotUtil.isAncestorOf(table, snapshotMain1Id, snapshotBaseId)).isTrue(); + assertThat(SnapshotUtil.isAncestorOf(table, snapshotBranchId, snapshotMain1Id)).isFalse(); + assertThat(SnapshotUtil.isAncestorOf(table, snapshotFork2Id, snapshotFork0Id)).isFalse(); - Assert.assertTrue(SnapshotUtil.isAncestorOf(table, snapshotBId)); - Assert.assertFalse(SnapshotUtil.isAncestorOf(table, snapshotCId)); + assertThat(SnapshotUtil.isAncestorOf(table, snapshotMain1Id)).isTrue(); + assertThat(SnapshotUtil.isAncestorOf(table, snapshotBranchId)).isFalse(); } @Test public void currentAncestors() { Iterable snapshots = SnapshotUtil.currentAncestors(table); - expectedSnapshots(new long[] {snapshotDId, snapshotBId, snapshotAId}, snapshots); + expectedSnapshots(new long[] {snapshotMain2Id, snapshotMain1Id, snapshotBaseId}, snapshots); List snapshotList = SnapshotUtil.currentAncestorIds(table); - Assert.assertArrayEquals( - new Long[] {snapshotDId, snapshotBId, snapshotAId}, snapshotList.toArray(new Long[0])); + assertThat(snapshotList.toArray(new Long[0])) + .isEqualTo(new Long[] {snapshotMain2Id, snapshotMain1Id, snapshotBaseId}); } @Test public void oldestAncestor() { Snapshot snapshot = SnapshotUtil.oldestAncestor(table); - Assert.assertEquals(snapshotAId, snapshot.snapshotId()); + assertThat(snapshot.snapshotId()).isEqualTo(snapshotBaseId); - snapshot = SnapshotUtil.oldestAncestorOf(table, snapshotDId); - Assert.assertEquals(snapshotAId, snapshot.snapshotId()); + snapshot = SnapshotUtil.oldestAncestorOf(table, snapshotMain2Id); + assertThat(snapshot.snapshotId()).isEqualTo(snapshotBaseId); - snapshot = SnapshotUtil.oldestAncestorAfter(table, snapshotATimestamp + 1); - Assert.assertEquals(snapshotBId, snapshot.snapshotId()); + snapshot = SnapshotUtil.oldestAncestorAfter(table, snapshotBaseTimestamp + 1); + assertThat(snapshot.snapshotId()).isEqualTo(snapshotMain1Id); } @Test public void snapshotsBetween() { List snapshotIdsBetween = - SnapshotUtil.snapshotIdsBetween(table, snapshotAId, snapshotDId); - Assert.assertArrayEquals( - new Long[] {snapshotDId, snapshotBId}, snapshotIdsBetween.toArray(new Long[0])); + SnapshotUtil.snapshotIdsBetween(table, snapshotBaseId, snapshotMain2Id); + assertThat(snapshotIdsBetween.toArray(new Long[0])) + .isEqualTo(new Long[] {snapshotMain2Id, snapshotMain1Id}); Iterable ancestorsBetween = - SnapshotUtil.ancestorsBetween(table, snapshotDId, snapshotBId); - expectedSnapshots(new long[] {snapshotDId}, ancestorsBetween); + SnapshotUtil.ancestorsBetween(table, snapshotMain2Id, snapshotMain1Id); + expectedSnapshots(new long[] {snapshotMain2Id}, ancestorsBetween); + + ancestorsBetween = SnapshotUtil.ancestorsBetween(table, snapshotMain2Id, snapshotBranchId); + expectedSnapshots( + new long[] {snapshotMain2Id, snapshotMain1Id, snapshotBaseId}, ancestorsBetween); + } + + @Test + public void ancestorsOf() { + Iterable snapshots = SnapshotUtil.ancestorsOf(snapshotFork2Id, table::snapshot); + expectedSnapshots(new long[] {snapshotFork2Id, snapshotFork1Id}, snapshots); + + Iterator snapshotIter = snapshots.iterator(); + while (snapshotIter.hasNext()) { + snapshotIter.next(); + } - ancestorsBetween = SnapshotUtil.ancestorsBetween(table, snapshotDId, snapshotCId); - expectedSnapshots(new long[] {snapshotDId, snapshotBId, snapshotAId}, ancestorsBetween); + // Once snapshot iterator has been exhausted, call hasNext again to make sure it is stable. + Assertions.assertThat(snapshotIter).isExhausted(); } private void expectedSnapshots(long[] snapshotIdExpected, Iterable snapshotsActual) { @@ -156,6 +185,6 @@ private void expectedSnapshots(long[] snapshotIdExpected, Iterable sna StreamSupport.stream(snapshotsActual.spliterator(), false) .mapToLong(Snapshot::snapshotId) .toArray(); - Assert.assertArrayEquals(snapshotIdExpected, actualSnapshots); + assertThat(actualSnapshots).isEqualTo(snapshotIdExpected); } } diff --git a/core/src/test/java/org/apache/iceberg/util/TestSortOrderUtil.java b/core/src/test/java/org/apache/iceberg/util/TestSortOrderUtil.java index 516279bc7f7b..02c81de93222 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestSortOrderUtil.java +++ b/core/src/test/java/org/apache/iceberg/util/TestSortOrderUtil.java @@ -21,21 +21,18 @@ import static org.apache.iceberg.NullOrder.NULLS_LAST; import static org.apache.iceberg.SortDirection.ASC; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; -import java.io.IOException; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.SortOrder; import org.apache.iceberg.TestTables; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.types.Types; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestSortOrderUtil { @@ -47,15 +44,9 @@ public class TestSortOrderUtil { required(12, "ts", Types.TimestampType.withZone()), required(13, "category", Types.StringType.get())); - @Rule public TemporaryFolder temp = new TemporaryFolder(); - private File tableDir = null; + @TempDir private File tableDir; - @Before - public void setupTableDir() throws IOException { - this.tableDir = temp.newFolder(); - } - - @After + @AfterEach public void cleanupTables() { TestTables.clearTables(); } @@ -69,12 +60,13 @@ public void testEmptySpecsV1() { // pass PartitionSpec.unpartitioned() on purpose as it has an empty schema SortOrder actualOrder = SortOrderUtil.buildSortOrder(table.schema(), spec, table.sortOrder()); - Assert.assertEquals("Order ID must be fresh", 1, actualOrder.orderId()); - Assert.assertEquals("Order must have 1 field", 1, actualOrder.fields().size()); - Assert.assertEquals("Field id must be fresh", 1, actualOrder.fields().get(0).sourceId()); - Assert.assertEquals("Direction must match", ASC, actualOrder.fields().get(0).direction()); - Assert.assertEquals( - "Null order must match", NULLS_LAST, actualOrder.fields().get(0).nullOrder()); + assertThat(actualOrder.orderId()).as("Order ID must be fresh").isOne(); + assertThat(actualOrder.fields()).as("Order must have 1 field").hasSize(1); + assertThat(actualOrder.fields().get(0).sourceId()).as("Field id must be fresh").isOne(); + assertThat(actualOrder.fields().get(0).direction()).as("Direction must match").isEqualTo(ASC); + assertThat(actualOrder.fields().get(0).nullOrder()) + .as("Null order must match") + .isEqualTo(NULLS_LAST); } @Test @@ -86,12 +78,13 @@ public void testEmptySpecsV2() { // pass PartitionSpec.unpartitioned() on purpose as it has an empty schema SortOrder actualOrder = SortOrderUtil.buildSortOrder(table.schema(), spec, table.sortOrder()); - Assert.assertEquals("Order ID must be fresh", 1, actualOrder.orderId()); - Assert.assertEquals("Order must have 1 field", 1, actualOrder.fields().size()); - Assert.assertEquals("Field id must be fresh", 1, actualOrder.fields().get(0).sourceId()); - Assert.assertEquals("Direction must match", ASC, actualOrder.fields().get(0).direction()); - Assert.assertEquals( - "Null order must match", NULLS_LAST, actualOrder.fields().get(0).nullOrder()); + assertThat(actualOrder.orderId()).as("Order ID must be fresh").isOne(); + assertThat(actualOrder.fields()).as("Order must have 1 field").hasSize(1); + assertThat(actualOrder.fields().get(0).sourceId()).as("Field id must be fresh").isOne(); + assertThat(actualOrder.fields().get(0).direction()).as("Direction must match").isEqualTo(ASC); + assertThat(actualOrder.fields().get(0).nullOrder()) + .as("Null order must match") + .isEqualTo(NULLS_LAST); } @Test @@ -107,10 +100,9 @@ public void testSortOrderClusteringNoPartitionFields() { .desc("id") .build(); - Assert.assertEquals( - "Should add spec fields as prefix", - expected, - SortOrderUtil.buildSortOrder(SCHEMA, spec, order)); + assertThat(SortOrderUtil.buildSortOrder(SCHEMA, spec, order)) + .as("Should add spec fields as prefix") + .isEqualTo(expected); } @Test @@ -124,10 +116,9 @@ public void testSortOrderClusteringAllPartitionFields() { .desc("id") .build(); - Assert.assertEquals( - "Should leave the order unchanged", - order, - SortOrderUtil.buildSortOrder(SCHEMA, spec, order)); + assertThat(SortOrderUtil.buildSortOrder(SCHEMA, spec, order)) + .as("Should leave the order unchanged") + .isEqualTo(order); } @Test @@ -141,10 +132,9 @@ public void testSortOrderClusteringAllPartitionFieldsReordered() { .desc("id") .build(); - Assert.assertEquals( - "Should leave the order unchanged", - order, - SortOrderUtil.buildSortOrder(SCHEMA, spec, order)); + assertThat(SortOrderUtil.buildSortOrder(SCHEMA, spec, order)) + .as("Should leave the order unchanged") + .isEqualTo(order); } @Test @@ -161,10 +151,9 @@ public void testSortOrderClusteringSomePartitionFields() { .desc("id") .build(); - Assert.assertEquals( - "Should add spec fields as prefix", - expected, - SortOrderUtil.buildSortOrder(SCHEMA, spec, order)); + assertThat(SortOrderUtil.buildSortOrder(SCHEMA, spec, order)) + .as("Should add spec fields as prefix") + .isEqualTo(expected); } @Test @@ -181,10 +170,9 @@ public void testSortOrderClusteringSatisfiedPartitionLast() { SortOrder expected = SortOrder.builderFor(SCHEMA).withOrderId(1).asc("category").asc("ts").desc("id").build(); - Assert.assertEquals( - "Should add spec fields as prefix", - expected, - SortOrderUtil.buildSortOrder(SCHEMA, spec, order)); + assertThat(SortOrderUtil.buildSortOrder(SCHEMA, spec, order)) + .as("Should add spec fields as prefix") + .isEqualTo(expected); } @Test @@ -207,10 +195,9 @@ public void testSortOrderClusteringSatisfiedPartitionFirst() { .desc("id") .build(); - Assert.assertEquals( - "Should add spec fields as prefix", - expected, - SortOrderUtil.buildSortOrder(SCHEMA, spec, order)); + assertThat(SortOrderUtil.buildSortOrder(SCHEMA, spec, order)) + .as("Should add spec fields as prefix") + .isEqualTo(expected); } @Test @@ -234,10 +221,9 @@ public void testSortOrderClusteringSatisfiedPartitionFields() { .desc("id") .build(); - Assert.assertEquals( - "Should add spec fields as prefix", - expected, - SortOrderUtil.buildSortOrder(SCHEMA, spec, order)); + assertThat(SortOrderUtil.buildSortOrder(SCHEMA, spec, order)) + .as("Should add spec fields as prefix") + .isEqualTo(expected); } @Test @@ -267,10 +253,9 @@ public void testSortOrderClusteringWithRedundantPartitionFields() { .desc("id") .build(); - Assert.assertEquals( - "Should add spec fields as prefix", - expected, - SortOrderUtil.buildSortOrder(table.schema(), updatedSpec, order)); + assertThat(SortOrderUtil.buildSortOrder(table.schema(), updatedSpec, order)) + .as("Should add spec fields as prefix") + .isEqualTo(expected); } @Test @@ -298,9 +283,8 @@ public void testSortOrderClusteringWithRedundantPartitionFieldsMissing() { .desc("id") .build(); - Assert.assertEquals( - "Should add spec fields as prefix", - expected, - SortOrderUtil.buildSortOrder(table.schema(), updatedSpec, order)); + assertThat(SortOrderUtil.buildSortOrder(table.schema(), updatedSpec, order)) + .as("Should add spec fields as prefix") + .isEqualTo(expected); } } diff --git a/core/src/test/java/org/apache/iceberg/util/TestStructLikeMap.java b/core/src/test/java/org/apache/iceberg/util/TestStructLikeMap.java index 184fa18fc265..db4176ce231e 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestStructLikeMap.java +++ b/core/src/test/java/org/apache/iceberg/util/TestStructLikeMap.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.util; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.Collection; import java.util.Map; import java.util.Set; @@ -25,11 +27,9 @@ import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestStructLikeMap { private static final Types.StructType STRUCT_TYPE = @@ -43,28 +43,22 @@ public void testSingleRecord() { Record record1 = gRecord.copy(ImmutableMap.of("id", 1, "data", "aaa")); Map map = StructLikeMap.create(STRUCT_TYPE); - Assert.assertEquals(0, map.size()); + assertThat(map).isEmpty(); map.put(record1, "1-aaa"); - Assert.assertEquals(1, map.size()); - Assert.assertFalse(map.isEmpty()); - Assert.assertTrue(map.containsKey(record1)); - Assert.assertTrue(map.containsValue("1-aaa")); - Assert.assertEquals("1-aaa", map.get(record1)); + assertThat(map).hasSize(1).containsEntry(record1, "1-aaa"); Set keySet = map.keySet(); - Assert.assertEquals(1, keySet.size()); - Assert.assertTrue(keySet.contains(record1)); + assertThat(keySet).hasSize(1).contains(record1); Collection values = map.values(); - Assert.assertEquals(1, values.size()); - Assert.assertEquals("1-aaa", values.toArray(new String[0])[0]); + assertThat(values).hasSize(1).first().isEqualTo("1-aaa"); Set> entrySet = map.entrySet(); - Assert.assertEquals(1, entrySet.size()); + assertThat(entrySet).hasSize(1); for (Map.Entry entry : entrySet) { - Assert.assertEquals(record1, entry.getKey()); - Assert.assertEquals("1-aaa", entry.getValue()); + assertThat(entry.getKey()).isEqualTo(record1); + assertThat(entry.getValue()).isEqualTo("1-aaa"); break; } } @@ -79,38 +73,31 @@ public void testMultipleRecord() { record3.setField("data", null); Map map = StructLikeMap.create(STRUCT_TYPE); - Assert.assertEquals(0, map.size()); + assertThat(map).isEmpty(); map.putAll(ImmutableMap.of(record1, "1-aaa", record2, "2-bbb", record3, "3-null")); - Assert.assertEquals(3, map.size()); - Assert.assertTrue(map.containsKey(record1)); - Assert.assertTrue(map.containsKey(record2)); - Assert.assertTrue(map.containsKey(record3)); - Assert.assertTrue(map.containsValue("1-aaa")); - Assert.assertTrue(map.containsValue("2-bbb")); - Assert.assertTrue(map.containsValue("3-null")); - Assert.assertEquals("1-aaa", map.get(record1)); - Assert.assertEquals("2-bbb", map.get(record2)); - Assert.assertEquals("3-null", map.get(record3)); + assertThat(map) + .hasSize(3) + .containsEntry(record1, "1-aaa") + .containsEntry(record2, "2-bbb") + .containsEntry(record3, "3-null"); Set keySet = map.keySet(); - Assert.assertEquals(3, keySet.size()); - Assert.assertEquals(ImmutableSet.of(record1, record2, record3), keySet); + assertThat(keySet).hasSize(3).containsOnly(record1, record2, record3); Collection values = map.values(); - Assert.assertEquals(3, values.size()); - Assert.assertEquals(ImmutableSet.of("1-aaa", "2-bbb", "3-null"), Sets.newHashSet(values)); + assertThat(values).hasSize(3).containsExactlyInAnyOrder("1-aaa", "2-bbb", "3-null"); Set> entrySet = map.entrySet(); - Assert.assertEquals(3, entrySet.size()); + assertThat(entrySet).hasSize(3); Set structLikeSet = Sets.newHashSet(); Set valueSet = Sets.newHashSet(); for (Map.Entry entry : entrySet) { structLikeSet.add(entry.getKey()); valueSet.add(entry.getValue()); } - Assert.assertEquals(ImmutableSet.of(record1, record2, record3), structLikeSet); - Assert.assertEquals(ImmutableSet.of("1-aaa", "2-bbb", "3-null"), valueSet); + assertThat(structLikeSet).containsExactlyInAnyOrder(record1, record2, record3); + assertThat(valueSet).containsExactlyInAnyOrder("1-aaa", "2-bbb", "3-null"); } @Test @@ -120,29 +107,27 @@ public void testRemove() { Map map = StructLikeMap.create(STRUCT_TYPE); map.put(record, "1-aaa"); - Assert.assertEquals(1, map.size()); - Assert.assertEquals("1-aaa", map.get(record)); - Assert.assertEquals("1-aaa", map.remove(record)); - Assert.assertEquals(0, map.size()); + assertThat(map).hasSize(1).containsEntry(record, "1-aaa"); + assertThat(map.remove(record)).isEqualTo("1-aaa"); + assertThat(map).isEmpty(); map.put(record, "1-aaa"); - Assert.assertEquals("1-aaa", map.get(record)); + assertThat(map).containsEntry(record, "1-aaa"); } @Test public void testNullKeys() { Map map = StructLikeMap.create(STRUCT_TYPE); - Assert.assertFalse(map.containsKey(null)); + assertThat(map).doesNotContainKey(null); map.put(null, "aaa"); - Assert.assertTrue(map.containsKey(null)); - Assert.assertEquals("aaa", map.get(null)); + assertThat(map).containsEntry(null, "aaa"); String replacedValue = map.put(null, "bbb"); - Assert.assertEquals("aaa", replacedValue); + assertThat(replacedValue).isEqualTo("aaa"); String removedValue = map.remove(null); - Assert.assertEquals("bbb", removedValue); + assertThat(removedValue).isEqualTo("bbb"); } @Test @@ -155,13 +140,11 @@ public void testKeysWithNulls() { map.put(record1, "aaa"); map.put(record2, "bbb"); - Assert.assertEquals("aaa", map.get(record1)); - Assert.assertEquals("bbb", map.get(record2)); + assertThat(map).containsEntry(record1, "aaa").containsEntry(record2, "bbb"); Record record3 = record1.copy(); - Assert.assertTrue(map.containsKey(record3)); - Assert.assertEquals("aaa", map.get(record3)); + assertThat(map).containsEntry(record3, "aaa"); - Assert.assertEquals("aaa", map.remove(record3)); + assertThat(map.remove(record3)).isEqualTo("aaa"); } } diff --git a/core/src/test/java/org/apache/iceberg/util/TestStructLikeSet.java b/core/src/test/java/org/apache/iceberg/util/TestStructLikeSet.java index 77726ee6359d..9c21e3f1cc48 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestStructLikeSet.java +++ b/core/src/test/java/org/apache/iceberg/util/TestStructLikeSet.java @@ -18,13 +18,14 @@ */ package org.apache.iceberg.util; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.Set; import org.apache.iceberg.StructLike; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestStructLikeSet { private static final Types.StructType STRUCT_TYPE = @@ -35,17 +36,17 @@ public class TestStructLikeSet { @Test public void testNullElements() { Set set = StructLikeSet.create(STRUCT_TYPE); - Assert.assertFalse(set.contains(null)); + assertThat(set).doesNotContain((StructLike) null); set.add(null); - Assert.assertTrue(set.contains(null)); + assertThat(set).contains((StructLike) null); boolean added = set.add(null); - Assert.assertFalse(added); + assertThat(added).isFalse(); boolean removed = set.remove(null); - Assert.assertTrue(removed); - Assert.assertTrue(set.isEmpty()); + assertThat(removed).isTrue(); + assertThat(set).isEmpty(); } @Test @@ -58,13 +59,12 @@ public void testElementsWithNulls() { set.add(record1); set.add(record2); - Assert.assertTrue(set.contains(record1)); - Assert.assertTrue(set.contains(record2)); + assertThat(set).contains(record1, record2); Record record3 = record1.copy(); - Assert.assertTrue(set.contains(record3)); + assertThat(set).contains(record3); boolean removed = set.remove(record3); - Assert.assertTrue(removed); + assertThat(removed).isTrue(); } } diff --git a/core/src/test/java/org/apache/iceberg/util/TestTableScanUtil.java b/core/src/test/java/org/apache/iceberg/util/TestTableScanUtil.java index 4976ff4d9d0d..0dff941616ab 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestTableScanUtil.java +++ b/core/src/test/java/org/apache/iceberg/util/TestTableScanUtil.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.util; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -38,12 +40,10 @@ import org.apache.iceberg.StructLike; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; public class TestTableScanUtil { @@ -99,15 +99,13 @@ public void testPlanTaskWithDeleteFiles() { new BaseCombinedScanTask(Arrays.asList(testFiles.get(1), testFiles.get(2))), new BaseCombinedScanTask(Arrays.asList(testFiles.get(3), testFiles.get(4)))); - Assert.assertEquals( - "Should plan 3 Combined tasks since there is delete files to be considered", - 3, - combinedScanTasks.size()); + assertThat(combinedScanTasks) + .as("Should plan 3 Combined tasks since there is delete files to be considered") + .hasSize(3); for (int i = 0; i < expectedCombinedTasks.size(); ++i) { - Assert.assertEquals( - "Scan tasks detail in combined task check failed", - expectedCombinedTasks.get(i).files(), - combinedScanTasks.get(i).files()); + assertThat(combinedScanTasks.get(i).files()) + .as("Scan tasks detail in combined task check failed") + .isEqualTo(expectedCombinedTasks.get(i).files()); } } @@ -125,8 +123,7 @@ public void testTaskGroupPlanning() { CloseableIterable> taskGroups = TableScanUtil.planTaskGroups(CloseableIterable.withNoopClose(tasks), 128, 10, 4); - - Assert.assertEquals("Must have 3 task groups", 3, Iterables.size(taskGroups)); + assertThat(taskGroups).as("Must have 3 task groups").hasSize(3); } @Test @@ -139,7 +136,7 @@ public void testTaskMerging() { new ChildTask3(32), new ChildTask3(32)); List mergedTasks = TableScanUtil.mergeTasks(tasks); - Assert.assertEquals("Appropriate tasks should be merged", 3, mergedTasks.size()); + assertThat(mergedTasks).as("Appropriate tasks should be merged").hasSize(3); } private static final Schema TEST_SCHEMA = @@ -172,11 +169,11 @@ public void testTaskGroupPlanningByPartition() { int count = 0; for (ScanTaskGroup task : TableScanUtil.planTaskGroups(tasks, 512, 10, 4, SPEC1.partitionType())) { - Assert.assertEquals(4, task.filesCount()); - Assert.assertEquals(64 + 128 + 64 + 128, task.sizeBytes()); + assertThat(task.filesCount()).isEqualTo(4); + assertThat(task.sizeBytes()).isEqualTo(64 + 128 + 64 + 128); count += 1; } - Assert.assertEquals(1, count); + assertThat(count).isOne(); // We have 2 files from partition 1 and 2 files from partition 2, so they should be combined // separately @@ -190,11 +187,11 @@ public void testTaskGroupPlanningByPartition() { count = 0; for (ScanTaskGroup task : TableScanUtil.planTaskGroups(tasks, 512, 10, 4, SPEC1.partitionType())) { - Assert.assertEquals(2, task.filesCount()); - Assert.assertEquals(64 + 128, task.sizeBytes()); + assertThat(task.filesCount()).isEqualTo(2); + assertThat(task.sizeBytes()).isEqualTo(64 + 128); count += 1; } - Assert.assertEquals(2, count); + assertThat(count).isEqualTo(2); // Similar to the case above, but now files have different partition specs tasks = @@ -207,11 +204,11 @@ public void testTaskGroupPlanningByPartition() { count = 0; for (ScanTaskGroup task : TableScanUtil.planTaskGroups(tasks, 512, 10, 4, SPEC1.partitionType())) { - Assert.assertEquals(2, task.filesCount()); - Assert.assertEquals(64 + 128, task.sizeBytes()); + assertThat(task.filesCount()).isEqualTo(2); + assertThat(task.sizeBytes()).isEqualTo(64 + 128); count += 1; } - Assert.assertEquals(2, count); + assertThat(count).isEqualTo(2); // Combining within partitions should also respect split size. In this case, the split size // is equal to the file size, so each partition will have 2 tasks. @@ -225,11 +222,11 @@ public void testTaskGroupPlanningByPartition() { count = 0; for (ScanTaskGroup task : TableScanUtil.planTaskGroups(tasks, 128, 10, 4, SPEC1.partitionType())) { - Assert.assertEquals(1, task.filesCount()); - Assert.assertEquals(128, task.sizeBytes()); + assertThat(task.filesCount()).isOne(); + assertThat(task.sizeBytes()).isEqualTo(128); count += 1; } - Assert.assertEquals(4, count); + assertThat(count).isEqualTo(4); // The following should throw exception since `SPEC2` is not an intersection of partition specs // across all tasks. @@ -243,6 +240,20 @@ public void testTaskGroupPlanningByPartition() { .hasMessageStartingWith("Cannot find field"); } + @Test + public void testAdaptiveSplitSize() { + long scanSize = 500L * 1024 * 1024 * 1024; // 500 GB + int parallelism = 500; + long smallDefaultSplitSize = 128 * 1024 * 1024; // 128 MB + long largeDefaultSplitSize = 2L * 1024 * 1024 * 1024; // 2 GB + + long adjusted1 = TableScanUtil.adjustSplitSize(scanSize, parallelism, smallDefaultSplitSize); + assertThat(adjusted1).isEqualTo(smallDefaultSplitSize); + + long adjusted2 = TableScanUtil.adjustSplitSize(scanSize, parallelism, largeDefaultSplitSize); + assertThat(adjusted2).isEqualTo(scanSize / parallelism); + } + private PartitionScanTask taskWithPartition( PartitionSpec spec, StructLike partition, long sizeBytes) { PartitionScanTask task = Mockito.mock(PartitionScanTask.class); diff --git a/core/src/test/java/org/apache/iceberg/util/TestTasks.java b/core/src/test/java/org/apache/iceberg/util/TestTasks.java index f337a3500db4..2ca69c66bb6c 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestTasks.java +++ b/core/src/test/java/org/apache/iceberg/util/TestTasks.java @@ -18,11 +18,12 @@ */ package org.apache.iceberg.util; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.stream.IntStream; import org.apache.iceberg.metrics.Counter; import org.apache.iceberg.metrics.DefaultMetricsContext; -import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestTasks { @@ -45,7 +46,7 @@ public void attemptCounterIsIncreasedOnRetries() { } }); - Assertions.assertThat(counter.value()).isEqualTo(retries + 1); + assertThat(counter.value()).isEqualTo(retries + 1); } @Test @@ -54,6 +55,6 @@ public void attemptCounterIsIncreasedWithoutRetries() { Tasks.foreach(IntStream.range(0, 10)).countAttempts(counter).run(x -> {}); - Assertions.assertThat(counter.value()).isEqualTo(1L); + assertThat(counter.value()).isOne(); } } diff --git a/core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java b/core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java index 96c9126bbdb3..13e7c843c79f 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java +++ b/core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.util; +import static org.assertj.core.api.Assertions.assertThat; + import java.nio.ByteBuffer; import java.nio.charset.CharsetEncoder; import java.nio.charset.StandardCharsets; @@ -25,8 +27,7 @@ import java.util.Random; import org.apache.iceberg.relocated.com.google.common.primitives.UnsignedBytes; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestZOrderByteUtil { private static final byte IIIIIIII = (byte) 255; @@ -102,8 +103,9 @@ public void testInterleaveRandomExamples() { String stringResult = interleaveStrings(testStrings); - Assert.assertEquals( - "String interleave didn't match byte interleave", stringResult, byteResultAsString); + assertThat(byteResultAsString) + .as("String interleave didn't match byte interleave") + .isEqualTo(stringResult); } } @@ -126,8 +128,9 @@ public void testReuseInterleaveBuffer() { String stringResult = interleaveStrings(testStrings); - Assert.assertEquals( - "String interleave didn't match byte interleave", stringResult, byteResultAsString); + assertThat(byteResultAsString) + .as("String interleave didn't match byte interleave") + .isEqualTo(stringResult); } } @@ -136,8 +139,9 @@ public void testInterleaveEmptyBits() { byte[][] test = new byte[4][10]; byte[] expected = new byte[40]; - Assert.assertArrayEquals( - "Should combine empty arrays", expected, ZOrderByteUtils.interleaveBits(test, 40)); + assertThat(ZOrderByteUtils.interleaveBits(test, 40)) + .as("Should combine empty arrays") + .isEqualTo(expected); } @Test @@ -149,8 +153,9 @@ public void testInterleaveFullBits() { test[3] = new byte[] {IIIIIIII, IIIIIIII, IIIIIIII}; byte[] expected = new byte[] {IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII}; - Assert.assertArrayEquals( - "Should combine full arrays", expected, ZOrderByteUtils.interleaveBits(test, 6)); + assertThat(ZOrderByteUtils.interleaveBits(test, 6)) + .as("Should combine full arrays") + .isEqualTo(expected); } @Test @@ -164,8 +169,9 @@ public void testInterleaveMixedBits() { new byte[] { OOOOOOOO, OOOOOOOO, OOOOOOOO, OOOOIIII, IOIOIOIO, IOIOIOIO, OIOIOIOI, OIOIOIOI, OOOOIIII }; - Assert.assertArrayEquals( - "Should combine mixed byte arrays", expected, ZOrderByteUtils.interleaveBits(test, 9)); + assertThat(ZOrderByteUtils.interleaveBits(test, 9)) + .as("Should combine mixed byte arrays") + .isEqualTo(expected); } @Test @@ -181,17 +187,17 @@ public void testIntOrdering() { int byteCompare = Integer.signum(UnsignedBytes.lexicographicalComparator().compare(aBytes, bBytes)); - Assert.assertEquals( - String.format( - "Ordering of ints should match ordering of bytes, %s ~ %s -> %s != %s ~ %s -> %s ", - aInt, - bInt, - intCompare, - Arrays.toString(aBytes), - Arrays.toString(bBytes), - byteCompare), - intCompare, - byteCompare); + assertThat(byteCompare) + .as( + String.format( + "Ordering of ints should match ordering of bytes, %s ~ %s -> %s != %s ~ %s -> %s ", + aInt, + bInt, + intCompare, + Arrays.toString(aBytes), + Arrays.toString(bBytes), + byteCompare)) + .isEqualTo(intCompare); } } @@ -208,17 +214,17 @@ public void testLongOrdering() { int byteCompare = Integer.signum(UnsignedBytes.lexicographicalComparator().compare(aBytes, bBytes)); - Assert.assertEquals( - String.format( - "Ordering of longs should match ordering of bytes, %s ~ %s -> %s != %s ~ %s -> %s ", - aLong, - bLong, - longCompare, - Arrays.toString(aBytes), - Arrays.toString(bBytes), - byteCompare), - longCompare, - byteCompare); + assertThat(byteCompare) + .as( + String.format( + "Ordering of longs should match ordering of bytes, %s ~ %s -> %s != %s ~ %s -> %s ", + aLong, + bLong, + longCompare, + Arrays.toString(aBytes), + Arrays.toString(bBytes), + byteCompare)) + .isEqualTo(longCompare); } } @@ -235,17 +241,17 @@ public void testShortOrdering() { int byteCompare = Integer.signum(UnsignedBytes.lexicographicalComparator().compare(aBytes, bBytes)); - Assert.assertEquals( - String.format( - "Ordering of longs should match ordering of bytes, %s ~ %s -> %s != %s ~ %s -> %s ", - aShort, - bShort, - longCompare, - Arrays.toString(aBytes), - Arrays.toString(bBytes), - byteCompare), - longCompare, - byteCompare); + assertThat(byteCompare) + .as( + String.format( + "Ordering of longs should match ordering of bytes, %s ~ %s -> %s != %s ~ %s -> %s ", + aShort, + bShort, + longCompare, + Arrays.toString(aBytes), + Arrays.toString(bBytes), + byteCompare)) + .isEqualTo(longCompare); } } @@ -262,17 +268,17 @@ public void testTinyOrdering() { int byteCompare = Integer.signum(UnsignedBytes.lexicographicalComparator().compare(aBytes, bBytes)); - Assert.assertEquals( - String.format( - "Ordering of longs should match ordering of bytes, %s ~ %s -> %s != %s ~ %s -> %s ", - aByte, - bByte, - longCompare, - Arrays.toString(aBytes), - Arrays.toString(bBytes), - byteCompare), - longCompare, - byteCompare); + assertThat(byteCompare) + .as( + String.format( + "Ordering of longs should match ordering of bytes, %s ~ %s -> %s != %s ~ %s -> %s ", + aByte, + bByte, + longCompare, + Arrays.toString(aBytes), + Arrays.toString(bBytes), + byteCompare)) + .isEqualTo(longCompare); } } @@ -289,17 +295,17 @@ public void testFloatOrdering() { int byteCompare = Integer.signum(UnsignedBytes.lexicographicalComparator().compare(aBytes, bBytes)); - Assert.assertEquals( - String.format( - "Ordering of floats should match ordering of bytes, %s ~ %s -> %s != %s ~ %s -> %s ", - aFloat, - bFloat, - floatCompare, - Arrays.toString(aBytes), - Arrays.toString(bBytes), - byteCompare), - floatCompare, - byteCompare); + assertThat(byteCompare) + .as( + String.format( + "Ordering of floats should match ordering of bytes, %s ~ %s -> %s != %s ~ %s -> %s ", + aFloat, + bFloat, + floatCompare, + Arrays.toString(aBytes), + Arrays.toString(bBytes), + byteCompare)) + .isEqualTo(floatCompare); } } @@ -316,17 +322,17 @@ public void testDoubleOrdering() { int byteCompare = Integer.signum(UnsignedBytes.lexicographicalComparator().compare(aBytes, bBytes)); - Assert.assertEquals( - String.format( - "Ordering of doubles should match ordering of bytes, %s ~ %s -> %s != %s ~ %s -> %s ", - aDouble, - bDouble, - doubleCompare, - Arrays.toString(aBytes), - Arrays.toString(bBytes), - byteCompare), - doubleCompare, - byteCompare); + assertThat(byteCompare) + .as( + String.format( + "Ordering of doubles should match ordering of bytes, %s ~ %s -> %s != %s ~ %s -> %s ", + aDouble, + bDouble, + doubleCompare, + Arrays.toString(aBytes), + Arrays.toString(bBytes), + byteCompare)) + .isEqualTo(doubleCompare); } } @@ -344,17 +350,17 @@ public void testStringOrdering() { int byteCompare = Integer.signum(UnsignedBytes.lexicographicalComparator().compare(aBytes, bBytes)); - Assert.assertEquals( - String.format( - "Ordering of strings should match ordering of bytes, %s ~ %s -> %s != %s ~ %s -> %s ", - aString, - bString, - stringCompare, - Arrays.toString(aBytes), - Arrays.toString(bBytes), - byteCompare), - stringCompare, - byteCompare); + assertThat(byteCompare) + .as( + String.format( + "Ordering of strings should match ordering of bytes, %s ~ %s -> %s != %s ~ %s -> %s ", + aString, + bString, + stringCompare, + Arrays.toString(aBytes), + Arrays.toString(bBytes), + byteCompare)) + .isEqualTo(stringCompare); } } @@ -372,17 +378,17 @@ public void testByteTruncateOrFill() { int byteCompare = Integer.signum(UnsignedBytes.lexicographicalComparator().compare(aBytes, bBytes)); - Assert.assertEquals( - String.format( - "Ordering of strings should match ordering of bytes, %s ~ %s -> %s != %s ~ %s -> %s ", - aBytesRaw, - bBytesRaw, - stringCompare, - Arrays.toString(aBytes), - Arrays.toString(bBytes), - byteCompare), - stringCompare, - byteCompare); + assertThat(byteCompare) + .as( + String.format( + "Ordering of strings should match ordering of bytes, %s ~ %s -> %s != %s ~ %s -> %s ", + aBytesRaw, + bBytesRaw, + stringCompare, + Arrays.toString(aBytes), + Arrays.toString(bBytes), + byteCompare)) + .isEqualTo(stringCompare); } } } diff --git a/core/src/test/java/org/apache/iceberg/view/TestSQLViewRepresentationParser.java b/core/src/test/java/org/apache/iceberg/view/TestSQLViewRepresentationParser.java index 54f2cf85bc69..8eae11a4e67d 100644 --- a/core/src/test/java/org/apache/iceberg/view/TestSQLViewRepresentationParser.java +++ b/core/src/test/java/org/apache/iceberg/view/TestSQLViewRepresentationParser.java @@ -18,11 +18,9 @@ */ package org.apache.iceberg.view; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import com.fasterxml.jackson.databind.JsonNode; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestSQLViewRepresentationParser { @Test @@ -35,31 +33,21 @@ public void testParseSqlViewRepresentation() { .dialect("spark-sql") .build(); - Assert.assertEquals( - "Should be able to parse valid SQL view representation", - viewRepresentation, - SQLViewRepresentationParser.fromJson(requiredFields)); + Assertions.assertThat(SQLViewRepresentationParser.fromJson(requiredFields)) + .as("Should be able to parse valid SQL view representation") + .isEqualTo(viewRepresentation); String requiredAndOptionalFields = - "{\"type\":\"sql\", \"sql\": \"select * from foo\", \"dialect\": \"spark-sql\", " - + "\"default-catalog\":\"cat\", " - + "\"default-namespace\":[\"part1\",\"part2\"], " - + "\"field-aliases\":[\"col1\", \"col2\"], " - + "\"field-comments\":[\"Comment col1\", \"Comment col2\"]}"; + "{\"type\":\"sql\", \"sql\": \"select * from foo\", \"dialect\": \"spark-sql\"}"; SQLViewRepresentation viewWithOptionalFields = ImmutableSQLViewRepresentation.builder() .sql("select * from foo") .dialect("spark-sql") - .defaultCatalog("cat") - .fieldAliases(ImmutableList.of("col1", "col2")) - .fieldComments(ImmutableList.of("Comment col1", "Comment col2")) - .defaultNamespace(Namespace.of("part1", "part2")) .build(); - Assert.assertEquals( - "Should be able to parse valid SQL view representation", - viewWithOptionalFields, - SQLViewRepresentationParser.fromJson(requiredAndOptionalFields)); + Assertions.assertThat(SQLViewRepresentationParser.fromJson(requiredAndOptionalFields)) + .as("Should be able to parse valid SQL view representation") + .isEqualTo(viewWithOptionalFields); } @Test @@ -77,39 +65,18 @@ public void testParseSqlViewRepresentationMissingRequiredFields() { @Test public void testViewRepresentationSerialization() { - String requiredFields = - "{\"type\":\"sql\",\"sql\":\"select * from foo\",\"dialect\":\"spark-sql\"}"; + String json = "{\"type\":\"sql\",\"sql\":\"select * from foo\",\"dialect\":\"spark-sql\"}"; SQLViewRepresentation viewRepresentation = ImmutableSQLViewRepresentation.builder() .sql("select * from foo") .dialect("spark-sql") .build(); - Assert.assertEquals( - "Should be able to serialize valid SQL view representation", - requiredFields, - ViewRepresentationParser.toJson(viewRepresentation)); - - String requiredAndOptionalFields = - "{\"type\":\"sql\",\"sql\":\"select * from foo\",\"dialect\":\"spark-sql\"," - + "\"default-catalog\":\"cat\"," - + "\"default-namespace\":[\"part1\",\"part2\"]," - + "\"field-aliases\":[\"col1\",\"col2\"]," - + "\"field-comments\":[\"Comment col1\",\"Comment col2\"]}"; - - SQLViewRepresentation viewWithOptionalFields = - ImmutableSQLViewRepresentation.builder() - .sql("select * from foo") - .dialect("spark-sql") - .defaultCatalog("cat") - .fieldAliases(ImmutableList.of("col1", "col2")) - .fieldComments(ImmutableList.of("Comment col1", "Comment col2")) - .defaultNamespace(Namespace.of("part1", "part2")) - .build(); - - Assert.assertEquals( - "Should be able to serialize valid SQL view representation", - requiredAndOptionalFields, - ViewRepresentationParser.toJson(viewWithOptionalFields)); + Assertions.assertThat(ViewRepresentationParser.toJson(viewRepresentation)) + .as("Should be able to serialize valid SQL view representation") + .isEqualTo(json); + Assertions.assertThat( + ViewRepresentationParser.fromJson(ViewRepresentationParser.toJson(viewRepresentation))) + .isEqualTo(viewRepresentation); } @Test @@ -117,5 +84,9 @@ public void testNullSqlViewRepresentation() { Assertions.assertThatThrownBy(() -> SQLViewRepresentationParser.toJson(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid SQL view representation: null"); + + Assertions.assertThatThrownBy(() -> SQLViewRepresentationParser.fromJson((JsonNode) null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse SQL view representation from null object"); } } diff --git a/core/src/test/java/org/apache/iceberg/view/TestViewHistoryEntryParser.java b/core/src/test/java/org/apache/iceberg/view/TestViewHistoryEntryParser.java index 1e2c33e44b00..99c24fbff456 100644 --- a/core/src/test/java/org/apache/iceberg/view/TestViewHistoryEntryParser.java +++ b/core/src/test/java/org/apache/iceberg/view/TestViewHistoryEntryParser.java @@ -20,8 +20,7 @@ import com.fasterxml.jackson.databind.JsonNode; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestViewHistoryEntryParser { @@ -30,10 +29,9 @@ public void testViewHistoryEntryFromJson() { String json = "{\"timestamp-ms\":123,\"version-id\":1}"; ViewHistoryEntry viewHistoryEntry = ImmutableViewHistoryEntry.builder().versionId(1).timestampMillis(123).build(); - Assert.assertEquals( - "Should be able to deserialize valid view history entry", - viewHistoryEntry, - ViewHistoryEntryParser.fromJson(json)); + Assertions.assertThat(ViewHistoryEntryParser.fromJson(json)) + .as("Should be able to deserialize valid view history entry") + .isEqualTo(viewHistoryEntry); } @Test @@ -41,10 +39,9 @@ public void testViewHistoryEntryToJson() { String json = "{\"timestamp-ms\":123,\"version-id\":1}"; ViewHistoryEntry viewHistoryEntry = ImmutableViewHistoryEntry.builder().versionId(1).timestampMillis(123).build(); - Assert.assertEquals( - "Should be able to serialize view history entry", - json, - ViewHistoryEntryParser.toJson(viewHistoryEntry)); + Assertions.assertThat(ViewHistoryEntryParser.toJson(viewHistoryEntry)) + .as("Should be able to serialize view history entry") + .isEqualTo(json); } @Test diff --git a/core/src/test/java/org/apache/iceberg/view/TestViewMetadata.java b/core/src/test/java/org/apache/iceberg/view/TestViewMetadata.java new file mode 100644 index 000000000000..ad167bb676ac --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/view/TestViewMetadata.java @@ -0,0 +1,294 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.view; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import org.apache.iceberg.Schema; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestViewMetadata { + + @Test + public void nullAndMissingFields() { + assertThatThrownBy(() -> ImmutableViewMetadata.builder().build()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Cannot build ViewMetadata, some of required attributes are not set [formatVersion, location, currentSchemaId, currentVersionId]"); + + assertThatThrownBy(() -> ImmutableViewMetadata.builder().formatVersion(1).build()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Cannot build ViewMetadata, some of required attributes are not set [location, currentSchemaId, currentVersionId]"); + + assertThatThrownBy( + () -> ImmutableViewMetadata.builder().formatVersion(1).location("location").build()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Cannot build ViewMetadata, some of required attributes are not set [currentSchemaId, currentVersionId]"); + + assertThatThrownBy( + () -> + ImmutableViewMetadata.builder() + .formatVersion(1) + .location("location") + .currentSchemaId(1) + .build()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Cannot build ViewMetadata, some of required attributes are not set [currentVersionId]"); + + assertThatThrownBy( + () -> + ImmutableViewMetadata.builder() + .formatVersion(1) + .location("location") + .currentSchemaId(1) + .currentVersionId(1) + .build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid view versions: empty"); + } + + @Test + public void unsupportedFormatVersion() { + assertThatThrownBy( + () -> + ImmutableViewMetadata.builder() + .formatVersion(23) + .location("location") + .currentSchemaId(1) + .currentVersionId(1) + .build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Unsupported format version: 23"); + } + + @Test + public void emptyViewVersion() { + assertThatThrownBy( + () -> + ImmutableViewMetadata.builder() + .formatVersion(1) + .location("location") + .currentSchemaId(1) + .currentVersionId(1) + .build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid view versions: empty"); + } + + @Test + public void emptySchemas() { + assertThatThrownBy( + () -> + ImmutableViewMetadata.builder() + .formatVersion(1) + .location("location") + .currentSchemaId(1) + .currentVersionId(1) + .addVersions( + ImmutableViewVersion.builder() + .schemaId(1) + .versionId(1) + .timestampMillis(23L) + .putSummary("operation", "op") + .build()) + .addHistory( + ImmutableViewHistoryEntry.builder() + .timestampMillis(23L) + .versionId(1) + .build()) + .build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid schemas: empty"); + } + + @Test + public void invalidCurrentVersionId() { + assertThatThrownBy( + () -> + ImmutableViewMetadata.builder() + .formatVersion(1) + .location("location") + .currentSchemaId(1) + .currentVersionId(23) + .addVersions( + ImmutableViewVersion.builder() + .schemaId(1) + .versionId(1) + .timestampMillis(23L) + .putSummary("operation", "op") + .build()) + .addHistory( + ImmutableViewHistoryEntry.builder() + .timestampMillis(23L) + .versionId(1) + .build()) + .addSchemas( + new Schema(1, Types.NestedField.required(1, "x", Types.LongType.get()))) + .build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot find current version 23 in view versions: [1]"); + } + + @Test + public void invalidCurrentSchemaId() { + assertThatThrownBy( + () -> + ImmutableViewMetadata.builder() + .formatVersion(1) + .location("location") + .currentSchemaId(23) + .currentVersionId(1) + .addVersions( + ImmutableViewVersion.builder() + .schemaId(1) + .versionId(1) + .timestampMillis(23L) + .putSummary("operation", "op") + .build()) + .addHistory( + ImmutableViewHistoryEntry.builder() + .timestampMillis(23L) + .versionId(1) + .build()) + .addSchemas( + new Schema(1, Types.NestedField.required(1, "x", Types.LongType.get()))) + .build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot find current schema with id 23 in schemas: [1]"); + } + + @Test + public void invalidVersionHistorySizeToKeep() { + ImmutableViewMetadata viewMetadata = + ImmutableViewMetadata.builder() + // setting history to < 1 shouldn't do anything and only issue a WARN + .properties(ImmutableMap.of(ViewProperties.VERSION_HISTORY_SIZE, "0")) + .formatVersion(1) + .location("location") + .currentSchemaId(1) + .currentVersionId(3) + .addVersions( + ImmutableViewVersion.builder() + .schemaId(1) + .versionId(1) + .timestampMillis(23L) + .putSummary("operation", "a") + .build()) + .addVersions( + ImmutableViewVersion.builder() + .schemaId(1) + .versionId(2) + .timestampMillis(24L) + .putSummary("operation", "b") + .build()) + .addVersions( + ImmutableViewVersion.builder() + .schemaId(1) + .versionId(3) + .timestampMillis(25L) + .putSummary("operation", "c") + .build()) + .addHistory( + ImmutableViewHistoryEntry.builder().versionId(1).timestampMillis(23L).build()) + .addHistory( + ImmutableViewHistoryEntry.builder().versionId(2).timestampMillis(24L).build()) + .addSchemas(new Schema(1, Types.NestedField.required(1, "x", Types.LongType.get()))) + .build(); + + assertThat(viewMetadata.versions()).hasSize(3); + assertThat(viewMetadata.history()).hasSize(2); + } + + @Test + public void emptyHistory() { + assertThatThrownBy( + () -> + ImmutableViewMetadata.builder() + .formatVersion(1) + .location("location") + .currentSchemaId(1) + .currentVersionId(2) + .addVersions( + ImmutableViewVersion.builder() + .schemaId(1) + .versionId(1) + .timestampMillis(23L) + .putSummary("operation", "op") + .build()) + .addVersions( + ImmutableViewVersion.builder() + .schemaId(1) + .versionId(2) + .timestampMillis(24L) + .putSummary("operation", "op") + .build()) + .addSchemas( + new Schema(1, Types.NestedField.required(1, "x", Types.LongType.get()))) + .build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid view history: empty"); + } + + @Test + public void viewHistoryNormalization() { + ImmutableViewMetadata viewMetadata = + ImmutableViewMetadata.builder() + .properties(ImmutableMap.of(ViewProperties.VERSION_HISTORY_SIZE, "1")) + .formatVersion(1) + .location("location") + .currentSchemaId(1) + .currentVersionId(3) + .addVersions( + ImmutableViewVersion.builder() + .schemaId(1) + .versionId(1) + .timestampMillis(23L) + .putSummary("operation", "a") + .build()) + .addVersions( + ImmutableViewVersion.builder() + .schemaId(1) + .versionId(2) + .timestampMillis(24L) + .putSummary("operation", "b") + .build()) + .addVersions( + ImmutableViewVersion.builder() + .schemaId(1) + .versionId(3) + .timestampMillis(25L) + .putSummary("operation", "c") + .build()) + .addHistory( + ImmutableViewHistoryEntry.builder().versionId(1).timestampMillis(23L).build()) + .addHistory( + ImmutableViewHistoryEntry.builder().versionId(2).timestampMillis(24L).build()) + .addSchemas(new Schema(1, Types.NestedField.required(1, "x", Types.LongType.get()))) + .build(); + + assertThat(viewMetadata.versions()).hasSize(1); + assertThat(viewMetadata.history()).hasSize(1); + } +} diff --git a/core/src/test/java/org/apache/iceberg/view/TestViewMetadataParser.java b/core/src/test/java/org/apache/iceberg/view/TestViewMetadataParser.java new file mode 100644 index 000000000000..f4bf2118f140 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/view/TestViewMetadataParser.java @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.view; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import com.fasterxml.jackson.databind.JsonNode; +import java.nio.file.Path; +import java.nio.file.Paths; +import org.apache.iceberg.Schema; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestViewMetadataParser { + + private static final Schema TEST_SCHEMA = + new Schema( + 1, + Types.NestedField.required(1, "x", Types.LongType.get()), + Types.NestedField.required(2, "y", Types.LongType.get(), "comment"), + Types.NestedField.required(3, "z", Types.LongType.get())); + + @Test + public void nullAndEmptyCheck() { + assertThatThrownBy(() -> ViewMetadataParser.fromJson((String) null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse view metadata from null string"); + + assertThatThrownBy(() -> ViewMetadataParser.fromJson((JsonNode) null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse view metadata from null object"); + + assertThatThrownBy(() -> ViewMetadataParser.toJson(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid view metadata: null"); + } + + @Test + public void readAndWriteValidViewMetadata() throws Exception { + ViewVersion version1 = + ImmutableViewVersion.builder() + .versionId(1) + .timestampMillis(4353L) + .summary(ImmutableMap.of("operation", "create")) + .schemaId(1) + .defaultCatalog("some-catalog") + .addRepresentations( + ImmutableSQLViewRepresentation.builder() + .sql("select 'foo' foo") + .dialect("spark-sql") + .build()) + .build(); + + ViewHistoryEntry historyEntry1 = + ImmutableViewHistoryEntry.builder().timestampMillis(4353L).versionId(1).build(); + + ViewVersion version2 = + ImmutableViewVersion.builder() + .versionId(2) + .schemaId(1) + .timestampMillis(5555L) + .summary(ImmutableMap.of("operation", "replace")) + .defaultCatalog("some-catalog") + .addRepresentations( + ImmutableSQLViewRepresentation.builder() + .sql("select 1 id, 'abc' data") + .dialect("spark-sql") + .build()) + .build(); + + ViewHistoryEntry historyEntry2 = + ImmutableViewHistoryEntry.builder().timestampMillis(5555L).versionId(2).build(); + + String json = readViewMetadataInputFile("org/apache/iceberg/view/ValidViewMetadata.json"); + ViewMetadata expectedViewMetadata = + ImmutableViewMetadata.builder() + .currentSchemaId(1) + .schemas(ImmutableList.of(TEST_SCHEMA)) + .versions(ImmutableList.of(version1, version2)) + .history(ImmutableList.of(historyEntry1, historyEntry2)) + .location("s3://bucket/test/location") + .properties(ImmutableMap.of("some-key", "some-value")) + .currentVersionId(2) + .formatVersion(1) + .build(); + + ViewMetadata actual = ViewMetadataParser.fromJson(json); + assertThat(actual) + .usingRecursiveComparison() + .ignoringFieldsOfTypes(Schema.class) + .isEqualTo(expectedViewMetadata); + for (Schema schema : expectedViewMetadata.schemas()) { + assertThat(schema.sameSchema(actual.schemasById().get(schema.schemaId()))).isTrue(); + } + + actual = ViewMetadataParser.fromJson(ViewMetadataParser.toJson(expectedViewMetadata)); + assertThat(actual) + .usingRecursiveComparison() + .ignoringFieldsOfTypes(Schema.class) + .isEqualTo(expectedViewMetadata); + for (Schema schema : expectedViewMetadata.schemas()) { + assertThat(schema.sameSchema(actual.schemasById().get(schema.schemaId()))).isTrue(); + } + } + + @Test + public void readViewMetadataWithLimitedNumberVersionEntries() throws Exception { + String json = + readViewMetadataInputFile("org/apache/iceberg/view/ViewMetadataLimitedVersions.json"); + + ViewMetadata viewMetadata = ViewMetadataParser.fromJson(json); + assertThat(viewMetadata.versions()).hasSize(1); + assertThat(viewMetadata.history()).hasSize(1); + } + + @Test + public void failReadingViewMetadataMissingLocation() throws Exception { + String json = + readViewMetadataInputFile("org/apache/iceberg/view/ViewMetadataMissingLocation.json"); + assertThatThrownBy(() -> ViewMetadataParser.fromJson(json)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse missing string: location"); + } + + @Test + public void failReadingViewMetadataMissingCurrentSchema() throws Exception { + String json = + readViewMetadataInputFile("org/apache/iceberg/view/ViewMetadataMissingCurrentSchema.json"); + assertThatThrownBy(() -> ViewMetadataParser.fromJson(json)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse missing int: current-schema-id"); + } + + @Test + public void failReadingViewMetadataInvalidSchemaId() throws Exception { + String json = + readViewMetadataInputFile("org/apache/iceberg/view/ViewMetadataInvalidCurrentSchema.json"); + assertThatThrownBy(() -> ViewMetadataParser.fromJson(json)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot find current schema with id 1234 in schemas: [1]"); + } + + @Test + public void failReadingViewMetadataMissingVersion() throws Exception { + String json = + readViewMetadataInputFile("org/apache/iceberg/view/ViewMetadataMissingCurrentVersion.json"); + assertThatThrownBy(() -> ViewMetadataParser.fromJson(json)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse missing int: current-version-id"); + } + + @Test + public void failReadingViewMetadataInvalidVersionId() throws Exception { + String json = + readViewMetadataInputFile("org/apache/iceberg/view/ViewMetadataInvalidCurrentVersion.json"); + assertThatThrownBy(() -> ViewMetadataParser.fromJson(json)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot find current version 1234 in view versions: [1, 2]"); + } + + private String readViewMetadataInputFile(String fileName) throws Exception { + Path path = Paths.get(getClass().getClassLoader().getResource(fileName).toURI()); + return String.join("", java.nio.file.Files.readAllLines(path)); + } +} diff --git a/core/src/test/java/org/apache/iceberg/view/TestViewRepresentationParser.java b/core/src/test/java/org/apache/iceberg/view/TestViewRepresentationParser.java index b45869c33641..37f9ae2eaa12 100644 --- a/core/src/test/java/org/apache/iceberg/view/TestViewRepresentationParser.java +++ b/core/src/test/java/org/apache/iceberg/view/TestViewRepresentationParser.java @@ -19,8 +19,7 @@ package org.apache.iceberg.view; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestViewRepresentationParser { @@ -28,9 +27,9 @@ public class TestViewRepresentationParser { public void testParseUnknownViewRepresentation() { String json = "{\"type\":\"unknown-sql-representation\"}"; ViewRepresentation unknownRepresentation = ViewRepresentationParser.fromJson(json); - Assert.assertEquals( - unknownRepresentation, - ImmutableUnknownViewRepresentation.builder().type("unknown-sql-representation").build()); + Assertions.assertThat( + ImmutableUnknownViewRepresentation.builder().type("unknown-sql-representation").build()) + .isEqualTo(unknownRepresentation); Assertions.assertThatThrownBy(() -> ViewRepresentationParser.toJson(unknownRepresentation)) .isInstanceOf(UnsupportedOperationException.class) diff --git a/core/src/test/java/org/apache/iceberg/view/TestViewVersionParser.java b/core/src/test/java/org/apache/iceberg/view/TestViewVersionParser.java index 9ecb68377abc..3272a012b548 100644 --- a/core/src/test/java/org/apache/iceberg/view/TestViewVersionParser.java +++ b/core/src/test/java/org/apache/iceberg/view/TestViewVersionParser.java @@ -19,10 +19,10 @@ package org.apache.iceberg.view; import com.fasterxml.jackson.databind.JsonNode; +import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestViewVersionParser { @@ -57,10 +57,9 @@ public void testParseViewVersion() { "{\"version-id\":1, \"timestamp-ms\":12345, \"schema-id\":1, \"summary\":{\"operation\":\"create\", \"user\":\"some-user\"}, \"representations\":%s}", serializedRepresentations); - Assert.assertEquals( - "Should be able to parse valid view version", - expectedViewVersion, - ViewVersionParser.fromJson(serializedViewVersion)); + Assertions.assertThat(ViewVersionParser.fromJson(serializedViewVersion)) + .as("Should be able to parse valid view version") + .isEqualTo(expectedViewVersion); } @Test @@ -82,6 +81,8 @@ public void testSerializeViewVersion() { .timestampMillis(12345) .addRepresentations(firstRepresentation, secondRepresentation) .summary(ImmutableMap.of("operation", "create", "user", "some-user")) + .defaultNamespace(Namespace.of("one", "two")) + .defaultCatalog("catalog") .schemaId(1) .build(); @@ -91,13 +92,13 @@ public void testSerializeViewVersion() { String expectedViewVersion = String.format( - "{\"version-id\":1,\"timestamp-ms\":12345,\"schema-id\":1,\"summary\":{\"operation\":\"create\",\"user\":\"some-user\"},\"representations\":%s}", + "{\"version-id\":1,\"timestamp-ms\":12345,\"schema-id\":1,\"summary\":{\"operation\":\"create\",\"user\":\"some-user\"}," + + "\"default-catalog\":\"catalog\",\"default-namespace\":[\"one\",\"two\"],\"representations\":%s}", expectedRepresentations); - Assert.assertEquals( - "Should be able to serialize valid view version", - expectedViewVersion, - ViewVersionParser.toJson(viewVersion)); + Assertions.assertThat(ViewVersionParser.toJson(viewVersion)) + .as("Should be able to serialize valid view version") + .isEqualTo(expectedViewVersion); } @Test diff --git a/core/src/test/resources/TableMetadataV2ValidMinimal.json b/core/src/test/resources/TableMetadataV2ValidMinimal.json new file mode 100644 index 000000000000..529b10d1fdf1 --- /dev/null +++ b/core/src/test/resources/TableMetadataV2ValidMinimal.json @@ -0,0 +1,71 @@ +{ + "format-version": 2, + "table-uuid": "9c12d441-03fe-4693-9a96-a0705ddf69c1", + "location": "s3://bucket/test/location", + "last-sequence-number": 34, + "last-updated-ms": 1602638573590, + "last-column-id": 3, + "current-schema-id": 0, + "schemas": [ + { + "type": "struct", + "schema-id": 0, + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + }, + { + "id": 2, + "name": "y", + "required": true, + "type": "long", + "doc": "comment" + }, + { + "id": 3, + "name": "z", + "required": true, + "type": "long" + } + ] + } + ], + "default-spec-id": 0, + "partition-specs": [ + { + "spec-id": 0, + "fields": [ + { + "name": "x", + "transform": "identity", + "source-id": 1, + "field-id": 1000 + } + ] + } + ], + "last-partition-id": 1000, + "default-sort-order-id": 3, + "sort-orders": [ + { + "order-id": 3, + "fields": [ + { + "transform": "identity", + "source-id": 2, + "direction": "asc", + "null-order": "nulls-first" + }, + { + "transform": "bucket[4]", + "source-id": 3, + "direction": "desc", + "null-order": "nulls-last" + } + ] + } + ] +} \ No newline at end of file diff --git a/core/src/test/resources/org/apache/iceberg/view/ValidViewMetadata.json b/core/src/test/resources/org/apache/iceberg/view/ValidViewMetadata.json new file mode 100644 index 000000000000..9c0ae0ecbef6 --- /dev/null +++ b/core/src/test/resources/org/apache/iceberg/view/ValidViewMetadata.json @@ -0,0 +1,76 @@ +{ + "format-version": 1, + "location": "s3://bucket/test/location", + "properties": {"some-key": "some-value"}, + "current-schema-id": 1, + "schemas": [ + { + "type": "struct", + "schema-id": 1, + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + }, + { + "id": 2, + "name": "y", + "required": true, + "type": "long", + "doc": "comment" + }, + { + "id": 3, + "name": "z", + "required": true, + "type": "long" + } + ] + } + ], + "current-version-id": 2, + "versions": [ + { + "version-id": 1, + "timestamp-ms": 4353, + "summary": {"operation":"create"}, + "schema-id": 1, + "default-catalog": "some-catalog", + "default-namespace": [], + "representations": [ + { + "type": "sql", + "sql": "select 'foo' foo", + "dialect": "spark-sql" + } + ] + }, + { + "version-id": 2, + "timestamp-ms": 5555, + "summary": {"operation": "replace"}, + "schema-id": 1, + "default-catalog": "some-catalog", + "default-namespace": [], + "representations": [ + { + "type": "sql", + "sql": "select 1 id, 'abc' data", + "dialect": "spark-sql" + } + ] + } + ], + "version-log": [ + { + "timestamp-ms": 4353, + "version-id": 1 + }, + { + "timestamp-ms": 5555, + "version-id": 2 + } + ] +} \ No newline at end of file diff --git a/core/src/test/resources/org/apache/iceberg/view/ViewMetadataInvalidCurrentSchema.json b/core/src/test/resources/org/apache/iceberg/view/ViewMetadataInvalidCurrentSchema.json new file mode 100644 index 000000000000..b90befe515c2 --- /dev/null +++ b/core/src/test/resources/org/apache/iceberg/view/ViewMetadataInvalidCurrentSchema.json @@ -0,0 +1,76 @@ +{ + "format-version": 1, + "location": "s3://bucket/test/location", + "properties": {"some-key": "some-value"}, + "current-schema-id": 1234, + "schemas": [ + { + "type": "struct", + "schema-id": 1, + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + }, + { + "id": 2, + "name": "y", + "required": true, + "type": "long", + "doc": "comment" + }, + { + "id": 3, + "name": "z", + "required": true, + "type": "long" + } + ] + } + ], + "current-version-id": 2, + "versions": [ + { + "version-id": 1, + "timestamp-ms": 4353, + "summary": {"operation":"create"}, + "schema-id": 1, + "default-catalog": "some-catalog", + "default-namespace": [], + "representations": [ + { + "type": "sql", + "sql": "select 'foo' foo", + "dialect": "spark-sql" + } + ] + }, + { + "version-id": 2, + "timestamp-ms": 5555, + "summary": {"operation": "replace"}, + "schema-id": 1, + "default-catalog": "some-catalog", + "default-namespace": [], + "representations": [ + { + "type": "sql", + "sql": "select 1 id, 'abc' data", + "dialect": "spark-sql" + } + ] + } + ], + "version-log": [ + { + "timestamp-ms": 4353, + "version-id": 1 + }, + { + "timestamp-ms": 5555, + "version-id": 2 + } + ] +} \ No newline at end of file diff --git a/core/src/test/resources/org/apache/iceberg/view/ViewMetadataInvalidCurrentVersion.json b/core/src/test/resources/org/apache/iceberg/view/ViewMetadataInvalidCurrentVersion.json new file mode 100644 index 000000000000..fbcb2c9a4176 --- /dev/null +++ b/core/src/test/resources/org/apache/iceberg/view/ViewMetadataInvalidCurrentVersion.json @@ -0,0 +1,76 @@ +{ + "format-version": 1, + "location": "s3://bucket/test/location", + "properties": {"some-key": "some-value"}, + "current-schema-id": 1, + "current-version-id": 1234, + "schemas": [ + { + "type": "struct", + "schema-id": 1, + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + }, + { + "id": 2, + "name": "y", + "required": true, + "type": "long", + "doc": "comment" + }, + { + "id": 3, + "name": "z", + "required": true, + "type": "long" + } + ] + } + ], + "versions": [ + { + "version-id": 1, + "timestamp-ms": 4353, + "summary": {"operation":"create"}, + "schema-id": 1, + "default-catalog": "some-catalog", + "default-namespace": [], + "representations": [ + { + "type": "sql", + "sql": "select 'foo' foo", + "dialect": "spark-sql" + } + ] + }, + { + "version-id": 2, + "timestamp-ms": 5555, + "summary": {"operation": "replace"}, + "schema-id": 1, + "default-catalog": "some-catalog", + "default-namespace": [], + "representations": [ + { + "type": "sql", + "sql": "select 1 id, 'abc' data", + "dialect": "spark-sql" + } + ] + } + ], + "version-log": [ + { + "timestamp-ms": 4353, + "version-id": 1 + }, + { + "timestamp-ms": 5555, + "version-id": 2 + } + ] +} \ No newline at end of file diff --git a/core/src/test/resources/org/apache/iceberg/view/ViewMetadataLimitedVersions.json b/core/src/test/resources/org/apache/iceberg/view/ViewMetadataLimitedVersions.json new file mode 100644 index 000000000000..db80a3baefb7 --- /dev/null +++ b/core/src/test/resources/org/apache/iceberg/view/ViewMetadataLimitedVersions.json @@ -0,0 +1,76 @@ +{ + "format-version": 1, + "location": "s3://bucket/test/location", + "properties": {"version.history.num-entries": "1"}, + "current-schema-id": 1, + "schemas": [ + { + "type": "struct", + "schema-id": 1, + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + }, + { + "id": 2, + "name": "y", + "required": true, + "type": "long", + "doc": "comment" + }, + { + "id": 3, + "name": "z", + "required": true, + "type": "long" + } + ] + } + ], + "current-version-id": 2, + "versions": [ + { + "version-id": 1, + "timestamp-ms": 4353, + "summary": {"operation":"create"}, + "schema-id": 1, + "default-catalog": "some-catalog", + "default-namespace": [], + "representations": [ + { + "type": "sql", + "sql": "select 'foo' foo", + "dialect": "spark-sql" + } + ] + }, + { + "version-id": 2, + "timestamp-ms": 5555, + "summary": {"operation": "replace"}, + "schema-id": 1, + "default-catalog": "some-catalog", + "default-namespace": [], + "representations": [ + { + "type": "sql", + "sql": "select 1 id, 'abc' data", + "dialect": "spark-sql" + } + ] + } + ], + "version-log": [ + { + "timestamp-ms": 4353, + "version-id": 1 + }, + { + "timestamp-ms": 5555, + "version-id": 2 + } + ] +} \ No newline at end of file diff --git a/core/src/test/resources/org/apache/iceberg/view/ViewMetadataMissingCurrentSchema.json b/core/src/test/resources/org/apache/iceberg/view/ViewMetadataMissingCurrentSchema.json new file mode 100644 index 000000000000..f2dc76723ddb --- /dev/null +++ b/core/src/test/resources/org/apache/iceberg/view/ViewMetadataMissingCurrentSchema.json @@ -0,0 +1,75 @@ +{ + "format-version": 1, + "location": "s3://bucket/test/location", + "properties": {"some-key": "some-value"}, + "schemas": [ + { + "type": "struct", + "schema-id": 1, + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + }, + { + "id": 2, + "name": "y", + "required": true, + "type": "long", + "doc": "comment" + }, + { + "id": 3, + "name": "z", + "required": true, + "type": "long" + } + ] + } + ], + "current-version-id": 2, + "versions": [ + { + "version-id": 1, + "timestamp-ms": 4353, + "summary": {"operation":"create"}, + "schema-id": 1, + "default-catalog": "some-catalog", + "default-namespace": [], + "representations": [ + { + "type": "sql", + "sql": "select 'foo' foo", + "dialect": "spark-sql" + } + ] + }, + { + "version-id": 2, + "timestamp-ms": 5555, + "summary": {"operation": "replace"}, + "schema-id": 1, + "default-catalog": "some-catalog", + "default-namespace": [], + "representations": [ + { + "type": "sql", + "sql": "select 1 id, 'abc' data", + "dialect": "spark-sql" + } + ] + } + ], + "version-log": [ + { + "timestamp-ms": 4353, + "version-id": 1 + }, + { + "timestamp-ms": 5555, + "version-id": 2 + } + ] +} \ No newline at end of file diff --git a/core/src/test/resources/org/apache/iceberg/view/ViewMetadataMissingCurrentVersion.json b/core/src/test/resources/org/apache/iceberg/view/ViewMetadataMissingCurrentVersion.json new file mode 100644 index 000000000000..f09a7a4aa6b5 --- /dev/null +++ b/core/src/test/resources/org/apache/iceberg/view/ViewMetadataMissingCurrentVersion.json @@ -0,0 +1,75 @@ +{ + "format-version": 1, + "location": "s3://bucket/test/location", + "properties": {"some-key": "some-value"}, + "current-schema-id": 1, + "schemas": [ + { + "type": "struct", + "schema-id": 1, + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + }, + { + "id": 2, + "name": "y", + "required": true, + "type": "long", + "doc": "comment" + }, + { + "id": 3, + "name": "z", + "required": true, + "type": "long" + } + ] + } + ], + "versions": [ + { + "version-id": 1, + "timestamp-ms": 4353, + "summary": {"operation":"create"}, + "schema-id": 1, + "default-catalog": "some-catalog", + "default-namespace": [], + "representations": [ + { + "type": "sql", + "sql": "select 'foo' foo", + "dialect": "spark-sql" + } + ] + }, + { + "version-id": 2, + "timestamp-ms": 5555, + "summary": {"operation": "replace"}, + "schema-id": 1, + "default-catalog": "some-catalog", + "default-namespace": [], + "representations": [ + { + "type": "sql", + "sql": "select 1 id, 'abc' data", + "dialect": "spark-sql" + } + ] + } + ], + "version-log": [ + { + "timestamp-ms": 4353, + "version-id": 1 + }, + { + "timestamp-ms": 5555, + "version-id": 2 + } + ] +} \ No newline at end of file diff --git a/core/src/test/resources/org/apache/iceberg/view/ViewMetadataMissingLocation.json b/core/src/test/resources/org/apache/iceberg/view/ViewMetadataMissingLocation.json new file mode 100644 index 000000000000..d0fa7d9392a9 --- /dev/null +++ b/core/src/test/resources/org/apache/iceberg/view/ViewMetadataMissingLocation.json @@ -0,0 +1,75 @@ +{ + "format-version": 1, + "properties": {"some-key": "some-value"}, + "current-schema-id": 1, + "current-version-id": 2, + "schemas": [ + { + "type": "struct", + "schema-id": 1, + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + }, + { + "id": 2, + "name": "y", + "required": true, + "type": "long", + "doc": "comment" + }, + { + "id": 3, + "name": "z", + "required": true, + "type": "long" + } + ] + } + ], + "versions": [ + { + "version-id": 1, + "timestamp-ms": 4353, + "summary": {"operation":"create"}, + "schema-id": 1, + "default-catalog": "some-catalog", + "default-namespace": [], + "representations": [ + { + "type": "sql", + "sql": "select 'foo' foo", + "dialect": "spark-sql" + } + ] + }, + { + "version-id": 2, + "timestamp-ms": 5555, + "summary": {"operation": "replace"}, + "schema-id": 1, + "default-catalog": "some-catalog", + "default-namespace": [], + "representations": [ + { + "type": "sql", + "sql": "select 1 id, 'abc' data", + "dialect": "spark-sql" + } + ] + } + ], + "version-log": [ + { + "timestamp-ms": 4353, + "version-id": 1 + }, + { + "timestamp-ms": 5555, + "version-id": 2 + } + ] +} \ No newline at end of file diff --git a/data/src/main/java/org/apache/iceberg/data/TableMigrationUtil.java b/data/src/main/java/org/apache/iceberg/data/TableMigrationUtil.java index aa1885a31e8c..0fb290f947f8 100644 --- a/data/src/main/java/org/apache/iceberg/data/TableMigrationUtil.java +++ b/data/src/main/java/org/apache/iceberg/data/TableMigrationUtil.java @@ -58,12 +58,12 @@ private TableMigrationUtil() {} * Returns the data files in a partition by listing the partition location. * *

For Parquet and ORC partitions, this will read metrics from the file footer. For Avro - * partitions, metrics are set to null. + * partitions, metrics other than row count are set to null. * - *

Note: certain metrics, like NaN counts, that are only supported by iceberg file writers but + *

Note: certain metrics, like NaN counts, that are only supported by Iceberg file writers but * not file footers, will not be populated. * - * @param partition partition key, e.g., "a=1/b=2" + * @param partition map of column names to column values for the partition * @param uri partition location URI * @param format partition format, avro, parquet or orc * @param spec a partition spec @@ -83,8 +83,28 @@ public static List listPartition( return listPartition(partition, uri, format, spec, conf, metricsConfig, mapping, 1); } + /** + * Returns the data files in a partition by listing the partition location. Metrics are read from + * the files and the file reading is done in parallel by a specified number of threads. + * + *

For Parquet and ORC partitions, this will read metrics from the file footer. For Avro + * partitions, metrics other than row count are set to null. + * + *

Note: certain metrics, like NaN counts, that are only supported by Iceberg file writers but + * not file footers, will not be populated. + * + * @param partition map of column names to column values for the partition + * @param partitionUri partition location URI + * @param format partition format, avro, parquet or orc + * @param spec a partition spec + * @param conf a Hadoop conf + * @param metricsSpec a metrics conf + * @param mapping a name mapping + * @param parallelism number of threads to use for file reading + * @return a List of DataFile + */ public static List listPartition( - Map partitionPath, + Map partition, String partitionUri, String format, PartitionSpec spec, @@ -94,16 +114,16 @@ public static List listPartition( int parallelism) { ExecutorService service = null; try { - String partitionKey = + List partitionValues = spec.fields().stream() .map(PartitionField::name) - .map(name -> String.format("%s=%s", name, partitionPath.get(name))) - .collect(Collectors.joining("/")); + .map(partition::get) + .collect(Collectors.toList()); - Path partition = new Path(partitionUri); - FileSystem fs = partition.getFileSystem(conf); + Path partitionDir = new Path(partitionUri); + FileSystem fs = partitionDir.getFileSystem(conf); List fileStatus = - Arrays.stream(fs.listStatus(partition, HIDDEN_PATH_FILTER)) + Arrays.stream(fs.listStatus(partitionDir, HIDDEN_PATH_FILTER)) .filter(FileStatus::isFile) .collect(Collectors.toList()); DataFile[] datafiles = new DataFile[fileStatus.size()]; @@ -120,7 +140,7 @@ public static List listPartition( index -> { Metrics metrics = getAvroMetrics(fileStatus.get(index).getPath(), conf); datafiles[index] = - buildDataFile(fileStatus.get(index), partitionKey, spec, metrics, "avro"); + buildDataFile(fileStatus.get(index), partitionValues, spec, metrics, "avro"); }); } else if (format.contains("parquet")) { task.run( @@ -128,7 +148,7 @@ public static List listPartition( Metrics metrics = getParquetMetrics(fileStatus.get(index).getPath(), conf, metricsSpec, mapping); datafiles[index] = - buildDataFile(fileStatus.get(index), partitionKey, spec, metrics, "parquet"); + buildDataFile(fileStatus.get(index), partitionValues, spec, metrics, "parquet"); }); } else if (format.contains("orc")) { task.run( @@ -136,7 +156,7 @@ public static List listPartition( Metrics metrics = getOrcMetrics(fileStatus.get(index).getPath(), conf, metricsSpec, mapping); datafiles[index] = - buildDataFile(fileStatus.get(index), partitionKey, spec, metrics, "orc"); + buildDataFile(fileStatus.get(index), partitionValues, spec, metrics, "orc"); }); } else { throw new UnsupportedOperationException("Unknown partition format: " + format); @@ -181,13 +201,17 @@ private static Metrics getOrcMetrics( } private static DataFile buildDataFile( - FileStatus stat, String partitionKey, PartitionSpec spec, Metrics metrics, String format) { + FileStatus stat, + List partitionValues, + PartitionSpec spec, + Metrics metrics, + String format) { return DataFiles.builder(spec) .withPath(stat.getPath().toString()) .withFormat(format) .withFileSizeInBytes(stat.getLen()) .withMetrics(metrics) - .withPartitionPath(partitionKey) + .withPartitionValues(partitionValues) .build(); } diff --git a/data/src/test/java/org/apache/iceberg/data/avro/TestSingleMessageEncoding.java b/data/src/test/java/org/apache/iceberg/data/avro/TestSingleMessageEncoding.java index ce85e8a90519..1b8da1eafc67 100644 --- a/data/src/test/java/org/apache/iceberg/data/avro/TestSingleMessageEncoding.java +++ b/data/src/test/java/org/apache/iceberg/data/avro/TestSingleMessageEncoding.java @@ -208,7 +208,7 @@ public void testByteBufferMissingPayload() throws Exception { Assertions.assertThatThrownBy(() -> decoder.decode(buffer)) .isInstanceOf(AvroRuntimeException.class) - .hasMessage("Decoding datum failed"); + .hasMessageContaining("Decoding datum failed"); } @Test diff --git a/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsAppendOutputStream.java b/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsAppendOutputStream.java index 18b5d621b80e..bf5ed46a0782 100644 --- a/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsAppendOutputStream.java +++ b/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsAppendOutputStream.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.dell.ecs; +import static org.assertj.core.api.Assertions.assertThat; + import com.emc.object.Range; import java.io.IOException; import java.io.InputStream; @@ -25,7 +27,6 @@ import org.apache.iceberg.dell.mock.ecs.EcsS3MockRule; import org.apache.iceberg.metrics.MetricsContext; import org.apache.iceberg.relocated.com.google.common.io.ByteStreams; -import org.junit.Assert; import org.junit.ClassRule; import org.junit.Test; @@ -54,10 +55,9 @@ public void testBaseObjectWrite() throws IOException { try (InputStream input = rule.client().readObjectStream(rule.bucket(), objectName, Range.fromOffset(0))) { - Assert.assertEquals( - "Must write all the object content", - "1" + "123" + "1234567" + "12345678901", - new String(ByteStreams.toByteArray(input), StandardCharsets.UTF_8)); + assertThat(new String(ByteStreams.toByteArray(input), StandardCharsets.UTF_8)) + .as("Must write all the object content") + .isEqualTo("1" + "123" + "1234567" + "12345678901"); } } @@ -87,10 +87,9 @@ public void testRewrite() throws IOException { try (InputStream input = rule.client().readObjectStream(rule.bucket(), objectName, Range.fromOffset(0))) { - Assert.assertEquals( - "Must replace the object content", - "1234567" + "1234567", - new String(ByteStreams.toByteArray(input), StandardCharsets.UTF_8)); + assertThat(new String(ByteStreams.toByteArray(input), StandardCharsets.UTF_8)) + .as("Must replace the object content") + .isEqualTo("1234567" + "1234567"); } } } diff --git a/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsCatalog.java b/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsCatalog.java index c61bd1db172d..6c6136fade07 100644 --- a/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsCatalog.java +++ b/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsCatalog.java @@ -19,6 +19,7 @@ package org.apache.iceberg.dell.ecs; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; import java.io.IOException; import java.util.Map; @@ -135,23 +136,27 @@ public void testDropNamespace() { .isInstanceOf(NamespaceNotEmptyException.class) .hasMessage("Namespace a is not empty"); - Assert.assertTrue("Drop namespace [a, b1]", ecsCatalog.dropNamespace(Namespace.of("a", "b1"))); + assertThat(ecsCatalog.dropNamespace(Namespace.of("a", "b1"))) + .as("Drop namespace [a, b1]") + .isTrue(); - Assert.assertFalse( - "The [a, b1] is absent", ecsCatalog.namespaceExists(Namespace.of("a", "b1"))); - Assert.assertTrue( - "The [a, b1] is not in list result of [a]", - ecsCatalog.listNamespaces(Namespace.of("a")).isEmpty()); + assertThat(ecsCatalog.namespaceExists(Namespace.of("a", "b1"))) + .as("The [a, b1] is absent") + .isFalse(); + assertThat(ecsCatalog.listNamespaces(Namespace.of("a"))) + .as("The [a, b1] is not in list result of [a]") + .isEmpty(); } @Test public void testDropTable() { ecsCatalog.createTable(TableIdentifier.of("a"), SCHEMA); - Assert.assertFalse( - "Drop an unknown table return false", ecsCatalog.dropTable(TableIdentifier.of("unknown"))); + assertThat(ecsCatalog.dropTable(TableIdentifier.of("unknown"))) + .as("Drop an unknown table return false") + .isFalse(); - Assert.assertTrue("Drop a table", ecsCatalog.dropTable(TableIdentifier.of("a"), true)); + assertThat(ecsCatalog.dropTable(TableIdentifier.of("a"), true)).as("Drop a table").isTrue(); } @Test @@ -176,9 +181,12 @@ public void testRenameTable() { ecsCatalog.renameTable(TableIdentifier.of("a", "t1"), TableIdentifier.of("b", "t2")); - Assert.assertFalse( - "Old table does not exist", ecsCatalog.tableExists(TableIdentifier.of("a", "t1"))); - Assert.assertTrue("New table exists", ecsCatalog.tableExists(TableIdentifier.of("b", "t2"))); + assertThat(ecsCatalog.tableExists(TableIdentifier.of("a", "t1"))) + .as("Old table does not exist") + .isFalse(); + assertThat(ecsCatalog.tableExists(TableIdentifier.of("b", "t2"))) + .as("New table exists") + .isTrue(); } @Test diff --git a/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsInputFile.java b/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsInputFile.java index e2bb907baccc..4f48c01f1318 100644 --- a/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsInputFile.java +++ b/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsInputFile.java @@ -18,13 +18,14 @@ */ package org.apache.iceberg.dell.ecs; +import static org.assertj.core.api.Assertions.assertThat; + import com.emc.object.s3.request.PutObjectRequest; import java.io.IOException; import java.io.InputStream; import java.nio.charset.StandardCharsets; import org.apache.iceberg.dell.mock.ecs.EcsS3MockRule; import org.apache.iceberg.relocated.com.google.common.io.ByteStreams; -import org.junit.Assert; import org.junit.ClassRule; import org.junit.Test; @@ -37,7 +38,7 @@ public void testAbsentFile() { String objectName = rule.randomObjectName(); EcsInputFile inputFile = EcsInputFile.fromLocation(new EcsURI(rule.bucket(), objectName).toString(), rule.client()); - Assert.assertFalse("File is absent", inputFile.exists()); + assertThat(inputFile.exists()).as("File is absent").isFalse(); } @Test @@ -49,13 +50,12 @@ public void testFileRead() throws IOException { rule.client() .putObject(new PutObjectRequest(rule.bucket(), objectName, "0123456789".getBytes())); - Assert.assertTrue("File should exists", inputFile.exists()); - Assert.assertEquals("File length should be 10", 10, inputFile.getLength()); + assertThat(inputFile.exists()).as("File should exists").isTrue(); + assertThat(inputFile.getLength()).as("File length should be 10").isEqualTo(10); try (InputStream inputStream = inputFile.newStream()) { - Assert.assertEquals( - "The file content should be 0123456789", - "0123456789", - new String(ByteStreams.toByteArray(inputStream), StandardCharsets.UTF_8)); + assertThat(new String(ByteStreams.toByteArray(inputStream), StandardCharsets.UTF_8)) + .as("The file content should be 0123456789") + .isEqualTo("0123456789"); } } } diff --git a/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsOutputFile.java b/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsOutputFile.java index 6d12a6aaa235..ab9ebfdcc6ca 100644 --- a/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsOutputFile.java +++ b/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsOutputFile.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.dell.ecs; +import static org.assertj.core.api.Assertions.assertThat; + import com.emc.object.Range; import java.io.IOException; import java.io.InputStream; @@ -27,7 +29,6 @@ import org.apache.iceberg.io.PositionOutputStream; import org.apache.iceberg.relocated.com.google.common.io.ByteStreams; import org.assertj.core.api.Assertions; -import org.junit.Assert; import org.junit.ClassRule; import org.junit.Test; @@ -48,10 +49,9 @@ public void testFileWrite() throws IOException { try (InputStream input = rule.client().readObjectStream(rule.bucket(), objectName, Range.fromOffset(0))) { - Assert.assertEquals( - "File content is expected", - "1234567890", - new String(ByteStreams.toByteArray(input), StandardCharsets.UTF_8)); + assertThat(new String(ByteStreams.toByteArray(input), StandardCharsets.UTF_8)) + .as("File content is expected") + .isEqualTo("1234567890"); } } @@ -71,10 +71,9 @@ public void testFileOverwrite() throws IOException { try (InputStream input = rule.client().readObjectStream(rule.bucket(), objectName, Range.fromOffset(0))) { - Assert.assertEquals( - "File content should be overwritten", - "abcdefghij", - new String(ByteStreams.toByteArray(input), StandardCharsets.UTF_8)); + assertThat(new String(ByteStreams.toByteArray(input), StandardCharsets.UTF_8)) + .as("File content should be overwritten") + .isEqualTo("abcdefghij"); } } diff --git a/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsSeekableInputStream.java b/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsSeekableInputStream.java index 6e747f2ace7b..a49793fec8b5 100644 --- a/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsSeekableInputStream.java +++ b/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsSeekableInputStream.java @@ -18,12 +18,13 @@ */ package org.apache.iceberg.dell.ecs; +import static org.assertj.core.api.Assertions.assertThat; + import com.emc.object.s3.request.PutObjectRequest; import java.io.IOException; import java.nio.charset.StandardCharsets; import org.apache.iceberg.dell.mock.ecs.EcsS3MockRule; import org.apache.iceberg.metrics.MetricsContext; -import org.junit.Assert; import org.junit.ClassRule; import org.junit.Test; @@ -41,7 +42,7 @@ public void testSeekPosRead() throws IOException { new EcsSeekableInputStream( rule.client(), new EcsURI(rule.bucket(), objectName), MetricsContext.nullMetrics())) { input.seek(2); - Assert.assertEquals("Expect 2 when seek to 2", '2', input.read()); + assertThat(input.read()).as("Expect 2 when seek to 2").isEqualTo('2'); } } @@ -56,7 +57,7 @@ public void testMultipleSeekPosRead() throws IOException { rule.client(), new EcsURI(rule.bucket(), objectName), MetricsContext.nullMetrics())) { input.seek(999); input.seek(3); - Assert.assertEquals("Expect 3 when seek to 3 finally", '3', input.read()); + assertThat(input.read()).as("Expect 3 when seek to 3 finally").isEqualTo('3'); } } @@ -69,7 +70,7 @@ public void testReadOneByte() throws IOException { try (EcsSeekableInputStream input = new EcsSeekableInputStream( rule.client(), new EcsURI(rule.bucket(), objectName), MetricsContext.nullMetrics())) { - Assert.assertEquals("The first byte should be 0 ", '0', input.read()); + assertThat(input.read()).as("The first byte should be 0 ").isEqualTo('0'); } } @@ -83,9 +84,10 @@ public void testReadBytes() throws IOException { new EcsSeekableInputStream( rule.client(), new EcsURI(rule.bucket(), objectName), MetricsContext.nullMetrics())) { byte[] buffer = new byte[3]; - Assert.assertEquals("The first read should be 3 bytes", 3, input.read(buffer)); - Assert.assertEquals( - "The first 3 bytes should be 012", "012", new String(buffer, StandardCharsets.UTF_8)); + assertThat(input.read(buffer)).as("The first read should be 3 bytes").isEqualTo(3); + assertThat(new String(buffer, StandardCharsets.UTF_8)) + .as("The first 3 bytes should be 012") + .isEqualTo("012"); } } } diff --git a/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsURI.java b/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsURI.java index 9952ffd82bd8..0e80ac03d505 100644 --- a/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsURI.java +++ b/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsURI.java @@ -18,10 +18,11 @@ */ package org.apache.iceberg.dell.ecs; +import static org.assertj.core.api.Assertions.assertThat; + import org.apache.iceberg.exceptions.ValidationException; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestEcsURI { @@ -48,13 +49,12 @@ public void testConstructorWithBucketAndName() { } private void assertURI(String bucket, String name, EcsURI ecsURI) { - Assert.assertEquals("bucket", bucket, ecsURI.bucket()); - Assert.assertEquals("name", name, ecsURI.name()); + assertThat(ecsURI.bucket()).as("bucket").isEqualTo(bucket); + assertThat(ecsURI.name()).as("name").isEqualTo(name); } @Test public void testInvalidLocation() { - Assertions.assertThatThrownBy(() -> new EcsURI("http://bucket/a")) .isInstanceOf(ValidationException.class) .hasMessage("Invalid ecs location: http://bucket/a"); diff --git a/dell/src/test/java/org/apache/iceberg/dell/ecs/TestPropertiesSerDesUtil.java b/dell/src/test/java/org/apache/iceberg/dell/ecs/TestPropertiesSerDesUtil.java index 90587bf695cf..dbc93fdf25ba 100644 --- a/dell/src/test/java/org/apache/iceberg/dell/ecs/TestPropertiesSerDesUtil.java +++ b/dell/src/test/java/org/apache/iceberg/dell/ecs/TestPropertiesSerDesUtil.java @@ -18,10 +18,11 @@ */ package org.apache.iceberg.dell.ecs; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.Map; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestPropertiesSerDesUtil { @@ -31,6 +32,6 @@ public void testPropertiesSerDes() { byte[] byteValue = PropertiesSerDesUtil.toBytes(properties); Map result = PropertiesSerDesUtil.read(byteValue, PropertiesSerDesUtil.currentVersion()); - Assert.assertEquals("Ser/Des will return the same content.", properties, result); + assertThat(properties).as("Ser/Des will return the same content.").isEqualTo(result); } } diff --git a/dell/src/test/java/org/apache/iceberg/dell/mock/ecs/EcsS3MockRule.java b/dell/src/test/java/org/apache/iceberg/dell/mock/ecs/EcsS3MockRule.java index f02a0d13a6b0..7948531846d2 100644 --- a/dell/src/test/java/org/apache/iceberg/dell/mock/ecs/EcsS3MockRule.java +++ b/dell/src/test/java/org/apache/iceberg/dell/mock/ecs/EcsS3MockRule.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.dell.mock.ecs; +import static org.assertj.core.api.Assertions.assertThat; + import com.emc.object.s3.S3Client; import com.emc.object.s3.bean.ListObjectsResult; import com.emc.object.s3.bean.ObjectKey; @@ -31,7 +33,6 @@ import org.apache.iceberg.dell.DellClientFactories; import org.apache.iceberg.dell.DellProperties; import org.apache.iceberg.dell.mock.MockDellClientFactory; -import org.junit.Assert; import org.junit.rules.TestRule; import org.junit.runner.Description; import org.junit.runners.model.Statement; @@ -71,7 +72,7 @@ public static EcsS3MockRule manualCreateBucket() { /** Load rule from thread local and check bucket */ public static EcsS3MockRule rule(String id) { EcsS3MockRule rule = TEST_RULE_FOR_MOCK_CLIENT.get(); - Assert.assertTrue("Test Rule must match id", rule != null && rule.bucket().equals(id)); + assertThat(rule).isNotNull().extracting(EcsS3MockRule::bucket).isEqualTo(id); return rule; } diff --git a/dell/src/test/java/org/apache/iceberg/dell/mock/ecs/MockS3Client.java b/dell/src/test/java/org/apache/iceberg/dell/mock/ecs/MockS3Client.java index fc401c7c4257..e0b3b19c046b 100644 --- a/dell/src/test/java/org/apache/iceberg/dell/mock/ecs/MockS3Client.java +++ b/dell/src/test/java/org/apache/iceberg/dell/mock/ecs/MockS3Client.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.dell.mock.ecs; +import static org.assertj.core.api.Assertions.assertThat; + import com.emc.object.Protocol; import com.emc.object.Range; import com.emc.object.s3.S3Client; @@ -91,7 +93,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.relocated.com.google.common.io.ByteStreams; -import org.junit.Assert; /** Memorized s3 client used in tests. */ public class MockS3Client implements S3Client { @@ -121,7 +122,7 @@ public PutObjectResult putObject(PutObjectRequest request) { } } else if (request.getIfNoneMatch() != null) { // Put if absent - Assert.assertEquals("If-None-Match only allow *", "*", request.getIfNoneMatch()); + assertThat(request.getIfNoneMatch()).as("If-None-Match only allow *").isEqualTo("*"); if (this.objectData.putIfAbsent(objectId, data) != null) { throw new S3Exception("", 412, "PreconditionFailed", ""); } @@ -212,7 +213,7 @@ public ListObjectsResult listObjects(ListObjectsRequest request) { String marker = request.getMarker(); String delimiter = request.getDelimiter(); // Use a small default value in mock client - Assert.assertNull("MaxKeys does not set", request.getMaxKeys()); + assertThat(request.getMaxKeys()).as("MaxKeys does not set").isNull(); int maxKeys = 5; List objectResults = Lists.newArrayListWithCapacity(maxKeys); diff --git a/dell/src/test/java/org/apache/iceberg/dell/mock/ecs/ObjectData.java b/dell/src/test/java/org/apache/iceberg/dell/mock/ecs/ObjectData.java index 6f16a2f73c0a..08ffacb84e36 100644 --- a/dell/src/test/java/org/apache/iceberg/dell/mock/ecs/ObjectData.java +++ b/dell/src/test/java/org/apache/iceberg/dell/mock/ecs/ObjectData.java @@ -20,9 +20,10 @@ import com.emc.object.Range; import com.emc.object.s3.S3ObjectMetadata; -import com.emc.object.shadow.org.apache.commons.codec.digest.DigestUtils; import java.io.ByteArrayInputStream; import java.io.InputStream; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; import java.util.Arrays; import java.util.LinkedHashMap; import java.util.Map; @@ -31,6 +32,12 @@ public class ObjectData { public final byte[] content; public final Map userMetadata; + private static final char[] HEX_ARRAY = "0123456789ABCDEF".toCharArray(); + + private ObjectData(byte[] content, Map userMetadata) { + this.content = content; + this.userMetadata = userMetadata; + } public static ObjectData create(byte[] content, S3ObjectMetadata metadata) { Map userMetadata = new LinkedHashMap<>(); @@ -41,11 +48,6 @@ public static ObjectData create(byte[] content, S3ObjectMetadata metadata) { return new ObjectData(content, userMetadata); } - private ObjectData(byte[] content, Map userMetadata) { - this.content = content; - this.userMetadata = userMetadata; - } - public int length() { return content.length; } @@ -70,9 +72,27 @@ public InputStream createInputStream(Range range) { public S3ObjectMetadata createFullMetadata() { S3ObjectMetadata metadata = new S3ObjectMetadata(); - metadata.setETag(DigestUtils.md5Hex(content)); + MessageDigest md = null; + try { + md = MessageDigest.getInstance("MD5"); + } catch (NoSuchAlgorithmException e) { + throw new RuntimeException(e); + } + md.update(content); + byte[] digest = md.digest(); + metadata.setETag(bytesToHex(digest)); metadata.setContentLength((long) content.length); metadata.setUserMetadata(userMetadata); return metadata; } + + private static String bytesToHex(byte[] bytes) { + char[] hexChars = new char[bytes.length * 2]; + for (int j = 0; j < bytes.length; j++) { + int accum = bytes[j] & 0xFF; + hexChars[j * 2] = HEX_ARRAY[accum >>> 4]; + hexChars[j * 2 + 1] = HEX_ARRAY[accum & 0x0F]; + } + return new String(hexChars); + } } diff --git a/delta-lake/src/integration/java/org/apache/iceberg/delta/SparkDeltaLakeSnapshotTestBase.java b/delta-lake/src/integration/java/org/apache/iceberg/delta/SparkDeltaLakeSnapshotTestBase.java index 7592b8e862d0..930481b12a48 100644 --- a/delta-lake/src/integration/java/org/apache/iceberg/delta/SparkDeltaLakeSnapshotTestBase.java +++ b/delta-lake/src/integration/java/org/apache/iceberg/delta/SparkDeltaLakeSnapshotTestBase.java @@ -23,8 +23,8 @@ import org.apache.iceberg.hive.TestHiveMetastore; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.internal.SQLConf; -import org.junit.AfterClass; -import org.junit.BeforeClass; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; @SuppressWarnings("VisibilityModifier") public abstract class SparkDeltaLakeSnapshotTestBase { @@ -32,7 +32,7 @@ public abstract class SparkDeltaLakeSnapshotTestBase { protected static HiveConf hiveConf = null; protected static SparkSession spark = null; - @BeforeClass + @BeforeAll public static void startMetastoreAndSpark() { SparkDeltaLakeSnapshotTestBase.metastore = new TestHiveMetastore(); metastore.start(); @@ -52,7 +52,7 @@ public static void startMetastoreAndSpark() { .getOrCreate(); } - @AfterClass + @AfterAll public static void stopMetastoreAndSpark() throws Exception { if (metastore != null) { metastore.stop(); diff --git a/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java index bace582c0658..52966a484e68 100644 --- a/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java +++ b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java @@ -34,6 +34,7 @@ import java.io.IOException; import java.net.URI; import java.nio.file.Files; +import java.nio.file.Path; import java.nio.file.Paths; import java.sql.Timestamp; import java.util.Iterator; @@ -41,6 +42,7 @@ import java.util.Map; import java.util.Random; import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.commons.codec.DecoderException; import org.apache.commons.codec.net.URLCodec; import org.apache.iceberg.Snapshot; @@ -58,15 +60,13 @@ import org.apache.spark.sql.connector.catalog.CatalogPlugin; import org.apache.spark.sql.delta.catalog.DeltaCatalog; import org.assertj.core.api.Assertions; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + public class TestSnapshotDeltaLakeTable extends SparkDeltaLakeSnapshotTestBase { private static final String SNAPSHOT_SOURCE_PROP = "snapshot_source"; private static final String DELTA_SOURCE_VALUE = "delta"; @@ -77,27 +77,24 @@ public class TestSnapshotDeltaLakeTable extends SparkDeltaLakeSnapshotTestBase { private static Dataset typeTestDataFrame; private static Dataset nestedDataFrame; - @Parameterized.Parameters(name = "Catalog Name {0} - Options {2}") - public static Object[][] parameters() { - return new Object[][] { - new Object[] { - icebergCatalogName, - SparkCatalog.class.getName(), - ImmutableMap.of( - "type", - "hive", - "default-namespace", - "default", - "parquet-enabled", - "true", - "cache-enabled", - "false" // Spark will delete tables using v1, leaving the cache out of sync - ) - } - }; + static Stream parameters() { + return Stream.of( + Arguments.of( + icebergCatalogName, + SparkCatalog.class.getName(), + ImmutableMap.of( + "type", + "hive", + "default-namespace", + "default", + "parquet-enabled", + "true", + "cache-enabled", + "false" // Spark will delete tables using v1, leaving the cache out of sync + ))); } - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; public TestSnapshotDeltaLakeTable( String catalogName, String implementation, Map config) { @@ -105,7 +102,7 @@ public TestSnapshotDeltaLakeTable( spark.conf().set("spark.sql.catalog." + defaultSparkCatalog, DeltaCatalog.class.getName()); } - @BeforeClass + @BeforeAll public static void beforeClass() { spark.sql(String.format("CREATE DATABASE IF NOT EXISTS %s", NAMESPACE)); @@ -158,15 +155,16 @@ public static void beforeClass() { .withColumn("structCol3", expr("STRUCT(structCol2, mapCol3, arrayCol)")); } - @AfterClass + @AfterAll public static void afterClass() { spark.sql(String.format("DROP DATABASE IF EXISTS %s CASCADE", NAMESPACE)); } - @Test - public void testBasicSnapshotPartitioned() throws IOException { + @ParameterizedTest(name = "Catalog Name {0} - Options {2}") + @MethodSource("parameters") + public void testBasicSnapshotPartitioned() { String partitionedIdentifier = destName(defaultSparkCatalog, "partitioned_table"); - String partitionedLocation = temp.newFolder().toURI().toString(); + String partitionedLocation = temp.toFile().toURI().toString(); writeDeltaTable(nestedDataFrame, partitionedIdentifier, partitionedLocation, "id"); spark.sql("DELETE FROM " + partitionedIdentifier + " WHERE id=3"); @@ -184,10 +182,11 @@ public void testBasicSnapshotPartitioned() throws IOException { checkIcebergTableLocation(newTableIdentifier, partitionedLocation); } - @Test - public void testBasicSnapshotUnpartitioned() throws IOException { + @ParameterizedTest(name = "Catalog Name {0} - Options {2}") + @MethodSource("parameters") + public void testBasicSnapshotUnpartitioned() { String unpartitionedIdentifier = destName(defaultSparkCatalog, "unpartitioned_table"); - String unpartitionedLocation = temp.newFolder().toURI().toString(); + String unpartitionedLocation = temp.toFile().toURI().toString(); writeDeltaTable(nestedDataFrame, unpartitionedIdentifier, unpartitionedLocation, null); spark.sql("DELETE FROM " + unpartitionedIdentifier + " WHERE id=3"); @@ -205,11 +204,12 @@ public void testBasicSnapshotUnpartitioned() throws IOException { checkIcebergTableLocation(newTableIdentifier, unpartitionedLocation); } - @Test - public void testSnapshotWithNewLocation() throws IOException { + @ParameterizedTest(name = "Catalog Name {0} - Options {2}") + @MethodSource("parameters") + public void testSnapshotWithNewLocation() { String partitionedIdentifier = destName(defaultSparkCatalog, "partitioned_table"); - String partitionedLocation = temp.newFolder().toURI().toString(); - String newIcebergTableLocation = temp.newFolder().toURI().toString(); + String partitionedLocation = temp.toFile().toURI().toString(); + String newIcebergTableLocation = temp.toFile().toURI().toString(); writeDeltaTable(nestedDataFrame, partitionedIdentifier, partitionedLocation, "id"); spark.sql("DELETE FROM " + partitionedIdentifier + " WHERE id=3"); @@ -228,10 +228,11 @@ public void testSnapshotWithNewLocation() throws IOException { checkIcebergTableLocation(newTableIdentifier, newIcebergTableLocation); } - @Test - public void testSnapshotWithAdditionalProperties() throws IOException { + @ParameterizedTest(name = "Catalog Name {0} - Options {2}") + @MethodSource("parameters") + public void testSnapshotWithAdditionalProperties() { String unpartitionedIdentifier = destName(defaultSparkCatalog, "unpartitioned_table"); - String unpartitionedLocation = temp.newFolder().toURI().toString(); + String unpartitionedLocation = temp.toFile().toURI().toString(); writeDeltaTable(nestedDataFrame, unpartitionedIdentifier, unpartitionedLocation, null); spark.sql("DELETE FROM " + unpartitionedIdentifier + " WHERE id=3"); @@ -266,12 +267,13 @@ public void testSnapshotWithAdditionalProperties() throws IOException { unpartitionedLocation); } - @Test - public void testSnapshotTableWithExternalDataFiles() throws IOException { + @ParameterizedTest(name = "Catalog Name {0} - Options {2}") + @MethodSource("parameters") + public void testSnapshotTableWithExternalDataFiles() { String unpartitionedIdentifier = destName(defaultSparkCatalog, "unpartitioned_table"); String externalDataFilesIdentifier = destName(defaultSparkCatalog, "external_data_files_table"); - String unpartitionedLocation = temp.newFolder().toURI().toString(); - String externalDataFilesTableLocation = temp.newFolder().toURI().toString(); + String unpartitionedLocation = temp.toFile().toURI().toString(); + String externalDataFilesTableLocation = temp.toFile().toURI().toString(); writeDeltaTable(nestedDataFrame, unpartitionedIdentifier, unpartitionedLocation, null); spark.sql("DELETE FROM " + unpartitionedIdentifier + " WHERE id=3"); @@ -295,10 +297,11 @@ public void testSnapshotTableWithExternalDataFiles() throws IOException { checkDataFilePathsIntegrity(newTableIdentifier, externalDataFilesTableLocation); } - @Test - public void testSnapshotSupportedTypes() throws IOException { + @ParameterizedTest(name = "Catalog Name {0} - Options {2}") + @MethodSource("parameters") + public void testSnapshotSupportedTypes() { String typeTestIdentifier = destName(defaultSparkCatalog, "type_test_table"); - String typeTestTableLocation = temp.newFolder().toURI().toString(); + String typeTestTableLocation = temp.toFile().toURI().toString(); writeDeltaTable(typeTestDataFrame, typeTestIdentifier, typeTestTableLocation, "stringCol"); String newTableIdentifier = destName(icebergCatalogName, "iceberg_type_test_table"); @@ -313,10 +316,11 @@ public void testSnapshotSupportedTypes() throws IOException { checkIcebergTableProperties(newTableIdentifier, ImmutableMap.of(), typeTestTableLocation); } - @Test + @ParameterizedTest(name = "Catalog Name {0} - Options {2}") + @MethodSource("parameters") public void testSnapshotVacuumTable() throws IOException { String vacuumTestIdentifier = destName(defaultSparkCatalog, "vacuum_test_table"); - String vacuumTestTableLocation = temp.newFolder().toURI().toString(); + String vacuumTestTableLocation = temp.toFile().toURI().toString(); writeDeltaTable(nestedDataFrame, vacuumTestIdentifier, vacuumTestTableLocation, null); Random random = new Random(); @@ -348,10 +352,11 @@ public void testSnapshotVacuumTable() throws IOException { checkIcebergTableLocation(newTableIdentifier, vacuumTestTableLocation); } - @Test + @ParameterizedTest(name = "Catalog Name {0} - Options {2}") + @MethodSource("parameters") public void testSnapshotLogCleanTable() throws IOException { String logCleanTestIdentifier = destName(defaultSparkCatalog, "log_clean_test_table"); - String logCleanTestTableLocation = temp.newFolder().toURI().toString(); + String logCleanTestTableLocation = temp.toFile().toURI().toString(); writeDeltaTable(nestedDataFrame, logCleanTestIdentifier, logCleanTestTableLocation, "id"); Random random = new Random(); diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java index 8cc04f4c3637..8bb0fbf7913b 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java @@ -25,14 +25,15 @@ import io.delta.standalone.actions.RemoveFile; import io.delta.standalone.exceptions.DeltaStandaloneException; import java.io.File; +import java.io.UnsupportedEncodingException; import java.net.URI; +import java.net.URLDecoder; +import java.nio.charset.StandardCharsets; import java.sql.Timestamp; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.stream.Collectors; -import org.apache.commons.codec.DecoderException; -import org.apache.commons.codec.net.URLCodec; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; @@ -451,13 +452,13 @@ private void tagCurrentSnapshot(long deltaVersion, Transaction transaction) { private static String getFullFilePath(String path, String tableRoot) { URI dataFileUri = URI.create(path); try { - String decodedPath = new URLCodec().decode(path); + String decodedPath = URLDecoder.decode(path, StandardCharsets.UTF_8.name()); if (dataFileUri.isAbsolute()) { return decodedPath; } else { return tableRoot + File.separator + decodedPath; } - } catch (DecoderException e) { + } catch (UnsupportedEncodingException e) { throw new IllegalArgumentException(String.format("Cannot decode path %s", path), e); } } diff --git a/delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseSnapshotDeltaLakeTableAction.java b/delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseSnapshotDeltaLakeTableAction.java index ff33173f0e50..22e27c1e51e1 100644 --- a/delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseSnapshotDeltaLakeTableAction.java +++ b/delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseSnapshotDeltaLakeTableAction.java @@ -28,23 +28,20 @@ import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.assertj.core.api.Assertions; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestBaseSnapshotDeltaLakeTableAction { - @Rule public TemporaryFolder temp1 = new TemporaryFolder(); - @Rule public TemporaryFolder temp2 = new TemporaryFolder(); + @TempDir private File sourceFolder; + @TempDir private File destFolder; private String sourceTableLocation; private final Configuration testHadoopConf = new Configuration(); private String newTableLocation; private final Catalog testCatalog = new TestCatalog(); - @Before + @BeforeEach public void before() throws IOException { - File sourceFolder = temp1.newFolder(); - File destFolder = temp2.newFolder(); sourceTableLocation = sourceFolder.toURI().toString(); newTableLocation = destFolder.toURI().toString(); } diff --git a/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java b/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java index 29a5c63c3d22..20989ee8c013 100644 --- a/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java +++ b/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java @@ -32,8 +32,8 @@ import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.assertj.core.api.Assertions; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; public class TestDeltaLakeTypeToType { private static final String optionalBooleanType = "testNullableBoolType"; @@ -48,7 +48,7 @@ public class TestDeltaLakeTypeToType { private StructType deltaShallowNullTypeSchema; private StructType deltaNullTypeSchema; - @Before + @BeforeEach public void constructDeltaLakeSchema() { deltaAtomicSchema = new StructType() diff --git a/docs/aws.md b/docs/aws.md index e1eb5699c42b..ecabffe77e05 100644 --- a/docs/aws.md +++ b/docs/aws.md @@ -187,8 +187,8 @@ For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/ #### Skip Name Validation Allow user to skip name validation for table name and namespaces. -It is recommended to stick to Glue best practice in -https://docs.aws.amazon.com/athena/latest/ug/glue-best-practices.html to make sure operations are Hive compatible. +It is recommended to stick to [Glue best practices](https://docs.aws.amazon.com/athena/latest/ug/glue-best-practices.html) +to make sure operations are Hive compatible. This is only added for users that have existing conventions using non-standard characters. When database name and table name validation are skipped, there is no guarantee that downstream systems would all support the names. diff --git a/docs/branching-and-tagging.md b/docs/branching-and-tagging.md index 575c36678487..dd70d6c45aab 100644 --- a/docs/branching-and-tagging.md +++ b/docs/branching-and-tagging.md @@ -33,7 +33,7 @@ menu: Iceberg table metadata maintains a snapshot log, which represents the changes applied to a table. Snapshots are fundamental in Iceberg as they are the basis for reader isolation and time travel queries. -For controlling metadata size and storage costs, Iceberg provides snapshot lifecycle management procedures such as [`expire_snapshots`](../../spark/spark-procedures/#expire-snapshots) for removing unused snapshots and no longer neccessary data files based on table snapshot retention properties. +For controlling metadata size and storage costs, Iceberg provides snapshot lifecycle management procedures such as [`expire_snapshots`](../spark-procedures/#expire-snapshots) for removing unused snapshots and no longer necessary data files based on table snapshot retention properties. **For more sophisticated snapshot lifecycle management, Iceberg supports branches and tags which are named references to snapshots with their own independent lifecycles. This lifecycle is controlled by branch and tag level retention policies.** Branches are independent lineages of snapshots and point to the head of the lineage. @@ -54,32 +54,32 @@ Tags can be used for retaining important historical snapshots for auditing purpo ![Historical Tags](../img/historical-snapshot-tag.png) -The above diagram demonstrates retaininig important historical snapshot with the following retention policy, defined +The above diagram demonstrates retaining important historical snapshot with the following retention policy, defined via Spark SQL. 1. Retain 1 snapshot per week for 1 month. This can be achieved by tagging the weekly snapshot and setting the tag retention to be a month. snapshots will be kept, and the branch reference itself will be retained for 1 week. ```sql -- Create a tag for the first end of week snapshot. Retain the snapshot for a week -ALTER TABLE prod.db.table CREATE TAG 'EOW-01' AS OF VERSION 7 RETAIN 7 DAYS +ALTER TABLE prod.db.table CREATE TAG `EOW-01` AS OF VERSION 7 RETAIN 7 DAYS ``` 2. Retain 1 snapshot per month for 6 months. This can be achieved by tagging the monthly snapshot and setting the tag retention to be 6 months. ```sql -- Create a tag for the first end of month snapshot. Retain the snapshot for 6 months -ALTER TABLE prod.db.table CREATE TAG 'EOM-01' AS OF VERSION 30 RETAIN 180 DAYS +ALTER TABLE prod.db.table CREATE TAG `EOM-01` AS OF VERSION 30 RETAIN 180 DAYS ``` 3. Retain 1 snapshot per year forever. This can be achieved by tagging the annual snapshot. The default retention for branches and tags is forever. ```sql -- Create a tag for the end of the year and retain it forever. -ALTER TABLE prod.db.table CREATE TAG 'EOY-2023' AS OF VERSION 365 +ALTER TABLE prod.db.table CREATE TAG `EOY-2023` AS OF VERSION 365 ``` 4. Create a temporary "test-branch" which is retained for 7 days and the latest 2 snapshots on the branch are retained. ```sql -- Create a branch "test-branch" which will be retained for 7 days along with the latest 2 snapshots -ALTER TABLE prod.db.table CREATE BRANCH test-branch RETAIN 7 DAYS WITH RETENTION 2 SNAPSHOTS +ALTER TABLE prod.db.table CREATE BRANCH `test-branch` RETAIN 7 DAYS WITH RETENTION 2 SNAPSHOTS ``` ### Audit Branch @@ -115,7 +115,7 @@ table.manageSnapshots().fastForward("main", "audit-branch").commit() Creating, querying and writing to branches and tags are supported in the Iceberg Java library, and in Spark and Flink engine integrations. -- [Iceberg Java Library](../../java-api-quickstart/#branching-and-tagging) +- [Iceberg Java Library](../java-api-quickstart/#branching-and-tagging) - [Spark DDLs](../spark-ddl/#branching-and-tagging-ddl) - [Spark Reads](../spark-queries/#time-travel) - [Spark Branch Writes](../spark-writes/#writing-to-branches) diff --git a/docs/flink-ddl.md b/docs/flink-ddl.md index 30ff6b2f3c36..9508a5f0903d 100644 --- a/docs/flink-ddl.md +++ b/docs/flink-ddl.md @@ -150,8 +150,8 @@ USE iceberg_db; ```sql CREATE TABLE `hive_catalog`.`default`.`sample` ( id BIGINT COMMENT 'unique id', - data STRING -); + data STRING NOT NULL +) WITH ('format-version'='2'); ``` Table create commands support the commonly used [Flink create clauses](https://nightlies.apache.org/flink/flink-docs-master/docs/dev/table/sql/create/) including: @@ -160,20 +160,35 @@ Table create commands support the commonly used [Flink create clauses](https://n * `COMMENT 'table document'` to set a table description. * `WITH ('key'='value', ...)` to set [table configuration](../configuration) which will be stored in Iceberg table properties. -Currently, it does not support computed column, primary key and watermark definition etc. +Currently, it does not support computed column and watermark definition etc. + +#### `PRIMARY KEY` + +Primary key constraint can be declared for a column or a set of columns, which must be unique and do not contain null. +It's required for [`UPSERT` mode](../flink/flink-writes.md#upsert). + +```sql +CREATE TABLE `hive_catalog`.`default`.`sample` ( + id BIGINT COMMENT 'unique id', + data STRING NOT NULL, + PRIMARY KEY(`id`) NOT ENFORCED +) WITH ('format-version'='2'); +``` -### `PARTITIONED BY` +#### `PARTITIONED BY` To create a partition table, use `PARTITIONED BY`: ```sql CREATE TABLE `hive_catalog`.`default`.`sample` ( - id BIGINT COMMENT 'unique id', - data STRING -) PARTITIONED BY (data); + id BIGINT COMMENT 'unique id', + data STRING NOT NULL +) +PARTITIONED BY (data) +WITH ('format-version'='2'); ``` -Iceberg support hidden partition but Flink don't support partitioning by a function on columns, so there is no way to support hidden partition in Flink DDL. +Iceberg supports hidden partitioning but Flink doesn't support partitioning by a function on columns. There is no way to support hidden partitions in the Flink DDL. ### `CREATE TABLE LIKE` @@ -181,8 +196,8 @@ To create a table with the same schema, partitioning, and table properties as an ```sql CREATE TABLE `hive_catalog`.`default`.`sample` ( - id BIGINT COMMENT 'unique id', - data STRING + id BIGINT COMMENT 'unique id', + data STRING ); CREATE TABLE `hive_catalog`.`default`.`sample_like` LIKE `hive_catalog`.`default`.`sample`; diff --git a/docs/flink-getting-started.md b/docs/flink-getting-started.md index 6a229b7d864f..1a9d9b4cd2ec 100644 --- a/docs/flink-getting-started.md +++ b/docs/flink-getting-started.md @@ -274,7 +274,7 @@ env.execute("Test Iceberg DataStream"); ### Branch Writes Writing to branches in Iceberg tables is also supported via the `toBranch` API in `FlinkSink` -For more information on branches please refer to [branches](../../tables/branching). +For more information on branches please refer to [branches](../tables/branching). ```java FlinkSink.forRowData(input) .tableLoader(tableLoader) diff --git a/docs/flink-queries.md b/docs/flink-queries.md index 24ff608a1463..fa17fdbd7931 100644 --- a/docs/flink-queries.md +++ b/docs/flink-queries.md @@ -195,7 +195,7 @@ IcebergSource source = IcebergSource.forRowData() .streaming(true) .streamingStartingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) .monitorInterval(Duration.ofSeconds(60)) - .build() + .build(); DataStream stream = env.fromSource( source, @@ -428,15 +428,15 @@ To show a table's current partitions: SELECT * FROM prod.db.table$partitions; ``` -| partition | record_count | file_count | spec_id | -| -------------- | ------------ | ---------- | ------- | -| {20211001, 11} | 1 | 1 | 0 | -| {20211002, 11} | 1 | 1 | 0 | -| {20211001, 10} | 1 | 1 | 0 | -| {20211002, 10} | 1 | 1 | 0 | +| partition | spec_id | record_count | file_count | total_data_file_size_in_bytes | position_delete_record_count | position_delete_file_count | equality_delete_record_count | equality_delete_file_count | last_updated_at(μs) | last_updated_snapshot_id | +| -------------- |---------|---------------|------------|--------------------------|------------------------------|----------------------------|------------------------------|----------------------------|---------------------|--------------------------| +| {20211001, 11} | 0 | 1 | 1 | 100 | 2 | 1 | 0 | 0 | 1633086034192000 | 9205185327307503337 | +| {20211002, 11} | 0 | 4 | 3 | 500 | 1 | 1 | 0 | 0 | 1633172537358000 | 867027598972211003 | +| {20211001, 10} | 0 | 7 | 4 | 700 | 0 | 0 | 0 | 0 | 1633082598716000 | 3280122546965981531 | +| {20211002, 10} | 0 | 3 | 2 | 400 | 0 | 0 | 1 | 1 | 1633169159489000 | 6941468797545315876 | Note: -For unpartitioned tables, the partitions table will contain only the record_count and file_count columns. +For unpartitioned tables, the partitions table will not contain the partition and spec_id fields. ### All Metadata Tables diff --git a/docs/flink-writes.md b/docs/flink-writes.md index 99442db44de4..690fcd845498 100644 --- a/docs/flink-writes.md +++ b/docs/flink-writes.md @@ -69,9 +69,9 @@ Iceberg supports `UPSERT` based on the primary key when writing data into v2 tab ```sql CREATE TABLE `hive_catalog`.`default`.`sample` ( - `id` INT UNIQUE COMMENT 'unique id', - `data` STRING NOT NULL, - PRIMARY KEY(`id`) NOT ENFORCED + `id` INT UNIQUE COMMENT 'unique id', + `data` STRING NOT NULL, + PRIMARY KEY(`id`) NOT ENFORCED ) with ('format-version'='2', 'write.upsert.enabled'='true'); ``` @@ -193,7 +193,7 @@ FlinkSink.builderFor( ### Branch Writes Writing to branches in Iceberg tables is also supported via the `toBranch` API in `FlinkSink` -For more information on branches please refer to [branches](../../tables/branching). +For more information on branches please refer to [branches](../tables/branching). ```java FlinkSink.forRowData(input) .tableLoader(tableLoader) diff --git a/docs/nessie.md b/docs/nessie.md index da3c7a84b3e0..bf89b664fc4f 100644 --- a/docs/nessie.md +++ b/docs/nessie.md @@ -110,8 +110,9 @@ There is nothing special above about the `nessie` name. A spark catalog can have settings for the `catalog-impl` and the required config to start Nessie correctly. Once you have a Nessie catalog you have access to your entire Nessie repo. You can then perform create/delete/merge operations on branches and perform commits on branches. Each Iceberg table in a Nessie Catalog is identified by an -arbitrary length namespace and table name (eg `data.base.name.table`). These namespaces are implicit and don't need to -be created separately. Any transaction on a Nessie enabled Iceberg table is a single commit in Nessie. Nessie commits +arbitrary length namespace and table name (eg `data.base.name.table`). These namespaces must be explicitly created +as mentioned [here](https://projectnessie.org/blog/namespace-enforcement/). +Any transaction on a Nessie enabled Iceberg table is a single commit in Nessie. Nessie commits can encompass an arbitrary number of actions on an arbitrary number of tables, however in Iceberg this will be limited to the set of single table transactions currently available. diff --git a/docs/spark-configuration.md b/docs/spark-configuration.md index 866dd765b320..3a91f211769f 100644 --- a/docs/spark-configuration.md +++ b/docs/spark-configuration.md @@ -189,8 +189,23 @@ df.write | write-format | Table write.format.default | File format to use for this write operation; parquet, avro, or orc | | target-file-size-bytes | As per table property | Overrides this table's write.target-file-size-bytes | | check-nullability | true | Sets the nullable check on fields | -| snapshot-property._custom-key_ | null | Adds an entry with custom-key and corresponding value in the snapshot summary | +| snapshot-property._custom-key_ | null | Adds an entry with custom-key and corresponding value in the snapshot summary (the `snapshot-property.` prefix is only required for DSv2) | | fanout-enabled | false | Overrides this table's write.spark.fanout.enabled | | check-ordering | true | Checks if input schema and table schema are same | | isolation-level | null | Desired isolation level for Dataframe overwrite operations. `null` => no checks (for idempotent writes), `serializable` => check for concurrent inserts or deletes in destination partitions, `snapshot` => checks for concurrent deletes in destination partitions. | | validate-from-snapshot-id | null | If isolation level is set, id of base snapshot from which to check concurrent write conflicts into a table. Should be the snapshot before any reads from the table. Can be obtained via [Table API](../../api#table-metadata) or [Snapshots table](../spark-queries#snapshots). If null, the table's oldest known snapshot is used. | + +CommitMetadata provides an interface to add custom metadata to a snapshot summary during a SQL execution, which can be beneficial for purposes such as auditing or change tracking. If properties start with `snapshot-property.`, then that prefix will be removed from each property. Here is an example: + +```java +import org.apache.iceberg.spark.CommitMetadata; + +Map properties = Maps.newHashMap(); +properties.put("property_key", "property_value"); +CommitMetadata.withCommitProperties(properties, + () -> { + spark.sql("DELETE FROM " + tableName + " where id = 1"); + return 0; + }, + RuntimeException.class); +``` \ No newline at end of file diff --git a/docs/spark-ddl.md b/docs/spark-ddl.md index 6c80b5b2bc9e..21f6a50025d3 100644 --- a/docs/spark-ddl.md +++ b/docs/spark-ddl.md @@ -306,12 +306,17 @@ ALTER TABLE prod.db.sample ALTER COLUMN col FIRST ALTER TABLE prod.db.sample ALTER COLUMN nested.col AFTER other_col ``` -Nullability can be changed using `SET NOT NULL` and `DROP NOT NULL`: +Nullability for a non-nullable column can be changed using `DROP NOT NULL`: ```sql ALTER TABLE prod.db.sample ALTER COLUMN id DROP NOT NULL ``` +{{< hint info >}} +It is not possible to change a nullable column to a non-nullable column with `SET NOT NULL` because Iceberg doesn't know whether there is existing data with null values. +{{< /hint >}} + + {{< hint info >}} `ALTER COLUMN` is not used to update `struct` types. Use `ADD COLUMN` and `DROP COLUMN` to add or remove struct fields. {{< /hint >}} @@ -477,11 +482,11 @@ the snapshot to create the branch at and an optional retention clause. ```sql -- CREATE audit-branch at snapshot 1234 with default retention. -ALTER TABLE prod.db.sample CREATE BRANCH audit-branch +ALTER TABLE prod.db.sample CREATE BRANCH `audit-branch` AS OF VERSION 1234 -- CREATE audit-branch at snapshot 1234, retain audit-branch for 31 days, and retain the latest 31 days. The latest 3 snapshot snapshots, and 2 days worth of snapshots -ALTER TABLE prod.db.sample CREATE BRANCH audit-branch +ALTER TABLE prod.db.sample CREATE BRANCH `audit-branch` AS OF VERSION 1234 RETAIN 30 DAYS WITH RETENTION 3 SNAPSHOTS 2 DAYS ``` @@ -494,10 +499,10 @@ the snapshot to create the branch at and an optional retention clause. ```sql -- CREATE historical-tag at snapshot 1234 with default retention. -ALTER TABLE prod.db.sample CREATE TAG historical-tag AS OF VERSION 1234 +ALTER TABLE prod.db.sample CREATE TAG `historical-tag` AS OF VERSION 1234 -- CREATE historical-tag at snapshot 1234 and retain it for 1 year. -ALTER TABLE prod.db.sample CREATE TAG historical-tag +ALTER TABLE prod.db.sample CREATE TAG `historical-tag` AS OF VERSION 1234 RETAIN 365 DAYS ``` @@ -508,7 +513,7 @@ the `REPLACE BRANCH` sql. Retention can also be updated in this statement. ```sql -- REPLACE audit-branch to reference snapshot 4567 and update the retention to 60 days -ALTER TABLE prod.db.sample REPLACE BRANCH audit-branch +ALTER TABLE prod.db.sample REPLACE BRANCH `audit-branch` AS OF VERSION 4567 RETAIN 60 DAYS ``` @@ -517,7 +522,7 @@ AS OF VERSION 4567 RETAIN 60 DAYS Branches can be removed via the `DROP BRANCH` sql ```sql -ALTER TABLE prod.db.sample DROP BRANCH audit-branch +ALTER TABLE prod.db.sample DROP BRANCH `audit-branch` ``` #### `ALTER TABLE ... DROP TAG` @@ -525,5 +530,5 @@ ALTER TABLE prod.db.sample DROP BRANCH audit-branch Tags can be removed via the `DROP TAG` sql ```sql -ALTER TABLE prod.db.sample DROP TAG historical-tag +ALTER TABLE prod.db.sample DROP TAG `historical-tag` ``` \ No newline at end of file diff --git a/docs/spark-procedures.md b/docs/spark-procedures.md index b1275adde719..df9896fb4028 100644 --- a/docs/spark-procedures.md +++ b/docs/spark-procedures.md @@ -295,6 +295,7 @@ for list of all the supported options for this action. | ------------|------|-------------| | `rewritten_data_files_count` | int | Number of data which were re-written by this command | | `added_data_files_count` | int | Number of new data files which were written by this command | +| `rewritten_bytes_count` | long | Number of bytes which were written by this command | #### Examples @@ -513,6 +514,10 @@ will then treat these files as if they are part of the set of files owned by Ic `expire_snapshot` calls will be able to physically delete the added files. This method should not be used if `migrate` or `snapshot` are possible. +{{< hint warning >}} +Keep in mind the `add_files` procedure will fetch the Parquet metadata from each file being added just once. If you're using tiered storage, (such as [Amazon S3 Intelligent-Tiering storage class](https://aws.amazon.com/s3/storage-classes/intelligent-tiering/)), the underlying, file will be retrieved from the archive, and will remain on a higher tier for a set period of time. +{{< /hint >}} + #### Usage | Argument Name | Required? | Type | Description | @@ -748,4 +753,4 @@ as an `UPDATE_AFTER` image, resulting in the following pre/post update images: | id | name | _change_type | |-----|--------|--------------| | 3 | Robert | UPDATE_BEFORE| -| 3 | Dan | UPDATE_AFTER | \ No newline at end of file +| 3 | Dan | UPDATE_AFTER | diff --git a/docs/spark-queries.md b/docs/spark-queries.md index 535e5d62d9a0..4629b2e7ea87 100644 --- a/docs/spark-queries.md +++ b/docs/spark-queries.md @@ -311,15 +311,15 @@ To show a table's current partitions: SELECT * FROM prod.db.table.partitions; ``` -| partition | record_count | file_count | spec_id | -| -- | -- | -- | -- | -| {20211001, 11}| 1| 1| 0| -| {20211002, 11}| 1| 1| 0| -| {20211001, 10}| 1| 1| 0| -| {20211002, 10}| 1| 1| 0| +| partition | spec_id | record_count | file_count | total_data_file_size_in_bytes | position_delete_record_count | position_delete_file_count | equality_delete_record_count | equality_delete_file_count | last_updated_at(μs) | last_updated_snapshot_id | +| -------------- |---------|---------------|------------|--------------------------|------------------------------|----------------------------|------------------------------|----------------------------|---------------------|--------------------------| +| {20211001, 11} | 0 | 1 | 1 | 100 | 2 | 1 | 0 | 0 | 1633086034192000 | 9205185327307503337 | +| {20211002, 11} | 0 | 4 | 3 | 500 | 1 | 1 | 0 | 0 | 1633172537358000 | 867027598972211003 | +| {20211001, 10} | 0 | 7 | 4 | 700 | 0 | 0 | 0 | 0 | 1633082598716000 | 3280122546965981531 | +| {20211002, 10} | 0 | 3 | 2 | 400 | 0 | 0 | 1 | 1 | 1633169159489000 | 6941468797545315876 | Note: -1. For unpartitioned tables, the partitions table will contain only the record_count and file_count columns. +1. For unpartitioned tables, the partitions table will not contain the partition and spec_id fields. 2. The partitions metadata table shows partitions with data files or delete files in the current snapshot. However, delete files are not applied, and so in some cases partitions may be shown even though all their data rows are marked deleted by delete files. diff --git a/docs/spark-writes.md b/docs/spark-writes.md index 5a312da7f1ee..29c1e2244787 100644 --- a/docs/spark-writes.md +++ b/docs/spark-writes.md @@ -210,7 +210,7 @@ Branch writes can also be performed as part of a write-audit-publish (WAP) workf Note WAP branch and branch identifier cannot both be specified. Also, the branch must exist before performing the write. The operation does **not** create the branch if it does not exist. -For more information on branches please refer to [branches](../../tables/branching) +For more information on branches please refer to [branches](../tables/branching) ```sql -- INSERT (1,' a') (2, 'b') into the audit branch. @@ -397,25 +397,26 @@ so you may want to understand the type conversion in Iceberg in prior to design This type conversion table describes how Spark types are converted to the Iceberg types. The conversion applies on both creating Iceberg table and writing to Iceberg table via Spark. -| Spark | Iceberg | Notes | -|-----------------|-------------------------|-------| -| boolean | boolean | | -| short | integer | | -| byte | integer | | -| integer | integer | | -| long | long | | -| float | float | | -| double | double | | -| date | date | | -| timestamp | timestamp with timezone | | -| char | string | | -| varchar | string | | -| string | string | | -| binary | binary | | -| decimal | decimal | | -| struct | struct | | -| array | list | | -| map | map | | +| Spark | Iceberg | Notes | +|-----------------|----------------------------|-------| +| boolean | boolean | | +| short | integer | | +| byte | integer | | +| integer | integer | | +| long | long | | +| float | float | | +| double | double | | +| date | date | | +| timestamp | timestamp with timezone | | +| timestamp_ntz | timestamp without timezone | | +| char | string | | +| varchar | string | | +| string | string | | +| binary | binary | | +| decimal | decimal | | +| struct | struct | | +| array | list | | +| map | map | | {{< hint info >}} The table is based on representing conversion during creating table. In fact, broader supports are applied on write. Here're some points on write: @@ -438,7 +439,7 @@ This type conversion table describes how Iceberg types are converted to the Spar | date | date | | | time | | Not supported | | timestamp with timezone | timestamp | | -| timestamp without timezone | | Not supported | +| timestamp without timezone | timestamp_ntz | | | string | string | | | uuid | string | | | fixed | binary | | diff --git a/flink/v1.15/build.gradle b/flink/v1.15/build.gradle index c516612c85fa..d064bb8c4d31 100644 --- a/flink/v1.15/build.gradle +++ b/flink/v1.15/build.gradle @@ -112,6 +112,8 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { exclude group: 'javax.transaction', module: 'transaction-api' exclude group: 'com.zaxxer', module: 'HikariCP' } + + testImplementation 'org.awaitility:awaitility' } } @@ -238,6 +240,7 @@ project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") { task integrationTest(type: Test) { description = "Test Flink Runtime Jar against Flink ${flinkMajorVersion}" group = "verification" + jvmArgs += project.property('extraJvmArgs') testClassesDirs = sourceSets.integration.output.classesDirs classpath = sourceSets.integration.runtimeClasspath + files(shadowJar.archiveFile.get().asFile.path) inputs.file(shadowJar.archiveFile.get().asFile.path) diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java index 3ecddaa05c9b..825816fdf416 100644 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java +++ b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java @@ -124,14 +124,7 @@ public FlinkCatalog( } @Override - public void open() throws CatalogException { - // Create the default database if it does not exist. - try { - createDatabase(getDefaultDatabase(), ImmutableMap.of(), true); - } catch (DatabaseAlreadyExistException e) { - // Ignore the exception if it's already exist. - } - } + public void open() throws CatalogException {} @Override public void close() throws CatalogException { diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java index 3f04d35e851c..8e1f420b722d 100644 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java +++ b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java @@ -105,18 +105,21 @@ public DynamicTableSource createDynamicTableSource(Context context) { @Override public DynamicTableSink createDynamicTableSink(Context context) { - ObjectPath objectPath = context.getObjectIdentifier().toObjectPath(); + ObjectIdentifier objectIdentifier = context.getObjectIdentifier(); CatalogTable catalogTable = context.getCatalogTable(); Map writeProps = catalogTable.getOptions(); TableSchema tableSchema = TableSchemaUtils.getPhysicalSchema(catalogTable.getSchema()); TableLoader tableLoader; if (catalog != null) { - tableLoader = createTableLoader(catalog, objectPath); + tableLoader = createTableLoader(catalog, objectIdentifier.toObjectPath()); } else { tableLoader = createTableLoader( - catalogTable, writeProps, objectPath.getDatabaseName(), objectPath.getObjectName()); + catalogTable, + writeProps, + objectIdentifier.getDatabaseName(), + objectIdentifier.getObjectName()); } return new IcebergTableSink(tableLoader, tableSchema, context.getConfiguration(), writeProps); diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java index b1ffda156011..40e0b5f2a34e 100644 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java +++ b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java @@ -63,7 +63,8 @@ abstract class BaseDeltaTaskWriter extends BaseTaskWriter { this.wrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); this.keyWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(deleteSchema), deleteSchema.asStruct()); - this.keyProjection = RowDataProjection.create(schema, deleteSchema); + this.keyProjection = + RowDataProjection.create(flinkSchema, schema.asStruct(), deleteSchema.asStruct()); this.upsert = upsert; } diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java index cbdd18487069..d3859452a284 100644 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -48,6 +48,8 @@ import org.apache.iceberg.flink.FlinkReadOptions; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.source.assigner.OrderedSplitAssignerFactory; +import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; import org.apache.iceberg.flink.source.assigner.SplitAssigner; import org.apache.iceberg.flink.source.assigner.SplitAssignerFactory; import org.apache.iceberg.flink.source.enumerator.ContinuousIcebergEnumerator; @@ -63,6 +65,7 @@ import org.apache.iceberg.flink.source.reader.RowDataReaderFunction; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer; +import org.apache.iceberg.flink.source.split.SerializableComparator; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.util.ThreadPools; import org.slf4j.Logger; @@ -76,6 +79,7 @@ public class IcebergSource implements Source readerFunction; private final SplitAssignerFactory assignerFactory; + private final SerializableComparator splitComparator; // Can't use SerializableTable as enumerator needs a regular table // that can discover table changes @@ -86,11 +90,13 @@ public class IcebergSource implements Source readerFunction, SplitAssignerFactory assignerFactory, + SerializableComparator splitComparator, Table table) { this.tableLoader = tableLoader; this.scanContext = scanContext; this.readerFunction = readerFunction; this.assignerFactory = assignerFactory; + this.splitComparator = splitComparator; this.table = table; } @@ -146,7 +152,7 @@ public Boundedness getBoundedness() { public SourceReader createReader(SourceReaderContext readerContext) { IcebergSourceReaderMetrics metrics = new IcebergSourceReaderMetrics(readerContext.metricGroup(), lazyTable().name()); - return new IcebergSourceReader<>(metrics, readerFunction, readerContext); + return new IcebergSourceReader<>(metrics, readerFunction, splitComparator, readerContext); } @Override @@ -209,6 +215,7 @@ public static class Builder { private TableLoader tableLoader; private Table table; private SplitAssignerFactory splitAssignerFactory; + private SerializableComparator splitComparator; private ReaderFunction readerFunction; private ReadableConfig flinkConfig = new Configuration(); private final ScanContext.Builder contextBuilder = ScanContext.builder(); @@ -234,6 +241,12 @@ public Builder assignerFactory(SplitAssignerFactory assignerFactory) { return this; } + public Builder splitComparator( + SerializableComparator newSplitComparator) { + this.splitComparator = newSplitComparator; + return this; + } + public Builder readerFunction(ReaderFunction newReaderFunction) { this.readerFunction = newReaderFunction; return this; @@ -462,10 +475,18 @@ public IcebergSource build() { } } + if (splitAssignerFactory == null) { + if (splitComparator == null) { + splitAssignerFactory = new SimpleSplitAssignerFactory(); + } else { + splitAssignerFactory = new OrderedSplitAssignerFactory(splitComparator); + } + } + checkRequired(); // Since builder already load the table, pass it to the source to avoid double loading return new IcebergSource( - tableLoader, context, readerFunction, splitAssignerFactory, table); + tableLoader, context, readerFunction, splitAssignerFactory, splitComparator, table); } private void checkRequired() { diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssigner.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java similarity index 82% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssigner.java rename to flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java index b43660f907a6..37a0f1a6055f 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssigner.java +++ b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java @@ -20,7 +20,8 @@ import java.util.ArrayDeque; import java.util.Collection; -import java.util.Deque; +import java.util.PriorityQueue; +import java.util.Queue; import java.util.concurrent.CompletableFuture; import java.util.stream.Collectors; import javax.annotation.Nullable; @@ -28,24 +29,27 @@ import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; +import org.apache.iceberg.flink.source.split.SerializableComparator; /** * Since all methods are called in the source coordinator thread by enumerator, there is no need for * locking. */ @Internal -public class SimpleSplitAssigner implements SplitAssigner { +public class DefaultSplitAssigner implements SplitAssigner { - private final Deque pendingSplits; + private final Queue pendingSplits; private CompletableFuture availableFuture; - public SimpleSplitAssigner() { - this.pendingSplits = new ArrayDeque<>(); + public DefaultSplitAssigner(SerializableComparator comparator) { + this.pendingSplits = comparator == null ? new ArrayDeque<>() : new PriorityQueue<>(comparator); } - public SimpleSplitAssigner(Collection assignerState) { - this.pendingSplits = new ArrayDeque<>(assignerState.size()); - // Because simple assigner only tracks unassigned splits, + public DefaultSplitAssigner( + SerializableComparator comparator, + Collection assignerState) { + this(comparator); + // Because default assigner only tracks unassigned splits, // there is no need to filter splits based on status (unassigned) here. assignerState.forEach(splitState -> pendingSplits.add(splitState.split())); } diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java new file mode 100644 index 000000000000..e58478897aef --- /dev/null +++ b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.assigner; + +import java.util.Collection; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; +import org.apache.iceberg.flink.source.split.SerializableComparator; + +/** + * Create default assigner with a comparator that hands out splits where the order of the splits + * will be defined by the {@link SerializableComparator}. + */ +public class OrderedSplitAssignerFactory implements SplitAssignerFactory { + private final SerializableComparator comparator; + + public OrderedSplitAssignerFactory(SerializableComparator comparator) { + this.comparator = comparator; + } + + @Override + public SplitAssigner createAssigner() { + return new DefaultSplitAssigner(comparator); + } + + @Override + public SplitAssigner createAssigner(Collection assignerState) { + return new DefaultSplitAssigner(comparator, assignerState); + } +} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java index 1c14f4fcf9b9..a2e2ff364d46 100644 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java +++ b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java @@ -23,14 +23,15 @@ /** Create simple assigner that hands out splits without any guarantee in order or locality. */ public class SimpleSplitAssignerFactory implements SplitAssignerFactory { + public SimpleSplitAssignerFactory() {} @Override - public SimpleSplitAssigner createAssigner() { - return new SimpleSplitAssigner(); + public SplitAssigner createAssigner() { + return new DefaultSplitAssigner(null); } @Override - public SimpleSplitAssigner createAssigner(Collection assignerState) { - return new SimpleSplitAssigner(assignerState); + public SplitAssigner createAssigner(Collection assignerState) { + return new DefaultSplitAssigner(null, assignerState); } } diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java index 84c98055cebc..8d7d68f961cb 100644 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java +++ b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java @@ -25,6 +25,7 @@ import org.apache.flink.api.connector.source.SourceReaderContext; import org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBase; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.SerializableComparator; import org.apache.iceberg.flink.source.split.SplitRequestEvent; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -36,9 +37,10 @@ public class IcebergSourceReader public IcebergSourceReader( IcebergSourceReaderMetrics metrics, ReaderFunction readerFunction, + SerializableComparator splitComparator, SourceReaderContext context) { super( - () -> new IcebergSourceSplitReader<>(metrics, readerFunction, context), + () -> new IcebergSourceSplitReader<>(metrics, readerFunction, splitComparator, context), new IcebergSourceRecordEmitter<>(), context.getConfiguration(), context); diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java index bb016671e694..4e270dfa3d13 100644 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java +++ b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java @@ -22,6 +22,7 @@ import java.io.UncheckedIOException; import java.util.ArrayDeque; import java.util.Collections; +import java.util.List; import java.util.Queue; import org.apache.flink.api.connector.source.SourceReaderContext; import org.apache.flink.connector.base.source.reader.RecordsBySplits; @@ -31,7 +32,9 @@ import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.SerializableComparator; import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,6 +43,7 @@ class IcebergSourceSplitReader implements SplitReader, I private final IcebergSourceReaderMetrics metrics; private final ReaderFunction openSplitFunction; + private final SerializableComparator splitComparator; private final int indexOfSubtask; private final Queue splits; @@ -50,9 +54,11 @@ class IcebergSourceSplitReader implements SplitReader, I IcebergSourceSplitReader( IcebergSourceReaderMetrics metrics, ReaderFunction openSplitFunction, + SerializableComparator splitComparator, SourceReaderContext context) { this.metrics = metrics; this.openSplitFunction = openSplitFunction; + this.splitComparator = splitComparator; this.indexOfSubtask = context.getIndexOfSubtask(); this.splits = new ArrayDeque<>(); } @@ -93,8 +99,15 @@ public void handleSplitsChanges(SplitsChange splitsChange) { String.format("Unsupported split change: %s", splitsChange.getClass())); } - LOG.info("Add {} splits to reader", splitsChange.splits().size()); - splits.addAll(splitsChange.splits()); + if (splitComparator != null) { + List newSplits = Lists.newArrayList(splitsChange.splits()); + newSplits.sort(splitComparator); + LOG.info("Add {} splits to reader: {}", newSplits.size(), newSplits); + splits.addAll(newSplits); + } else { + LOG.info("Add {} splits to reader", splitsChange.splits().size()); + splits.addAll(splitsChange.splits()); + } metrics.incrementAssignedSplits(splitsChange.splits().size()); metrics.incrementAssignedBytes(calculateBytes(splitsChange)); } diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java new file mode 100644 index 000000000000..319648ca275c --- /dev/null +++ b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.split; + +import java.io.Serializable; +import java.util.Comparator; + +public interface SerializableComparator extends Comparator, Serializable {} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java new file mode 100644 index 000000000000..64e03d77debe --- /dev/null +++ b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.split; + +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * Provides implementations of {@link org.apache.iceberg.flink.source.split.SerializableComparator} + * which could be used for ordering splits. These are used by the {@link + * org.apache.iceberg.flink.source.assigner.OrderedSplitAssignerFactory} and the {@link + * org.apache.iceberg.flink.source.reader.IcebergSourceReader} + */ +public class SplitComparators { + private SplitComparators() {} + + /** Comparator which orders the splits based on the file sequence number of the data files */ + public static SerializableComparator fileSequenceNumber() { + return (IcebergSourceSplit o1, IcebergSourceSplit o2) -> { + Preconditions.checkArgument( + o1.task().files().size() == 1 && o2.task().files().size() == 1, + "Could not compare combined task. Please use 'split-open-file-cost' to prevent combining multiple files to a split"); + + Long seq1 = o1.task().files().iterator().next().file().fileSequenceNumber(); + Long seq2 = o2.task().files().iterator().next().file().fileSequenceNumber(); + + Preconditions.checkNotNull( + seq1, + "Invalid file sequence number: null. Doesn't support splits written with V1 format: %s", + o1); + Preconditions.checkNotNull( + seq2, + "IInvalid file sequence number: null. Doesn't support splits written with V1 format: %s", + o2); + + int temp = Long.compare(seq1, seq2); + if (temp != 0) { + return temp; + } else { + return o1.splitId().compareTo(o2.splitId()); + } + }; + } +} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java index 408c281004ed..5efb7413e70c 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java @@ -22,7 +22,6 @@ import java.io.IOException; import java.time.Duration; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -49,6 +48,7 @@ import org.apache.iceberg.TableScan; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.IcebergGenerics; +import org.apache.iceberg.data.InternalRecordWrapper; import org.apache.iceberg.data.Record; import org.apache.iceberg.deletes.EqualityDeleteWriter; import org.apache.iceberg.deletes.PositionDelete; @@ -349,7 +349,10 @@ public static void assertTableRecords(String tablePath, List expected, S public static StructLikeSet expectedRowSet(Table table, Record... records) { StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); - Collections.addAll(set, records); + InternalRecordWrapper wrapper = new InternalRecordWrapper(table.schema().asStruct()); + for (Record record : records) { + set.add(wrapper.copyFor(record)); + } return set; } @@ -361,12 +364,13 @@ public static StructLikeSet actualRowSet(Table table, Long snapshotId, String... throws IOException { table.refresh(); StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); + InternalRecordWrapper wrapper = new InternalRecordWrapper(table.schema().asStruct()); try (CloseableIterable reader = IcebergGenerics.read(table) .useSnapshot(snapshotId == null ? table.currentSnapshot().snapshotId() : snapshotId) .select(columns) .build()) { - reader.forEach(set::add); + reader.forEach(record -> set.add(wrapper.copyFor(record))); } return set; } diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java new file mode 100644 index 000000000000..2fbd7627efab --- /dev/null +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink; + +import java.io.File; +import java.util.concurrent.TimeUnit; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.Schema; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableDescriptor; +import org.apache.flink.table.api.TableEnvironment; +import org.assertj.core.api.Assertions; +import org.awaitility.Awaitility; +import org.junit.Test; + +public class TestFlinkAnonymousTable extends FlinkTestBase { + + @Test + public void testWriteAnonymousTable() throws Exception { + File warehouseDir = TEMPORARY_FOLDER.newFolder(); + TableEnvironment tEnv = getTableEnv(); + Table table = + tEnv.from( + TableDescriptor.forConnector("datagen") + .schema(Schema.newBuilder().column("f0", DataTypes.STRING()).build()) + .option("number-of-rows", "3") + .build()); + + TableDescriptor descriptor = + TableDescriptor.forConnector("iceberg") + .schema(Schema.newBuilder().column("f0", DataTypes.STRING()).build()) + .option("catalog-name", "hadoop_test") + .option("catalog-type", "hadoop") + .option("catalog-database", "test_db") + .option("catalog-table", "test") + .option("warehouse", warehouseDir.getAbsolutePath()) + .build(); + + table.insertInto(descriptor).execute(); + Awaitility.await() + .atMost(3, TimeUnit.SECONDS) + .untilAsserted( + () -> + Assertions.assertThat( + warehouseDir.toPath().resolve("test_db").resolve("test").toFile()) + .exists()); + } +} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java index a19e5f72bbe9..47b47cb6262d 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java @@ -24,12 +24,11 @@ import java.util.Objects; import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException; import org.apache.flink.types.Row; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.Schema; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -74,19 +73,6 @@ public void testCreateNamespace() { "Database should be created", validationNamespaceCatalog.namespaceExists(icebergNamespace)); } - @Test - public void testDefaultDatabase() { - sql("USE CATALOG %s", catalogName); - sql("SHOW TABLES"); - - Assert.assertEquals( - "Should use the current catalog", getTableEnv().getCurrentCatalog(), catalogName); - Assert.assertEquals( - "Should use the configured default namespace", - getTableEnv().getCurrentDatabase(), - "default"); - } - @Test public void testDropEmptyDatabase() { Assert.assertFalse( @@ -126,11 +112,11 @@ public void testDropNonEmptyNamespace() { "Table should exist", validationCatalog.tableExists(TableIdentifier.of(icebergNamespace, "tl"))); - AssertHelpers.assertThrowsCause( - "Should fail if trying to delete a non-empty database", - DatabaseNotEmptyException.class, - String.format("Database %s in catalog %s is not empty.", DATABASE, catalogName), - () -> sql("DROP DATABASE %s", flinkDatabase)); + Assertions.assertThatThrownBy(() -> sql("DROP DATABASE %s", flinkDatabase)) + .cause() + .isInstanceOf(DatabaseNotEmptyException.class) + .hasMessage( + String.format("Database %s in catalog %s is not empty.", DATABASE, catalogName)); sql("DROP TABLE %s.tl", flinkDatabase); } @@ -174,22 +160,17 @@ public void testListNamespace() { List databases = sql("SHOW DATABASES"); if (isHadoopCatalog) { - Assert.assertEquals("Should have 2 database", 2, databases.size()); - Assert.assertEquals( - "Should have db and default database", - Sets.newHashSet("default", "db"), - Sets.newHashSet(databases.get(0).getField(0), databases.get(1).getField(0))); + Assert.assertEquals("Should have 1 database", 1, databases.size()); + Assert.assertEquals("Should have db database", "db", databases.get(0).getField(0)); if (!baseNamespace.isEmpty()) { // test namespace not belongs to this catalog validationNamespaceCatalog.createNamespace( Namespace.of(baseNamespace.level(0), "UNKNOWN_NAMESPACE")); databases = sql("SHOW DATABASES"); - Assert.assertEquals("Should have 2 database", 2, databases.size()); + Assert.assertEquals("Should have 1 database", 1, databases.size()); Assert.assertEquals( - "Should have db and default database", - Sets.newHashSet("default", "db"), - Sets.newHashSet(databases.get(0).getField(0), databases.get(1).getField(0))); + "Should have db and default database", "db", databases.get(0).getField(0)); } } else { // If there are multiple classes extends FlinkTestBase, TestHiveMetastore may loose the @@ -301,10 +282,12 @@ public void testHadoopNotSupportMeta() { "Namespace should not already exist", validationNamespaceCatalog.namespaceExists(icebergNamespace)); - AssertHelpers.assertThrowsCause( - "Should fail if trying to create database with location in hadoop catalog.", - UnsupportedOperationException.class, - String.format("Cannot create namespace %s: metadata is not supported", icebergNamespace), - () -> sql("CREATE DATABASE %s WITH ('prop'='value')", flinkDatabase)); + Assertions.assertThatThrownBy( + () -> sql("CREATE DATABASE %s WITH ('prop'='value')", flinkDatabase)) + .cause() + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage( + String.format( + "Cannot create namespace %s: metadata is not supported", icebergNamespace)); } } diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java index f7edd5653ebd..6373d3a67b4c 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java @@ -43,7 +43,7 @@ public void before() { } @Test - public void testCreateCreateCatalogHive() { + public void testCreateCatalogHive() { String catalogName = "hiveCatalog"; props.put( FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HIVE); @@ -56,7 +56,7 @@ public void testCreateCreateCatalogHive() { } @Test - public void testCreateCreateCatalogHadoop() { + public void testCreateCatalogHadoop() { String catalogName = "hadoopCatalog"; props.put( FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HADOOP); @@ -69,7 +69,7 @@ public void testCreateCreateCatalogHadoop() { } @Test - public void testCreateCreateCatalogCustom() { + public void testCreateCatalogCustom() { String catalogName = "customCatalog"; props.put(CatalogProperties.CATALOG_IMPL, CustomHadoopCatalog.class.getName()); @@ -81,7 +81,7 @@ public void testCreateCreateCatalogCustom() { } @Test - public void testCreateCreateCatalogCustomWithHiveCatalogTypeSet() { + public void testCreateCatalogCustomWithHiveCatalogTypeSet() { String catalogName = "customCatalog"; props.put(CatalogProperties.CATALOG_IMPL, CustomHadoopCatalog.class.getName()); props.put( diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java index 1f8cbfe19152..4ecbd1c12921 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java @@ -23,31 +23,47 @@ import static org.apache.iceberg.flink.SimpleDataUtil.createRecord; import static org.apache.iceberg.flink.SimpleDataUtil.createUpdateAfter; import static org.apache.iceberg.flink.SimpleDataUtil.createUpdateBefore; +import static org.apache.iceberg.types.Types.NestedField.required; import java.io.File; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; +import java.time.OffsetDateTime; import java.util.Arrays; import java.util.List; import java.util.stream.Collectors; +import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.LocalZonedTimestampType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.RowKind; +import org.apache.iceberg.ContentFile; import org.apache.iceberg.FileContent; import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.RowDelta; +import org.apache.iceberg.Schema; import org.apache.iceberg.SerializableTable; import org.apache.iceberg.TableProperties; import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestTables; +import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.io.TaskWriter; import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.Types; import org.apache.iceberg.util.StructLikeSet; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -79,20 +95,6 @@ public void setupTable() throws IOException { this.metadataDir = new File(tableDir, "metadata"); } - private void initTable(boolean partitioned) { - if (partitioned) { - this.table = create(SCHEMA, PartitionSpec.builderFor(SCHEMA).identity("data").build()); - } else { - this.table = create(SCHEMA, PartitionSpec.unpartitioned()); - } - - table - .updateProperties() - .set(TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, String.valueOf(8 * 1024)) - .defaultFormat(format) - .commit(); - } - private int idFieldId() { return table.schema().findField("id").fieldId(); } @@ -170,18 +172,18 @@ private void testCdcEvents(boolean partitioned) throws IOException { @Test public void testUnpartitioned() throws IOException { - initTable(false); + createAndInitTable(false); testCdcEvents(false); } @Test public void testPartitioned() throws IOException { - initTable(true); + createAndInitTable(true); testCdcEvents(true); } private void testWritePureEqDeletes(boolean partitioned) throws IOException { - initTable(partitioned); + createAndInitTable(partitioned); List equalityFieldIds = Lists.newArrayList(idFieldId()); TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); taskWriterFactory.initialize(1, 1); @@ -210,7 +212,7 @@ public void testPartitionedPureEqDeletes() throws IOException { } private void testAbort(boolean partitioned) throws IOException { - initTable(partitioned); + createAndInitTable(partitioned); List equalityFieldIds = Lists.newArrayList(idFieldId()); TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); taskWriterFactory.initialize(1, 1); @@ -253,7 +255,7 @@ public void testPartitionedAbort() throws IOException { @Test public void testPartitionedTableWithDataAsKey() throws IOException { - initTable(true); + createAndInitTable(true); List equalityFieldIds = Lists.newArrayList(dataFieldId()); TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); taskWriterFactory.initialize(1, 1); @@ -298,7 +300,7 @@ public void testPartitionedTableWithDataAsKey() throws IOException { @Test public void testPartitionedTableWithDataAndIdAsKey() throws IOException { - initTable(true); + createAndInitTable(true); List equalityFieldIds = Lists.newArrayList(dataFieldId(), idFieldId()); TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); taskWriterFactory.initialize(1, 1); @@ -321,6 +323,62 @@ public void testPartitionedTableWithDataAndIdAsKey() throws IOException { "Should have expected records", expectedRowSet(createRecord(1, "aaa")), actualRowSet("*")); } + @Test + public void testEqualityColumnOnCustomPrecisionTSColumn() throws IOException { + Schema tableSchema = + new Schema( + required(3, "id", Types.IntegerType.get()), + required(4, "ts", Types.TimestampType.withZone())); + RowType flinkType = + new RowType( + false, + ImmutableList.of( + new RowType.RowField("id", new IntType()), + new RowType.RowField("ts", new LocalZonedTimestampType(3)))); + + this.table = create(tableSchema, PartitionSpec.unpartitioned()); + initTable(table); + + List equalityIds = ImmutableList.of(table.schema().findField("ts").fieldId()); + TaskWriterFactory taskWriterFactory = createTaskWriterFactory(flinkType, equalityIds); + taskWriterFactory.initialize(1, 1); + + TaskWriter writer = taskWriterFactory.create(); + RowDataSerializer serializer = new RowDataSerializer(flinkType); + OffsetDateTime start = OffsetDateTime.now(); + writer.write( + serializer.toBinaryRow( + GenericRowData.ofKind( + RowKind.INSERT, 1, TimestampData.fromInstant(start.toInstant())))); + writer.write( + serializer.toBinaryRow( + GenericRowData.ofKind( + RowKind.INSERT, 2, TimestampData.fromInstant(start.plusSeconds(1).toInstant())))); + writer.write( + serializer.toBinaryRow( + GenericRowData.ofKind( + RowKind.DELETE, 2, TimestampData.fromInstant(start.plusSeconds(1).toInstant())))); + + WriteResult result = writer.complete(); + // One data file + Assertions.assertThat(result.dataFiles().length).isEqualTo(1); + // One eq delete file + one pos delete file + Assertions.assertThat(result.deleteFiles().length).isEqualTo(2); + Assertions.assertThat( + Arrays.stream(result.deleteFiles()) + .map(ContentFile::content) + .collect(Collectors.toSet())) + .isEqualTo(Sets.newHashSet(FileContent.POSITION_DELETES, FileContent.EQUALITY_DELETES)); + commitTransaction(result); + + Record expectedRecord = GenericRecord.create(tableSchema); + expectedRecord.setField("id", 1); + int cutPrecisionNano = start.getNano() / 1000000 * 1000000; + expectedRecord.setField("ts", start.withNano(cutPrecisionNano)); + + Assertions.assertThat(actualRowSet("*")).isEqualTo(expectedRowSet(expectedRecord)); + } + private void commitTransaction(WriteResult result) { RowDelta rowDelta = table.newRowDelta(); Arrays.stream(result.dataFiles()).forEach(rowDelta::addRows); @@ -349,4 +407,34 @@ private TaskWriterFactory createTaskWriterFactory(List equalit equalityFieldIds, false); } + + private TaskWriterFactory createTaskWriterFactory( + RowType flinkType, List equalityFieldIds) { + return new RowDataTaskWriterFactory( + SerializableTable.copyOf(table), + flinkType, + 128 * 1024 * 1024, + format, + table.properties(), + equalityFieldIds, + true); + } + + private void createAndInitTable(boolean partitioned) { + if (partitioned) { + this.table = create(SCHEMA, PartitionSpec.builderFor(SCHEMA).identity("data").build()); + } else { + this.table = create(SCHEMA, PartitionSpec.unpartitioned()); + } + + initTable(table); + } + + private void initTable(TestTables.TestTable testTable) { + testTable + .updateProperties() + .set(TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, String.valueOf(8 * 1024)) + .defaultFormat(format) + .commit(); + } } diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java index 8dc68aad10aa..3a8071523b7c 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java @@ -20,19 +20,21 @@ import java.io.File; import java.util.List; -import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.BaseCombinedScanTask; import org.apache.iceberg.FileFormat; import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.ThreadPools; import org.junit.Assert; @@ -40,8 +42,6 @@ public class SplitHelpers { - private static final AtomicLong splitLengthIncrement = new AtomicLong(); - private SplitHelpers() {} /** @@ -54,16 +54,53 @@ private SplitHelpers() {} * *

Since the table and data files are deleted before this method return, caller shouldn't * attempt to read the data files. + * + *

By default, v1 Iceberg table is created. For v2 table use {@link + * SplitHelpers#createSplitsFromTransientHadoopTable(TemporaryFolder, int, int, String)} + * + * @param temporaryFolder Folder to place the data to + * @param fileCount The number of files to create and add to the table + * @param filesPerSplit The number of files used for a split */ public static List createSplitsFromTransientHadoopTable( TemporaryFolder temporaryFolder, int fileCount, int filesPerSplit) throws Exception { + return createSplitsFromTransientHadoopTable(temporaryFolder, fileCount, filesPerSplit, "1"); + } + + /** + * This create a list of IcebergSourceSplit from real files + *

  • Create a new Hadoop table under the {@code temporaryFolder} + *
  • write {@code fileCount} number of files to the new Iceberg table + *
  • Discover the splits from the table and partition the splits by the {@code filePerSplit} + * limit + *
  • Delete the Hadoop table + * + *

    Since the table and data files are deleted before this method return, caller shouldn't + * attempt to read the data files. + * + * @param temporaryFolder Folder to place the data to + * @param fileCount The number of files to create and add to the table + * @param filesPerSplit The number of files used for a split + * @param version The table version to create + */ + public static List createSplitsFromTransientHadoopTable( + TemporaryFolder temporaryFolder, int fileCount, int filesPerSplit, String version) + throws Exception { final File warehouseFile = temporaryFolder.newFolder(); Assert.assertTrue(warehouseFile.delete()); final String warehouse = "file:" + warehouseFile; Configuration hadoopConf = new Configuration(); final HadoopCatalog catalog = new HadoopCatalog(hadoopConf, warehouse); + ImmutableMap properties = + ImmutableMap.of(TableProperties.FORMAT_VERSION, version); try { - final Table table = catalog.createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + final Table table = + catalog.createTable( + TestFixtures.TABLE_IDENTIFIER, + TestFixtures.SCHEMA, + PartitionSpec.unpartitioned(), + null, + properties); final GenericAppenderHelper dataAppender = new GenericAppenderHelper(table, FileFormat.PARQUET, temporaryFolder); for (int i = 0; i < fileCount; ++i) { diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestSimpleSplitAssigner.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java similarity index 88% rename from flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestSimpleSplitAssigner.java rename to flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java index ee6c5cc3a6c8..f28677ca9d6a 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestSimpleSplitAssigner.java +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java @@ -30,22 +30,24 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; -public class TestSimpleSplitAssigner { +public abstract class SplitAssignerTestBase { @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); @Test public void testEmptyInitialization() { - SimpleSplitAssigner assigner = new SimpleSplitAssigner(); + SplitAssigner assigner = splitAssigner(); assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); } /** Test a sequence of interactions for StaticEnumerator */ @Test public void testStaticEnumeratorSequence() throws Exception { - SimpleSplitAssigner assigner = new SimpleSplitAssigner(); + SplitAssigner assigner = splitAssigner(); assigner.onDiscoveredSplits( - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 4, 2)); + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 4, 1)); + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); assertSnapshot(assigner, 1); assigner.onUnassignedSplits( @@ -61,7 +63,7 @@ public void testStaticEnumeratorSequence() throws Exception { /** Test a sequence of interactions for ContinuousEnumerator */ @Test public void testContinuousEnumeratorSequence() throws Exception { - SimpleSplitAssigner assigner = new SimpleSplitAssigner(); + SplitAssigner assigner = splitAssigner(); assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); List splits1 = @@ -81,7 +83,7 @@ public void testContinuousEnumeratorSequence() throws Exception { } private void assertAvailableFuture( - SimpleSplitAssigner assigner, int splitCount, Runnable addSplitsRunnable) { + SplitAssigner assigner, int splitCount, Runnable addSplitsRunnable) { // register callback AtomicBoolean futureCompleted = new AtomicBoolean(); CompletableFuture future = assigner.isAvailable(); @@ -102,7 +104,7 @@ private void assertAvailableFuture( assertSnapshot(assigner, 0); } - private void assertGetNext(SimpleSplitAssigner assigner, GetSplitResult.Status expectedStatus) { + protected void assertGetNext(SplitAssigner assigner, GetSplitResult.Status expectedStatus) { GetSplitResult result = assigner.getNext(null); Assert.assertEquals(expectedStatus, result.status()); switch (expectedStatus) { @@ -118,8 +120,10 @@ private void assertGetNext(SimpleSplitAssigner assigner, GetSplitResult.Status e } } - private void assertSnapshot(SimpleSplitAssigner assigner, int splitCount) { + protected void assertSnapshot(SplitAssigner assigner, int splitCount) { Collection stateBeforeGet = assigner.state(); Assert.assertEquals(splitCount, stateBeforeGet.size()); } + + protected abstract SplitAssigner splitAssigner(); } diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java new file mode 100644 index 000000000000..8994f3054abe --- /dev/null +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.assigner; + +import org.apache.iceberg.flink.source.SplitHelpers; +import org.junit.Test; + +public class TestDefaultSplitAssigner extends SplitAssignerTestBase { + @Override + protected SplitAssigner splitAssigner() { + return new DefaultSplitAssigner(null); + } + + /** Test the assigner when multiple files are in a single split */ + @Test + public void testMultipleFilesInASplit() throws Exception { + SplitAssigner assigner = splitAssigner(); + assigner.onDiscoveredSplits( + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 4, 2)); + + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertSnapshot(assigner, 1); + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + assertSnapshot(assigner, 0); + } +} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java new file mode 100644 index 000000000000..8b9e132e0e22 --- /dev/null +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.assigner; + +import java.util.List; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.flink.source.SplitHelpers; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.SerializableComparator; +import org.apache.iceberg.flink.source.split.SplitComparators; +import org.apache.iceberg.util.SerializationUtil; +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Test; + +public class TestFileSequenceNumberBasedSplitAssigner extends SplitAssignerTestBase { + @Override + protected SplitAssigner splitAssigner() { + return new OrderedSplitAssignerFactory(SplitComparators.fileSequenceNumber()).createAssigner(); + } + + /** Test the assigner when multiple files are in a single split */ + @Test + public void testMultipleFilesInAnIcebergSplit() { + SplitAssigner assigner = splitAssigner(); + Assertions.assertThatThrownBy( + () -> + assigner.onDiscoveredSplits( + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 4, 2, "2")), + "Multiple files in a split is not allowed") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Please use 'split-open-file-cost'"); + } + + /** Test sorted splits */ + @Test + public void testSplitSort() throws Exception { + SplitAssigner assigner = splitAssigner(); + List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 5, 1, "2"); + + assigner.onDiscoveredSplits(splits.subList(3, 5)); + assigner.onDiscoveredSplits(splits.subList(0, 1)); + assigner.onDiscoveredSplits(splits.subList(1, 3)); + + assertGetNext(assigner, 1L); + assertGetNext(assigner, 2L); + assertGetNext(assigner, 3L); + assertGetNext(assigner, 4L); + assertGetNext(assigner, 5L); + + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + } + + @Test + public void testSerializable() { + byte[] bytes = SerializationUtil.serializeToBytes(SplitComparators.fileSequenceNumber()); + SerializableComparator comparator = + SerializationUtil.deserializeFromBytes(bytes); + Assert.assertNotNull(comparator); + } + + protected void assertGetNext(SplitAssigner assigner, Long expectedSequenceNumber) { + GetSplitResult result = assigner.getNext(null); + ContentFile file = result.split().task().files().iterator().next().file(); + Assert.assertEquals(expectedSequenceNumber, file.fileSequenceNumber()); + } +} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java index d0ae8fdf7748..349eb11cf549 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java @@ -28,7 +28,7 @@ import org.apache.iceberg.flink.source.ScanContext; import org.apache.iceberg.flink.source.SplitHelpers; import org.apache.iceberg.flink.source.StreamingStartingStrategy; -import org.apache.iceberg.flink.source.assigner.SimpleSplitAssigner; +import org.apache.iceberg.flink.source.assigner.DefaultSplitAssigner; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; @@ -342,7 +342,7 @@ private static ContinuousIcebergEnumerator createEnumerator( ContinuousIcebergEnumerator enumerator = new ContinuousIcebergEnumerator( context, - new SimpleSplitAssigner(Collections.emptyList()), + new DefaultSplitAssigner(null, Collections.emptyList()), scanContext, splitPlanner, null); diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java index 56af0caf1298..def4f436851b 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java @@ -35,6 +35,7 @@ import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.TestHelpers; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.SerializableComparator; import org.apache.iceberg.hadoop.HadoopFileIO; import org.junit.Assert; import org.junit.ClassRule; @@ -55,13 +56,68 @@ public void testReaderMetrics() throws Exception { TestingReaderOutput readerOutput = new TestingReaderOutput<>(); TestingMetricGroup metricGroup = new TestingMetricGroup(); TestingReaderContext readerContext = new TestingReaderContext(new Configuration(), metricGroup); - IcebergSourceReader reader = createReader(metricGroup, readerContext); + IcebergSourceReader reader = createReader(metricGroup, readerContext, null); reader.start(); testOneSplitFetcher(reader, readerOutput, metricGroup, 1); testOneSplitFetcher(reader, readerOutput, metricGroup, 2); } + @Test + public void testReaderOrder() throws Exception { + // Create 2 splits + List> recordBatchList1 = + ReaderUtil.createRecordBatchList(0L, TestFixtures.SCHEMA, 1, 1); + CombinedScanTask task1 = + ReaderUtil.createCombinedScanTask( + recordBatchList1, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory); + + List> recordBatchList2 = + ReaderUtil.createRecordBatchList(1L, TestFixtures.SCHEMA, 1, 1); + CombinedScanTask task2 = + ReaderUtil.createCombinedScanTask( + recordBatchList2, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory); + + // Sort the splits in one way + List rowDataList1 = + read( + Arrays.asList( + IcebergSourceSplit.fromCombinedScanTask(task1), + IcebergSourceSplit.fromCombinedScanTask(task2)), + 2); + + // Reverse the splits + List rowDataList2 = + read( + Arrays.asList( + IcebergSourceSplit.fromCombinedScanTask(task2), + IcebergSourceSplit.fromCombinedScanTask(task1)), + 2); + + // Check that the order of the elements is not changed + Assert.assertEquals(rowDataList1.get(0), rowDataList2.get(0)); + Assert.assertEquals(rowDataList1.get(1), rowDataList2.get(1)); + } + + private List read(List splits, long expected) throws Exception { + TestingMetricGroup metricGroup = new TestingMetricGroup(); + TestingReaderContext readerContext = new TestingReaderContext(new Configuration(), metricGroup); + // Using IdBasedComparator, so we can have a deterministic order of the splits + IcebergSourceReader reader = createReader(metricGroup, readerContext, new IdBasedComparator()); + reader.start(); + + reader.addSplits(splits); + TestingReaderOutput readerOutput = new TestingReaderOutput<>(); + while (readerOutput.getEmittedRecords().size() < expected) { + reader.pollNext(readerOutput); + } + + reader.pollNext(readerOutput); + + Assert.assertEquals(expected, readerOutput.getEmittedRecords().size()); + return readerOutput.getEmittedRecords(); + } + private void testOneSplitFetcher( IcebergSourceReader reader, TestingReaderOutput readerOutput, @@ -96,7 +152,9 @@ private void testOneSplitFetcher( } private IcebergSourceReader createReader( - MetricGroup metricGroup, SourceReaderContext readerContext) { + MetricGroup metricGroup, + SourceReaderContext readerContext, + SerializableComparator splitComparator) { IcebergSourceReaderMetrics readerMetrics = new IcebergSourceReaderMetrics(metricGroup, "db.tbl"); RowDataReaderFunction readerFunction = @@ -109,6 +167,13 @@ private IcebergSourceReader createReader( new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), new PlaintextEncryptionManager(), Collections.emptyList()); - return new IcebergSourceReader<>(readerMetrics, readerFunction, readerContext); + return new IcebergSourceReader<>(readerMetrics, readerFunction, splitComparator, readerContext); + } + + private static class IdBasedComparator implements SerializableComparator { + @Override + public int compare(IcebergSourceSplit o1, IcebergSourceSplit o2) { + return o1.splitId().compareTo(o2.splitId()); + } } } diff --git a/flink/v1.16/build.gradle b/flink/v1.16/build.gradle index bae03096db9d..085b05161e70 100644 --- a/flink/v1.16/build.gradle +++ b/flink/v1.16/build.gradle @@ -112,6 +112,8 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { exclude group: 'javax.transaction', module: 'transaction-api' exclude group: 'com.zaxxer', module: 'HikariCP' } + + testImplementation 'org.awaitility:awaitility' } } @@ -238,6 +240,7 @@ project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") { task integrationTest(type: Test) { description = "Test Flink Runtime Jar against Flink ${flinkMajorVersion}" group = "verification" + jvmArgs += project.property('extraJvmArgs') testClassesDirs = sourceSets.integration.output.classesDirs classpath = sourceSets.integration.runtimeClasspath + files(shadowJar.archiveFile.get().asFile.path) inputs.file(shadowJar.archiveFile.get().asFile.path) diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java index 3ecddaa05c9b..825816fdf416 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java @@ -124,14 +124,7 @@ public FlinkCatalog( } @Override - public void open() throws CatalogException { - // Create the default database if it does not exist. - try { - createDatabase(getDefaultDatabase(), ImmutableMap.of(), true); - } catch (DatabaseAlreadyExistException e) { - // Ignore the exception if it's already exist. - } - } + public void open() throws CatalogException {} @Override public void close() throws CatalogException { diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java index 3f04d35e851c..8e1f420b722d 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java @@ -105,18 +105,21 @@ public DynamicTableSource createDynamicTableSource(Context context) { @Override public DynamicTableSink createDynamicTableSink(Context context) { - ObjectPath objectPath = context.getObjectIdentifier().toObjectPath(); + ObjectIdentifier objectIdentifier = context.getObjectIdentifier(); CatalogTable catalogTable = context.getCatalogTable(); Map writeProps = catalogTable.getOptions(); TableSchema tableSchema = TableSchemaUtils.getPhysicalSchema(catalogTable.getSchema()); TableLoader tableLoader; if (catalog != null) { - tableLoader = createTableLoader(catalog, objectPath); + tableLoader = createTableLoader(catalog, objectIdentifier.toObjectPath()); } else { tableLoader = createTableLoader( - catalogTable, writeProps, objectPath.getDatabaseName(), objectPath.getObjectName()); + catalogTable, + writeProps, + objectIdentifier.getDatabaseName(), + objectIdentifier.getObjectName()); } return new IcebergTableSink(tableLoader, tableSchema, context.getConfiguration(), writeProps); diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java index b1ffda156011..40e0b5f2a34e 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java @@ -63,7 +63,8 @@ abstract class BaseDeltaTaskWriter extends BaseTaskWriter { this.wrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); this.keyWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(deleteSchema), deleteSchema.asStruct()); - this.keyProjection = RowDataProjection.create(schema, deleteSchema); + this.keyProjection = + RowDataProjection.create(flinkSchema, schema.asStruct(), deleteSchema.asStruct()); this.upsert = upsert; } diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java index cbdd18487069..d3859452a284 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -48,6 +48,8 @@ import org.apache.iceberg.flink.FlinkReadOptions; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.source.assigner.OrderedSplitAssignerFactory; +import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; import org.apache.iceberg.flink.source.assigner.SplitAssigner; import org.apache.iceberg.flink.source.assigner.SplitAssignerFactory; import org.apache.iceberg.flink.source.enumerator.ContinuousIcebergEnumerator; @@ -63,6 +65,7 @@ import org.apache.iceberg.flink.source.reader.RowDataReaderFunction; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer; +import org.apache.iceberg.flink.source.split.SerializableComparator; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.util.ThreadPools; import org.slf4j.Logger; @@ -76,6 +79,7 @@ public class IcebergSource implements Source readerFunction; private final SplitAssignerFactory assignerFactory; + private final SerializableComparator splitComparator; // Can't use SerializableTable as enumerator needs a regular table // that can discover table changes @@ -86,11 +90,13 @@ public class IcebergSource implements Source readerFunction, SplitAssignerFactory assignerFactory, + SerializableComparator splitComparator, Table table) { this.tableLoader = tableLoader; this.scanContext = scanContext; this.readerFunction = readerFunction; this.assignerFactory = assignerFactory; + this.splitComparator = splitComparator; this.table = table; } @@ -146,7 +152,7 @@ public Boundedness getBoundedness() { public SourceReader createReader(SourceReaderContext readerContext) { IcebergSourceReaderMetrics metrics = new IcebergSourceReaderMetrics(readerContext.metricGroup(), lazyTable().name()); - return new IcebergSourceReader<>(metrics, readerFunction, readerContext); + return new IcebergSourceReader<>(metrics, readerFunction, splitComparator, readerContext); } @Override @@ -209,6 +215,7 @@ public static class Builder { private TableLoader tableLoader; private Table table; private SplitAssignerFactory splitAssignerFactory; + private SerializableComparator splitComparator; private ReaderFunction readerFunction; private ReadableConfig flinkConfig = new Configuration(); private final ScanContext.Builder contextBuilder = ScanContext.builder(); @@ -234,6 +241,12 @@ public Builder assignerFactory(SplitAssignerFactory assignerFactory) { return this; } + public Builder splitComparator( + SerializableComparator newSplitComparator) { + this.splitComparator = newSplitComparator; + return this; + } + public Builder readerFunction(ReaderFunction newReaderFunction) { this.readerFunction = newReaderFunction; return this; @@ -462,10 +475,18 @@ public IcebergSource build() { } } + if (splitAssignerFactory == null) { + if (splitComparator == null) { + splitAssignerFactory = new SimpleSplitAssignerFactory(); + } else { + splitAssignerFactory = new OrderedSplitAssignerFactory(splitComparator); + } + } + checkRequired(); // Since builder already load the table, pass it to the source to avoid double loading return new IcebergSource( - tableLoader, context, readerFunction, splitAssignerFactory, table); + tableLoader, context, readerFunction, splitAssignerFactory, splitComparator, table); } private void checkRequired() { diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssigner.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java similarity index 82% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssigner.java rename to flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java index b43660f907a6..37a0f1a6055f 100644 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssigner.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java @@ -20,7 +20,8 @@ import java.util.ArrayDeque; import java.util.Collection; -import java.util.Deque; +import java.util.PriorityQueue; +import java.util.Queue; import java.util.concurrent.CompletableFuture; import java.util.stream.Collectors; import javax.annotation.Nullable; @@ -28,24 +29,27 @@ import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; +import org.apache.iceberg.flink.source.split.SerializableComparator; /** * Since all methods are called in the source coordinator thread by enumerator, there is no need for * locking. */ @Internal -public class SimpleSplitAssigner implements SplitAssigner { +public class DefaultSplitAssigner implements SplitAssigner { - private final Deque pendingSplits; + private final Queue pendingSplits; private CompletableFuture availableFuture; - public SimpleSplitAssigner() { - this.pendingSplits = new ArrayDeque<>(); + public DefaultSplitAssigner(SerializableComparator comparator) { + this.pendingSplits = comparator == null ? new ArrayDeque<>() : new PriorityQueue<>(comparator); } - public SimpleSplitAssigner(Collection assignerState) { - this.pendingSplits = new ArrayDeque<>(assignerState.size()); - // Because simple assigner only tracks unassigned splits, + public DefaultSplitAssigner( + SerializableComparator comparator, + Collection assignerState) { + this(comparator); + // Because default assigner only tracks unassigned splits, // there is no need to filter splits based on status (unassigned) here. assignerState.forEach(splitState -> pendingSplits.add(splitState.split())); } diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java new file mode 100644 index 000000000000..e58478897aef --- /dev/null +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.assigner; + +import java.util.Collection; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; +import org.apache.iceberg.flink.source.split.SerializableComparator; + +/** + * Create default assigner with a comparator that hands out splits where the order of the splits + * will be defined by the {@link SerializableComparator}. + */ +public class OrderedSplitAssignerFactory implements SplitAssignerFactory { + private final SerializableComparator comparator; + + public OrderedSplitAssignerFactory(SerializableComparator comparator) { + this.comparator = comparator; + } + + @Override + public SplitAssigner createAssigner() { + return new DefaultSplitAssigner(comparator); + } + + @Override + public SplitAssigner createAssigner(Collection assignerState) { + return new DefaultSplitAssigner(comparator, assignerState); + } +} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java index 1c14f4fcf9b9..a2e2ff364d46 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java @@ -23,14 +23,15 @@ /** Create simple assigner that hands out splits without any guarantee in order or locality. */ public class SimpleSplitAssignerFactory implements SplitAssignerFactory { + public SimpleSplitAssignerFactory() {} @Override - public SimpleSplitAssigner createAssigner() { - return new SimpleSplitAssigner(); + public SplitAssigner createAssigner() { + return new DefaultSplitAssigner(null); } @Override - public SimpleSplitAssigner createAssigner(Collection assignerState) { - return new SimpleSplitAssigner(assignerState); + public SplitAssigner createAssigner(Collection assignerState) { + return new DefaultSplitAssigner(null, assignerState); } } diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java index 84c98055cebc..8d7d68f961cb 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java @@ -25,6 +25,7 @@ import org.apache.flink.api.connector.source.SourceReaderContext; import org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBase; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.SerializableComparator; import org.apache.iceberg.flink.source.split.SplitRequestEvent; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -36,9 +37,10 @@ public class IcebergSourceReader public IcebergSourceReader( IcebergSourceReaderMetrics metrics, ReaderFunction readerFunction, + SerializableComparator splitComparator, SourceReaderContext context) { super( - () -> new IcebergSourceSplitReader<>(metrics, readerFunction, context), + () -> new IcebergSourceSplitReader<>(metrics, readerFunction, splitComparator, context), new IcebergSourceRecordEmitter<>(), context.getConfiguration(), context); diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java index bb016671e694..4e270dfa3d13 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java @@ -22,6 +22,7 @@ import java.io.UncheckedIOException; import java.util.ArrayDeque; import java.util.Collections; +import java.util.List; import java.util.Queue; import org.apache.flink.api.connector.source.SourceReaderContext; import org.apache.flink.connector.base.source.reader.RecordsBySplits; @@ -31,7 +32,9 @@ import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.SerializableComparator; import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,6 +43,7 @@ class IcebergSourceSplitReader implements SplitReader, I private final IcebergSourceReaderMetrics metrics; private final ReaderFunction openSplitFunction; + private final SerializableComparator splitComparator; private final int indexOfSubtask; private final Queue splits; @@ -50,9 +54,11 @@ class IcebergSourceSplitReader implements SplitReader, I IcebergSourceSplitReader( IcebergSourceReaderMetrics metrics, ReaderFunction openSplitFunction, + SerializableComparator splitComparator, SourceReaderContext context) { this.metrics = metrics; this.openSplitFunction = openSplitFunction; + this.splitComparator = splitComparator; this.indexOfSubtask = context.getIndexOfSubtask(); this.splits = new ArrayDeque<>(); } @@ -93,8 +99,15 @@ public void handleSplitsChanges(SplitsChange splitsChange) { String.format("Unsupported split change: %s", splitsChange.getClass())); } - LOG.info("Add {} splits to reader", splitsChange.splits().size()); - splits.addAll(splitsChange.splits()); + if (splitComparator != null) { + List newSplits = Lists.newArrayList(splitsChange.splits()); + newSplits.sort(splitComparator); + LOG.info("Add {} splits to reader: {}", newSplits.size(), newSplits); + splits.addAll(newSplits); + } else { + LOG.info("Add {} splits to reader", splitsChange.splits().size()); + splits.addAll(splitsChange.splits()); + } metrics.incrementAssignedSplits(splitsChange.splits().size()); metrics.incrementAssignedBytes(calculateBytes(splitsChange)); } diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java new file mode 100644 index 000000000000..319648ca275c --- /dev/null +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.split; + +import java.io.Serializable; +import java.util.Comparator; + +public interface SerializableComparator extends Comparator, Serializable {} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java new file mode 100644 index 000000000000..64e03d77debe --- /dev/null +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.split; + +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * Provides implementations of {@link org.apache.iceberg.flink.source.split.SerializableComparator} + * which could be used for ordering splits. These are used by the {@link + * org.apache.iceberg.flink.source.assigner.OrderedSplitAssignerFactory} and the {@link + * org.apache.iceberg.flink.source.reader.IcebergSourceReader} + */ +public class SplitComparators { + private SplitComparators() {} + + /** Comparator which orders the splits based on the file sequence number of the data files */ + public static SerializableComparator fileSequenceNumber() { + return (IcebergSourceSplit o1, IcebergSourceSplit o2) -> { + Preconditions.checkArgument( + o1.task().files().size() == 1 && o2.task().files().size() == 1, + "Could not compare combined task. Please use 'split-open-file-cost' to prevent combining multiple files to a split"); + + Long seq1 = o1.task().files().iterator().next().file().fileSequenceNumber(); + Long seq2 = o2.task().files().iterator().next().file().fileSequenceNumber(); + + Preconditions.checkNotNull( + seq1, + "Invalid file sequence number: null. Doesn't support splits written with V1 format: %s", + o1); + Preconditions.checkNotNull( + seq2, + "IInvalid file sequence number: null. Doesn't support splits written with V1 format: %s", + o2); + + int temp = Long.compare(seq1, seq2); + if (temp != 0) { + return temp; + } else { + return o1.splitId().compareTo(o2.splitId()); + } + }; + } +} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java index 408c281004ed..5efb7413e70c 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java @@ -22,7 +22,6 @@ import java.io.IOException; import java.time.Duration; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -49,6 +48,7 @@ import org.apache.iceberg.TableScan; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.IcebergGenerics; +import org.apache.iceberg.data.InternalRecordWrapper; import org.apache.iceberg.data.Record; import org.apache.iceberg.deletes.EqualityDeleteWriter; import org.apache.iceberg.deletes.PositionDelete; @@ -349,7 +349,10 @@ public static void assertTableRecords(String tablePath, List expected, S public static StructLikeSet expectedRowSet(Table table, Record... records) { StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); - Collections.addAll(set, records); + InternalRecordWrapper wrapper = new InternalRecordWrapper(table.schema().asStruct()); + for (Record record : records) { + set.add(wrapper.copyFor(record)); + } return set; } @@ -361,12 +364,13 @@ public static StructLikeSet actualRowSet(Table table, Long snapshotId, String... throws IOException { table.refresh(); StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); + InternalRecordWrapper wrapper = new InternalRecordWrapper(table.schema().asStruct()); try (CloseableIterable reader = IcebergGenerics.read(table) .useSnapshot(snapshotId == null ? table.currentSnapshot().snapshotId() : snapshotId) .select(columns) .build()) { - reader.forEach(set::add); + reader.forEach(record -> set.add(wrapper.copyFor(record))); } return set; } diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java new file mode 100644 index 000000000000..2fbd7627efab --- /dev/null +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink; + +import java.io.File; +import java.util.concurrent.TimeUnit; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.Schema; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableDescriptor; +import org.apache.flink.table.api.TableEnvironment; +import org.assertj.core.api.Assertions; +import org.awaitility.Awaitility; +import org.junit.Test; + +public class TestFlinkAnonymousTable extends FlinkTestBase { + + @Test + public void testWriteAnonymousTable() throws Exception { + File warehouseDir = TEMPORARY_FOLDER.newFolder(); + TableEnvironment tEnv = getTableEnv(); + Table table = + tEnv.from( + TableDescriptor.forConnector("datagen") + .schema(Schema.newBuilder().column("f0", DataTypes.STRING()).build()) + .option("number-of-rows", "3") + .build()); + + TableDescriptor descriptor = + TableDescriptor.forConnector("iceberg") + .schema(Schema.newBuilder().column("f0", DataTypes.STRING()).build()) + .option("catalog-name", "hadoop_test") + .option("catalog-type", "hadoop") + .option("catalog-database", "test_db") + .option("catalog-table", "test") + .option("warehouse", warehouseDir.getAbsolutePath()) + .build(); + + table.insertInto(descriptor).execute(); + Awaitility.await() + .atMost(3, TimeUnit.SECONDS) + .untilAsserted( + () -> + Assertions.assertThat( + warehouseDir.toPath().resolve("test_db").resolve("test").toFile()) + .exists()); + } +} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java index a19e5f72bbe9..47b47cb6262d 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java @@ -24,12 +24,11 @@ import java.util.Objects; import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException; import org.apache.flink.types.Row; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.Schema; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -74,19 +73,6 @@ public void testCreateNamespace() { "Database should be created", validationNamespaceCatalog.namespaceExists(icebergNamespace)); } - @Test - public void testDefaultDatabase() { - sql("USE CATALOG %s", catalogName); - sql("SHOW TABLES"); - - Assert.assertEquals( - "Should use the current catalog", getTableEnv().getCurrentCatalog(), catalogName); - Assert.assertEquals( - "Should use the configured default namespace", - getTableEnv().getCurrentDatabase(), - "default"); - } - @Test public void testDropEmptyDatabase() { Assert.assertFalse( @@ -126,11 +112,11 @@ public void testDropNonEmptyNamespace() { "Table should exist", validationCatalog.tableExists(TableIdentifier.of(icebergNamespace, "tl"))); - AssertHelpers.assertThrowsCause( - "Should fail if trying to delete a non-empty database", - DatabaseNotEmptyException.class, - String.format("Database %s in catalog %s is not empty.", DATABASE, catalogName), - () -> sql("DROP DATABASE %s", flinkDatabase)); + Assertions.assertThatThrownBy(() -> sql("DROP DATABASE %s", flinkDatabase)) + .cause() + .isInstanceOf(DatabaseNotEmptyException.class) + .hasMessage( + String.format("Database %s in catalog %s is not empty.", DATABASE, catalogName)); sql("DROP TABLE %s.tl", flinkDatabase); } @@ -174,22 +160,17 @@ public void testListNamespace() { List databases = sql("SHOW DATABASES"); if (isHadoopCatalog) { - Assert.assertEquals("Should have 2 database", 2, databases.size()); - Assert.assertEquals( - "Should have db and default database", - Sets.newHashSet("default", "db"), - Sets.newHashSet(databases.get(0).getField(0), databases.get(1).getField(0))); + Assert.assertEquals("Should have 1 database", 1, databases.size()); + Assert.assertEquals("Should have db database", "db", databases.get(0).getField(0)); if (!baseNamespace.isEmpty()) { // test namespace not belongs to this catalog validationNamespaceCatalog.createNamespace( Namespace.of(baseNamespace.level(0), "UNKNOWN_NAMESPACE")); databases = sql("SHOW DATABASES"); - Assert.assertEquals("Should have 2 database", 2, databases.size()); + Assert.assertEquals("Should have 1 database", 1, databases.size()); Assert.assertEquals( - "Should have db and default database", - Sets.newHashSet("default", "db"), - Sets.newHashSet(databases.get(0).getField(0), databases.get(1).getField(0))); + "Should have db and default database", "db", databases.get(0).getField(0)); } } else { // If there are multiple classes extends FlinkTestBase, TestHiveMetastore may loose the @@ -301,10 +282,12 @@ public void testHadoopNotSupportMeta() { "Namespace should not already exist", validationNamespaceCatalog.namespaceExists(icebergNamespace)); - AssertHelpers.assertThrowsCause( - "Should fail if trying to create database with location in hadoop catalog.", - UnsupportedOperationException.class, - String.format("Cannot create namespace %s: metadata is not supported", icebergNamespace), - () -> sql("CREATE DATABASE %s WITH ('prop'='value')", flinkDatabase)); + Assertions.assertThatThrownBy( + () -> sql("CREATE DATABASE %s WITH ('prop'='value')", flinkDatabase)) + .cause() + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage( + String.format( + "Cannot create namespace %s: metadata is not supported", icebergNamespace)); } } diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java index f7edd5653ebd..6373d3a67b4c 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java @@ -43,7 +43,7 @@ public void before() { } @Test - public void testCreateCreateCatalogHive() { + public void testCreateCatalogHive() { String catalogName = "hiveCatalog"; props.put( FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HIVE); @@ -56,7 +56,7 @@ public void testCreateCreateCatalogHive() { } @Test - public void testCreateCreateCatalogHadoop() { + public void testCreateCatalogHadoop() { String catalogName = "hadoopCatalog"; props.put( FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HADOOP); @@ -69,7 +69,7 @@ public void testCreateCreateCatalogHadoop() { } @Test - public void testCreateCreateCatalogCustom() { + public void testCreateCatalogCustom() { String catalogName = "customCatalog"; props.put(CatalogProperties.CATALOG_IMPL, CustomHadoopCatalog.class.getName()); @@ -81,7 +81,7 @@ public void testCreateCreateCatalogCustom() { } @Test - public void testCreateCreateCatalogCustomWithHiveCatalogTypeSet() { + public void testCreateCatalogCustomWithHiveCatalogTypeSet() { String catalogName = "customCatalog"; props.put(CatalogProperties.CATALOG_IMPL, CustomHadoopCatalog.class.getName()); props.put( diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java index 1f8cbfe19152..4ecbd1c12921 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java @@ -23,31 +23,47 @@ import static org.apache.iceberg.flink.SimpleDataUtil.createRecord; import static org.apache.iceberg.flink.SimpleDataUtil.createUpdateAfter; import static org.apache.iceberg.flink.SimpleDataUtil.createUpdateBefore; +import static org.apache.iceberg.types.Types.NestedField.required; import java.io.File; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; +import java.time.OffsetDateTime; import java.util.Arrays; import java.util.List; import java.util.stream.Collectors; +import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.LocalZonedTimestampType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.RowKind; +import org.apache.iceberg.ContentFile; import org.apache.iceberg.FileContent; import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.RowDelta; +import org.apache.iceberg.Schema; import org.apache.iceberg.SerializableTable; import org.apache.iceberg.TableProperties; import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestTables; +import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.io.TaskWriter; import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.Types; import org.apache.iceberg.util.StructLikeSet; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -79,20 +95,6 @@ public void setupTable() throws IOException { this.metadataDir = new File(tableDir, "metadata"); } - private void initTable(boolean partitioned) { - if (partitioned) { - this.table = create(SCHEMA, PartitionSpec.builderFor(SCHEMA).identity("data").build()); - } else { - this.table = create(SCHEMA, PartitionSpec.unpartitioned()); - } - - table - .updateProperties() - .set(TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, String.valueOf(8 * 1024)) - .defaultFormat(format) - .commit(); - } - private int idFieldId() { return table.schema().findField("id").fieldId(); } @@ -170,18 +172,18 @@ private void testCdcEvents(boolean partitioned) throws IOException { @Test public void testUnpartitioned() throws IOException { - initTable(false); + createAndInitTable(false); testCdcEvents(false); } @Test public void testPartitioned() throws IOException { - initTable(true); + createAndInitTable(true); testCdcEvents(true); } private void testWritePureEqDeletes(boolean partitioned) throws IOException { - initTable(partitioned); + createAndInitTable(partitioned); List equalityFieldIds = Lists.newArrayList(idFieldId()); TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); taskWriterFactory.initialize(1, 1); @@ -210,7 +212,7 @@ public void testPartitionedPureEqDeletes() throws IOException { } private void testAbort(boolean partitioned) throws IOException { - initTable(partitioned); + createAndInitTable(partitioned); List equalityFieldIds = Lists.newArrayList(idFieldId()); TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); taskWriterFactory.initialize(1, 1); @@ -253,7 +255,7 @@ public void testPartitionedAbort() throws IOException { @Test public void testPartitionedTableWithDataAsKey() throws IOException { - initTable(true); + createAndInitTable(true); List equalityFieldIds = Lists.newArrayList(dataFieldId()); TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); taskWriterFactory.initialize(1, 1); @@ -298,7 +300,7 @@ public void testPartitionedTableWithDataAsKey() throws IOException { @Test public void testPartitionedTableWithDataAndIdAsKey() throws IOException { - initTable(true); + createAndInitTable(true); List equalityFieldIds = Lists.newArrayList(dataFieldId(), idFieldId()); TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); taskWriterFactory.initialize(1, 1); @@ -321,6 +323,62 @@ public void testPartitionedTableWithDataAndIdAsKey() throws IOException { "Should have expected records", expectedRowSet(createRecord(1, "aaa")), actualRowSet("*")); } + @Test + public void testEqualityColumnOnCustomPrecisionTSColumn() throws IOException { + Schema tableSchema = + new Schema( + required(3, "id", Types.IntegerType.get()), + required(4, "ts", Types.TimestampType.withZone())); + RowType flinkType = + new RowType( + false, + ImmutableList.of( + new RowType.RowField("id", new IntType()), + new RowType.RowField("ts", new LocalZonedTimestampType(3)))); + + this.table = create(tableSchema, PartitionSpec.unpartitioned()); + initTable(table); + + List equalityIds = ImmutableList.of(table.schema().findField("ts").fieldId()); + TaskWriterFactory taskWriterFactory = createTaskWriterFactory(flinkType, equalityIds); + taskWriterFactory.initialize(1, 1); + + TaskWriter writer = taskWriterFactory.create(); + RowDataSerializer serializer = new RowDataSerializer(flinkType); + OffsetDateTime start = OffsetDateTime.now(); + writer.write( + serializer.toBinaryRow( + GenericRowData.ofKind( + RowKind.INSERT, 1, TimestampData.fromInstant(start.toInstant())))); + writer.write( + serializer.toBinaryRow( + GenericRowData.ofKind( + RowKind.INSERT, 2, TimestampData.fromInstant(start.plusSeconds(1).toInstant())))); + writer.write( + serializer.toBinaryRow( + GenericRowData.ofKind( + RowKind.DELETE, 2, TimestampData.fromInstant(start.plusSeconds(1).toInstant())))); + + WriteResult result = writer.complete(); + // One data file + Assertions.assertThat(result.dataFiles().length).isEqualTo(1); + // One eq delete file + one pos delete file + Assertions.assertThat(result.deleteFiles().length).isEqualTo(2); + Assertions.assertThat( + Arrays.stream(result.deleteFiles()) + .map(ContentFile::content) + .collect(Collectors.toSet())) + .isEqualTo(Sets.newHashSet(FileContent.POSITION_DELETES, FileContent.EQUALITY_DELETES)); + commitTransaction(result); + + Record expectedRecord = GenericRecord.create(tableSchema); + expectedRecord.setField("id", 1); + int cutPrecisionNano = start.getNano() / 1000000 * 1000000; + expectedRecord.setField("ts", start.withNano(cutPrecisionNano)); + + Assertions.assertThat(actualRowSet("*")).isEqualTo(expectedRowSet(expectedRecord)); + } + private void commitTransaction(WriteResult result) { RowDelta rowDelta = table.newRowDelta(); Arrays.stream(result.dataFiles()).forEach(rowDelta::addRows); @@ -349,4 +407,34 @@ private TaskWriterFactory createTaskWriterFactory(List equalit equalityFieldIds, false); } + + private TaskWriterFactory createTaskWriterFactory( + RowType flinkType, List equalityFieldIds) { + return new RowDataTaskWriterFactory( + SerializableTable.copyOf(table), + flinkType, + 128 * 1024 * 1024, + format, + table.properties(), + equalityFieldIds, + true); + } + + private void createAndInitTable(boolean partitioned) { + if (partitioned) { + this.table = create(SCHEMA, PartitionSpec.builderFor(SCHEMA).identity("data").build()); + } else { + this.table = create(SCHEMA, PartitionSpec.unpartitioned()); + } + + initTable(table); + } + + private void initTable(TestTables.TestTable testTable) { + testTable + .updateProperties() + .set(TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, String.valueOf(8 * 1024)) + .defaultFormat(format) + .commit(); + } } diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java index 8dc68aad10aa..3a8071523b7c 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java @@ -20,19 +20,21 @@ import java.io.File; import java.util.List; -import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.BaseCombinedScanTask; import org.apache.iceberg.FileFormat; import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.ThreadPools; import org.junit.Assert; @@ -40,8 +42,6 @@ public class SplitHelpers { - private static final AtomicLong splitLengthIncrement = new AtomicLong(); - private SplitHelpers() {} /** @@ -54,16 +54,53 @@ private SplitHelpers() {} * *

    Since the table and data files are deleted before this method return, caller shouldn't * attempt to read the data files. + * + *

    By default, v1 Iceberg table is created. For v2 table use {@link + * SplitHelpers#createSplitsFromTransientHadoopTable(TemporaryFolder, int, int, String)} + * + * @param temporaryFolder Folder to place the data to + * @param fileCount The number of files to create and add to the table + * @param filesPerSplit The number of files used for a split */ public static List createSplitsFromTransientHadoopTable( TemporaryFolder temporaryFolder, int fileCount, int filesPerSplit) throws Exception { + return createSplitsFromTransientHadoopTable(temporaryFolder, fileCount, filesPerSplit, "1"); + } + + /** + * This create a list of IcebergSourceSplit from real files + *

  • Create a new Hadoop table under the {@code temporaryFolder} + *
  • write {@code fileCount} number of files to the new Iceberg table + *
  • Discover the splits from the table and partition the splits by the {@code filePerSplit} + * limit + *
  • Delete the Hadoop table + * + *

    Since the table and data files are deleted before this method return, caller shouldn't + * attempt to read the data files. + * + * @param temporaryFolder Folder to place the data to + * @param fileCount The number of files to create and add to the table + * @param filesPerSplit The number of files used for a split + * @param version The table version to create + */ + public static List createSplitsFromTransientHadoopTable( + TemporaryFolder temporaryFolder, int fileCount, int filesPerSplit, String version) + throws Exception { final File warehouseFile = temporaryFolder.newFolder(); Assert.assertTrue(warehouseFile.delete()); final String warehouse = "file:" + warehouseFile; Configuration hadoopConf = new Configuration(); final HadoopCatalog catalog = new HadoopCatalog(hadoopConf, warehouse); + ImmutableMap properties = + ImmutableMap.of(TableProperties.FORMAT_VERSION, version); try { - final Table table = catalog.createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + final Table table = + catalog.createTable( + TestFixtures.TABLE_IDENTIFIER, + TestFixtures.SCHEMA, + PartitionSpec.unpartitioned(), + null, + properties); final GenericAppenderHelper dataAppender = new GenericAppenderHelper(table, FileFormat.PARQUET, temporaryFolder); for (int i = 0; i < fileCount; ++i) { diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestSimpleSplitAssigner.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java similarity index 88% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestSimpleSplitAssigner.java rename to flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java index ee6c5cc3a6c8..f28677ca9d6a 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestSimpleSplitAssigner.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java @@ -30,22 +30,24 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; -public class TestSimpleSplitAssigner { +public abstract class SplitAssignerTestBase { @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); @Test public void testEmptyInitialization() { - SimpleSplitAssigner assigner = new SimpleSplitAssigner(); + SplitAssigner assigner = splitAssigner(); assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); } /** Test a sequence of interactions for StaticEnumerator */ @Test public void testStaticEnumeratorSequence() throws Exception { - SimpleSplitAssigner assigner = new SimpleSplitAssigner(); + SplitAssigner assigner = splitAssigner(); assigner.onDiscoveredSplits( - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 4, 2)); + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 4, 1)); + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); assertSnapshot(assigner, 1); assigner.onUnassignedSplits( @@ -61,7 +63,7 @@ public void testStaticEnumeratorSequence() throws Exception { /** Test a sequence of interactions for ContinuousEnumerator */ @Test public void testContinuousEnumeratorSequence() throws Exception { - SimpleSplitAssigner assigner = new SimpleSplitAssigner(); + SplitAssigner assigner = splitAssigner(); assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); List splits1 = @@ -81,7 +83,7 @@ public void testContinuousEnumeratorSequence() throws Exception { } private void assertAvailableFuture( - SimpleSplitAssigner assigner, int splitCount, Runnable addSplitsRunnable) { + SplitAssigner assigner, int splitCount, Runnable addSplitsRunnable) { // register callback AtomicBoolean futureCompleted = new AtomicBoolean(); CompletableFuture future = assigner.isAvailable(); @@ -102,7 +104,7 @@ private void assertAvailableFuture( assertSnapshot(assigner, 0); } - private void assertGetNext(SimpleSplitAssigner assigner, GetSplitResult.Status expectedStatus) { + protected void assertGetNext(SplitAssigner assigner, GetSplitResult.Status expectedStatus) { GetSplitResult result = assigner.getNext(null); Assert.assertEquals(expectedStatus, result.status()); switch (expectedStatus) { @@ -118,8 +120,10 @@ private void assertGetNext(SimpleSplitAssigner assigner, GetSplitResult.Status e } } - private void assertSnapshot(SimpleSplitAssigner assigner, int splitCount) { + protected void assertSnapshot(SplitAssigner assigner, int splitCount) { Collection stateBeforeGet = assigner.state(); Assert.assertEquals(splitCount, stateBeforeGet.size()); } + + protected abstract SplitAssigner splitAssigner(); } diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java new file mode 100644 index 000000000000..8994f3054abe --- /dev/null +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.assigner; + +import org.apache.iceberg.flink.source.SplitHelpers; +import org.junit.Test; + +public class TestDefaultSplitAssigner extends SplitAssignerTestBase { + @Override + protected SplitAssigner splitAssigner() { + return new DefaultSplitAssigner(null); + } + + /** Test the assigner when multiple files are in a single split */ + @Test + public void testMultipleFilesInASplit() throws Exception { + SplitAssigner assigner = splitAssigner(); + assigner.onDiscoveredSplits( + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 4, 2)); + + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertSnapshot(assigner, 1); + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + assertSnapshot(assigner, 0); + } +} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java new file mode 100644 index 000000000000..8b9e132e0e22 --- /dev/null +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.assigner; + +import java.util.List; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.flink.source.SplitHelpers; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.SerializableComparator; +import org.apache.iceberg.flink.source.split.SplitComparators; +import org.apache.iceberg.util.SerializationUtil; +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Test; + +public class TestFileSequenceNumberBasedSplitAssigner extends SplitAssignerTestBase { + @Override + protected SplitAssigner splitAssigner() { + return new OrderedSplitAssignerFactory(SplitComparators.fileSequenceNumber()).createAssigner(); + } + + /** Test the assigner when multiple files are in a single split */ + @Test + public void testMultipleFilesInAnIcebergSplit() { + SplitAssigner assigner = splitAssigner(); + Assertions.assertThatThrownBy( + () -> + assigner.onDiscoveredSplits( + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 4, 2, "2")), + "Multiple files in a split is not allowed") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Please use 'split-open-file-cost'"); + } + + /** Test sorted splits */ + @Test + public void testSplitSort() throws Exception { + SplitAssigner assigner = splitAssigner(); + List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 5, 1, "2"); + + assigner.onDiscoveredSplits(splits.subList(3, 5)); + assigner.onDiscoveredSplits(splits.subList(0, 1)); + assigner.onDiscoveredSplits(splits.subList(1, 3)); + + assertGetNext(assigner, 1L); + assertGetNext(assigner, 2L); + assertGetNext(assigner, 3L); + assertGetNext(assigner, 4L); + assertGetNext(assigner, 5L); + + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + } + + @Test + public void testSerializable() { + byte[] bytes = SerializationUtil.serializeToBytes(SplitComparators.fileSequenceNumber()); + SerializableComparator comparator = + SerializationUtil.deserializeFromBytes(bytes); + Assert.assertNotNull(comparator); + } + + protected void assertGetNext(SplitAssigner assigner, Long expectedSequenceNumber) { + GetSplitResult result = assigner.getNext(null); + ContentFile file = result.split().task().files().iterator().next().file(); + Assert.assertEquals(expectedSequenceNumber, file.fileSequenceNumber()); + } +} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java index d0ae8fdf7748..349eb11cf549 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java @@ -28,7 +28,7 @@ import org.apache.iceberg.flink.source.ScanContext; import org.apache.iceberg.flink.source.SplitHelpers; import org.apache.iceberg.flink.source.StreamingStartingStrategy; -import org.apache.iceberg.flink.source.assigner.SimpleSplitAssigner; +import org.apache.iceberg.flink.source.assigner.DefaultSplitAssigner; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; @@ -342,7 +342,7 @@ private static ContinuousIcebergEnumerator createEnumerator( ContinuousIcebergEnumerator enumerator = new ContinuousIcebergEnumerator( context, - new SimpleSplitAssigner(Collections.emptyList()), + new DefaultSplitAssigner(null, Collections.emptyList()), scanContext, splitPlanner, null); diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java index 56af0caf1298..def4f436851b 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java @@ -35,6 +35,7 @@ import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.TestHelpers; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.SerializableComparator; import org.apache.iceberg.hadoop.HadoopFileIO; import org.junit.Assert; import org.junit.ClassRule; @@ -55,13 +56,68 @@ public void testReaderMetrics() throws Exception { TestingReaderOutput readerOutput = new TestingReaderOutput<>(); TestingMetricGroup metricGroup = new TestingMetricGroup(); TestingReaderContext readerContext = new TestingReaderContext(new Configuration(), metricGroup); - IcebergSourceReader reader = createReader(metricGroup, readerContext); + IcebergSourceReader reader = createReader(metricGroup, readerContext, null); reader.start(); testOneSplitFetcher(reader, readerOutput, metricGroup, 1); testOneSplitFetcher(reader, readerOutput, metricGroup, 2); } + @Test + public void testReaderOrder() throws Exception { + // Create 2 splits + List> recordBatchList1 = + ReaderUtil.createRecordBatchList(0L, TestFixtures.SCHEMA, 1, 1); + CombinedScanTask task1 = + ReaderUtil.createCombinedScanTask( + recordBatchList1, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory); + + List> recordBatchList2 = + ReaderUtil.createRecordBatchList(1L, TestFixtures.SCHEMA, 1, 1); + CombinedScanTask task2 = + ReaderUtil.createCombinedScanTask( + recordBatchList2, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory); + + // Sort the splits in one way + List rowDataList1 = + read( + Arrays.asList( + IcebergSourceSplit.fromCombinedScanTask(task1), + IcebergSourceSplit.fromCombinedScanTask(task2)), + 2); + + // Reverse the splits + List rowDataList2 = + read( + Arrays.asList( + IcebergSourceSplit.fromCombinedScanTask(task2), + IcebergSourceSplit.fromCombinedScanTask(task1)), + 2); + + // Check that the order of the elements is not changed + Assert.assertEquals(rowDataList1.get(0), rowDataList2.get(0)); + Assert.assertEquals(rowDataList1.get(1), rowDataList2.get(1)); + } + + private List read(List splits, long expected) throws Exception { + TestingMetricGroup metricGroup = new TestingMetricGroup(); + TestingReaderContext readerContext = new TestingReaderContext(new Configuration(), metricGroup); + // Using IdBasedComparator, so we can have a deterministic order of the splits + IcebergSourceReader reader = createReader(metricGroup, readerContext, new IdBasedComparator()); + reader.start(); + + reader.addSplits(splits); + TestingReaderOutput readerOutput = new TestingReaderOutput<>(); + while (readerOutput.getEmittedRecords().size() < expected) { + reader.pollNext(readerOutput); + } + + reader.pollNext(readerOutput); + + Assert.assertEquals(expected, readerOutput.getEmittedRecords().size()); + return readerOutput.getEmittedRecords(); + } + private void testOneSplitFetcher( IcebergSourceReader reader, TestingReaderOutput readerOutput, @@ -96,7 +152,9 @@ private void testOneSplitFetcher( } private IcebergSourceReader createReader( - MetricGroup metricGroup, SourceReaderContext readerContext) { + MetricGroup metricGroup, + SourceReaderContext readerContext, + SerializableComparator splitComparator) { IcebergSourceReaderMetrics readerMetrics = new IcebergSourceReaderMetrics(metricGroup, "db.tbl"); RowDataReaderFunction readerFunction = @@ -109,6 +167,13 @@ private IcebergSourceReader createReader( new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), new PlaintextEncryptionManager(), Collections.emptyList()); - return new IcebergSourceReader<>(readerMetrics, readerFunction, readerContext); + return new IcebergSourceReader<>(readerMetrics, readerFunction, splitComparator, readerContext); + } + + private static class IdBasedComparator implements SerializableComparator { + @Override + public int compare(IcebergSourceSplit o1, IcebergSourceSplit o2) { + return o1.splitId().compareTo(o2.splitId()); + } } } diff --git a/flink/v1.17/build.gradle b/flink/v1.17/build.gradle index 3ecced90d3df..be34348a8de9 100644 --- a/flink/v1.17/build.gradle +++ b/flink/v1.17/build.gradle @@ -112,6 +112,8 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { exclude group: 'javax.transaction', module: 'transaction-api' exclude group: 'com.zaxxer', module: 'HikariCP' } + + testImplementation 'org.awaitility:awaitility' } } @@ -238,6 +240,7 @@ project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") { task integrationTest(type: Test) { description = "Test Flink Runtime Jar against Flink ${flinkMajorVersion}" group = "verification" + jvmArgs += project.property('extraJvmArgs') testClassesDirs = sourceSets.integration.output.classesDirs classpath = sourceSets.integration.runtimeClasspath + files(shadowJar.archiveFile.get().asFile.path) inputs.file(shadowJar.archiveFile.get().asFile.path) diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java index 3ecddaa05c9b..825816fdf416 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java @@ -124,14 +124,7 @@ public FlinkCatalog( } @Override - public void open() throws CatalogException { - // Create the default database if it does not exist. - try { - createDatabase(getDefaultDatabase(), ImmutableMap.of(), true); - } catch (DatabaseAlreadyExistException e) { - // Ignore the exception if it's already exist. - } - } + public void open() throws CatalogException {} @Override public void close() throws CatalogException { diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java index 3f04d35e851c..8e1f420b722d 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java @@ -105,18 +105,21 @@ public DynamicTableSource createDynamicTableSource(Context context) { @Override public DynamicTableSink createDynamicTableSink(Context context) { - ObjectPath objectPath = context.getObjectIdentifier().toObjectPath(); + ObjectIdentifier objectIdentifier = context.getObjectIdentifier(); CatalogTable catalogTable = context.getCatalogTable(); Map writeProps = catalogTable.getOptions(); TableSchema tableSchema = TableSchemaUtils.getPhysicalSchema(catalogTable.getSchema()); TableLoader tableLoader; if (catalog != null) { - tableLoader = createTableLoader(catalog, objectPath); + tableLoader = createTableLoader(catalog, objectIdentifier.toObjectPath()); } else { tableLoader = createTableLoader( - catalogTable, writeProps, objectPath.getDatabaseName(), objectPath.getObjectName()); + catalogTable, + writeProps, + objectIdentifier.getDatabaseName(), + objectIdentifier.getObjectName()); } return new IcebergTableSink(tableLoader, tableSchema, context.getConfiguration(), writeProps); diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java index b1ffda156011..40e0b5f2a34e 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java @@ -63,7 +63,8 @@ abstract class BaseDeltaTaskWriter extends BaseTaskWriter { this.wrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); this.keyWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(deleteSchema), deleteSchema.asStruct()); - this.keyProjection = RowDataProjection.create(schema, deleteSchema); + this.keyProjection = + RowDataProjection.create(flinkSchema, schema.asStruct(), deleteSchema.asStruct()); this.upsert = upsert; } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java index cbdd18487069..f85f6277263b 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -48,6 +48,8 @@ import org.apache.iceberg.flink.FlinkReadOptions; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.source.assigner.OrderedSplitAssignerFactory; +import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; import org.apache.iceberg.flink.source.assigner.SplitAssigner; import org.apache.iceberg.flink.source.assigner.SplitAssignerFactory; import org.apache.iceberg.flink.source.enumerator.ContinuousIcebergEnumerator; @@ -63,6 +65,7 @@ import org.apache.iceberg.flink.source.reader.RowDataReaderFunction; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer; +import org.apache.iceberg.flink.source.split.SerializableComparator; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.util.ThreadPools; import org.slf4j.Logger; @@ -76,6 +79,7 @@ public class IcebergSource implements Source readerFunction; private final SplitAssignerFactory assignerFactory; + private final SerializableComparator splitComparator; // Can't use SerializableTable as enumerator needs a regular table // that can discover table changes @@ -86,11 +90,13 @@ public class IcebergSource implements Source readerFunction, SplitAssignerFactory assignerFactory, + SerializableComparator splitComparator, Table table) { this.tableLoader = tableLoader; this.scanContext = scanContext; this.readerFunction = readerFunction; this.assignerFactory = assignerFactory; + this.splitComparator = splitComparator; this.table = table; } @@ -146,7 +152,7 @@ public Boundedness getBoundedness() { public SourceReader createReader(SourceReaderContext readerContext) { IcebergSourceReaderMetrics metrics = new IcebergSourceReaderMetrics(readerContext.metricGroup(), lazyTable().name()); - return new IcebergSourceReader<>(metrics, readerFunction, readerContext); + return new IcebergSourceReader<>(metrics, readerFunction, splitComparator, readerContext); } @Override @@ -163,12 +169,12 @@ public SplitEnumerator restoreEnumer @Override public SimpleVersionedSerializer getSplitSerializer() { - return IcebergSourceSplitSerializer.INSTANCE; + return new IcebergSourceSplitSerializer(scanContext.caseSensitive()); } @Override public SimpleVersionedSerializer getEnumeratorCheckpointSerializer() { - return IcebergEnumeratorStateSerializer.INSTANCE; + return new IcebergEnumeratorStateSerializer(scanContext.caseSensitive()); } private SplitEnumerator createEnumerator( @@ -209,6 +215,7 @@ public static class Builder { private TableLoader tableLoader; private Table table; private SplitAssignerFactory splitAssignerFactory; + private SerializableComparator splitComparator; private ReaderFunction readerFunction; private ReadableConfig flinkConfig = new Configuration(); private final ScanContext.Builder contextBuilder = ScanContext.builder(); @@ -234,6 +241,12 @@ public Builder assignerFactory(SplitAssignerFactory assignerFactory) { return this; } + public Builder splitComparator( + SerializableComparator newSplitComparator) { + this.splitComparator = newSplitComparator; + return this; + } + public Builder readerFunction(ReaderFunction newReaderFunction) { this.readerFunction = newReaderFunction; return this; @@ -462,10 +475,18 @@ public IcebergSource build() { } } + if (splitAssignerFactory == null) { + if (splitComparator == null) { + splitAssignerFactory = new SimpleSplitAssignerFactory(); + } else { + splitAssignerFactory = new OrderedSplitAssignerFactory(splitComparator); + } + } + checkRequired(); // Since builder already load the table, pass it to the source to avoid double loading return new IcebergSource( - tableLoader, context, readerFunction, splitAssignerFactory, table); + tableLoader, context, readerFunction, splitAssignerFactory, splitComparator, table); } private void checkRequired() { diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssigner.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java similarity index 82% rename from flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssigner.java rename to flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java index b43660f907a6..37a0f1a6055f 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssigner.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java @@ -20,7 +20,8 @@ import java.util.ArrayDeque; import java.util.Collection; -import java.util.Deque; +import java.util.PriorityQueue; +import java.util.Queue; import java.util.concurrent.CompletableFuture; import java.util.stream.Collectors; import javax.annotation.Nullable; @@ -28,24 +29,27 @@ import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; +import org.apache.iceberg.flink.source.split.SerializableComparator; /** * Since all methods are called in the source coordinator thread by enumerator, there is no need for * locking. */ @Internal -public class SimpleSplitAssigner implements SplitAssigner { +public class DefaultSplitAssigner implements SplitAssigner { - private final Deque pendingSplits; + private final Queue pendingSplits; private CompletableFuture availableFuture; - public SimpleSplitAssigner() { - this.pendingSplits = new ArrayDeque<>(); + public DefaultSplitAssigner(SerializableComparator comparator) { + this.pendingSplits = comparator == null ? new ArrayDeque<>() : new PriorityQueue<>(comparator); } - public SimpleSplitAssigner(Collection assignerState) { - this.pendingSplits = new ArrayDeque<>(assignerState.size()); - // Because simple assigner only tracks unassigned splits, + public DefaultSplitAssigner( + SerializableComparator comparator, + Collection assignerState) { + this(comparator); + // Because default assigner only tracks unassigned splits, // there is no need to filter splits based on status (unassigned) here. assignerState.forEach(splitState -> pendingSplits.add(splitState.split())); } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java new file mode 100644 index 000000000000..e58478897aef --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.assigner; + +import java.util.Collection; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; +import org.apache.iceberg.flink.source.split.SerializableComparator; + +/** + * Create default assigner with a comparator that hands out splits where the order of the splits + * will be defined by the {@link SerializableComparator}. + */ +public class OrderedSplitAssignerFactory implements SplitAssignerFactory { + private final SerializableComparator comparator; + + public OrderedSplitAssignerFactory(SerializableComparator comparator) { + this.comparator = comparator; + } + + @Override + public SplitAssigner createAssigner() { + return new DefaultSplitAssigner(comparator); + } + + @Override + public SplitAssigner createAssigner(Collection assignerState) { + return new DefaultSplitAssigner(comparator, assignerState); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java index 1c14f4fcf9b9..a2e2ff364d46 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java @@ -23,14 +23,15 @@ /** Create simple assigner that hands out splits without any guarantee in order or locality. */ public class SimpleSplitAssignerFactory implements SplitAssignerFactory { + public SimpleSplitAssignerFactory() {} @Override - public SimpleSplitAssigner createAssigner() { - return new SimpleSplitAssigner(); + public SplitAssigner createAssigner() { + return new DefaultSplitAssigner(null); } @Override - public SimpleSplitAssigner createAssigner(Collection assignerState) { - return new SimpleSplitAssigner(assignerState); + public SplitAssigner createAssigner(Collection assignerState) { + return new DefaultSplitAssigner(null, assignerState); } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java index 9998bee99fef..95d6db2cfbc4 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java @@ -35,9 +35,6 @@ public class IcebergEnumeratorStateSerializer implements SimpleVersionedSerializer { - public static final IcebergEnumeratorStateSerializer INSTANCE = - new IcebergEnumeratorStateSerializer(); - private static final int VERSION = 2; private static final ThreadLocal SERIALIZER_CACHE = @@ -45,8 +42,11 @@ public class IcebergEnumeratorStateSerializer private final IcebergEnumeratorPositionSerializer positionSerializer = IcebergEnumeratorPositionSerializer.INSTANCE; - private final IcebergSourceSplitSerializer splitSerializer = - IcebergSourceSplitSerializer.INSTANCE; + private final IcebergSourceSplitSerializer splitSerializer; + + public IcebergEnumeratorStateSerializer(boolean caseSensitive) { + this.splitSerializer = new IcebergSourceSplitSerializer(caseSensitive); + } @Override public int getVersion() { diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java index 84c98055cebc..8d7d68f961cb 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java @@ -25,6 +25,7 @@ import org.apache.flink.api.connector.source.SourceReaderContext; import org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBase; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.SerializableComparator; import org.apache.iceberg.flink.source.split.SplitRequestEvent; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -36,9 +37,10 @@ public class IcebergSourceReader public IcebergSourceReader( IcebergSourceReaderMetrics metrics, ReaderFunction readerFunction, + SerializableComparator splitComparator, SourceReaderContext context) { super( - () -> new IcebergSourceSplitReader<>(metrics, readerFunction, context), + () -> new IcebergSourceSplitReader<>(metrics, readerFunction, splitComparator, context), new IcebergSourceRecordEmitter<>(), context.getConfiguration(), context); diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java index bb016671e694..4e270dfa3d13 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java @@ -22,6 +22,7 @@ import java.io.UncheckedIOException; import java.util.ArrayDeque; import java.util.Collections; +import java.util.List; import java.util.Queue; import org.apache.flink.api.connector.source.SourceReaderContext; import org.apache.flink.connector.base.source.reader.RecordsBySplits; @@ -31,7 +32,9 @@ import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.SerializableComparator; import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,6 +43,7 @@ class IcebergSourceSplitReader implements SplitReader, I private final IcebergSourceReaderMetrics metrics; private final ReaderFunction openSplitFunction; + private final SerializableComparator splitComparator; private final int indexOfSubtask; private final Queue splits; @@ -50,9 +54,11 @@ class IcebergSourceSplitReader implements SplitReader, I IcebergSourceSplitReader( IcebergSourceReaderMetrics metrics, ReaderFunction openSplitFunction, + SerializableComparator splitComparator, SourceReaderContext context) { this.metrics = metrics; this.openSplitFunction = openSplitFunction; + this.splitComparator = splitComparator; this.indexOfSubtask = context.getIndexOfSubtask(); this.splits = new ArrayDeque<>(); } @@ -93,8 +99,15 @@ public void handleSplitsChanges(SplitsChange splitsChange) { String.format("Unsupported split change: %s", splitsChange.getClass())); } - LOG.info("Add {} splits to reader", splitsChange.splits().size()); - splits.addAll(splitsChange.splits()); + if (splitComparator != null) { + List newSplits = Lists.newArrayList(splitsChange.splits()); + newSplits.sort(splitComparator); + LOG.info("Add {} splits to reader: {}", newSplits.size(), newSplits); + splits.addAll(newSplits); + } else { + LOG.info("Add {} splits to reader", splitsChange.splits().size()); + splits.addAll(splitsChange.splits()); + } metrics.incrementAssignedSplits(splitsChange.splits().size()); metrics.incrementAssignedBytes(calculateBytes(splitsChange)); } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java index 35f8ade9843d..e4bfbf1452e2 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java @@ -21,19 +21,28 @@ import java.io.IOException; import java.io.Serializable; import java.util.Collection; +import java.util.List; import java.util.stream.Collectors; import javax.annotation.Nullable; import org.apache.flink.annotation.Internal; import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; import org.apache.flink.util.InstantiationUtil; +import org.apache.iceberg.BaseCombinedScanTask; import org.apache.iceberg.CombinedScanTask; import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.FileScanTaskParser; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; @Internal public class IcebergSourceSplit implements SourceSplit, Serializable { private static final long serialVersionUID = 1L; + private static final ThreadLocal SERIALIZER_CACHE = + ThreadLocal.withInitial(() -> new DataOutputSerializer(1024)); private final CombinedScanTask task; @@ -109,6 +118,7 @@ byte[] serializeV1() throws IOException { if (serializedBytesCache == null) { serializedBytesCache = InstantiationUtil.serializeObject(this); } + return serializedBytesCache; } @@ -120,4 +130,48 @@ static IcebergSourceSplit deserializeV1(byte[] serialized) throws IOException { throw new RuntimeException("Failed to deserialize the split.", e); } } + + byte[] serializeV2() throws IOException { + if (serializedBytesCache == null) { + DataOutputSerializer out = SERIALIZER_CACHE.get(); + Collection fileScanTasks = task.tasks(); + Preconditions.checkArgument( + fileOffset >= 0 && fileOffset < fileScanTasks.size(), + "Invalid file offset: %s. Should be within the range of [0, %s)", + fileOffset, + fileScanTasks.size()); + + out.writeInt(fileOffset); + out.writeLong(recordOffset); + out.writeInt(fileScanTasks.size()); + + for (FileScanTask fileScanTask : fileScanTasks) { + String taskJson = FileScanTaskParser.toJson(fileScanTask); + out.writeUTF(taskJson); + } + + serializedBytesCache = out.getCopyOfBuffer(); + out.clear(); + } + + return serializedBytesCache; + } + + static IcebergSourceSplit deserializeV2(byte[] serialized, boolean caseSensitive) + throws IOException { + DataInputDeserializer in = new DataInputDeserializer(serialized); + int fileOffset = in.readInt(); + long recordOffset = in.readLong(); + int taskCount = in.readInt(); + + List tasks = Lists.newArrayListWithCapacity(taskCount); + for (int i = 0; i < taskCount; ++i) { + String taskJson = in.readUTF(); + FileScanTask task = FileScanTaskParser.fromJson(taskJson, caseSensitive); + tasks.add(task); + } + + CombinedScanTask combinedScanTask = new BaseCombinedScanTask(tasks); + return IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, fileOffset, recordOffset); + } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java index ee0f364e17d6..8c089819e731 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java @@ -22,14 +22,15 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.core.io.SimpleVersionedSerializer; -/** - * TODO: use Java serialization for now. Will switch to more stable serializer from issue-1698. - */ @Internal public class IcebergSourceSplitSerializer implements SimpleVersionedSerializer { - public static final IcebergSourceSplitSerializer INSTANCE = new IcebergSourceSplitSerializer(); - private static final int VERSION = 1; + private static final int VERSION = 2; + + private final boolean caseSensitive; + + public IcebergSourceSplitSerializer(boolean caseSensitive) { + this.caseSensitive = caseSensitive; + } @Override public int getVersion() { @@ -38,7 +39,7 @@ public int getVersion() { @Override public byte[] serialize(IcebergSourceSplit split) throws IOException { - return split.serializeV1(); + return split.serializeV2(); } @Override @@ -46,6 +47,8 @@ public IcebergSourceSplit deserialize(int version, byte[] serialized) throws IOE switch (version) { case 1: return IcebergSourceSplit.deserializeV1(serialized); + case 2: + return IcebergSourceSplit.deserializeV2(serialized, caseSensitive); default: throw new IOException( String.format( diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java new file mode 100644 index 000000000000..319648ca275c --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.split; + +import java.io.Serializable; +import java.util.Comparator; + +public interface SerializableComparator extends Comparator, Serializable {} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java new file mode 100644 index 000000000000..64e03d77debe --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.split; + +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * Provides implementations of {@link org.apache.iceberg.flink.source.split.SerializableComparator} + * which could be used for ordering splits. These are used by the {@link + * org.apache.iceberg.flink.source.assigner.OrderedSplitAssignerFactory} and the {@link + * org.apache.iceberg.flink.source.reader.IcebergSourceReader} + */ +public class SplitComparators { + private SplitComparators() {} + + /** Comparator which orders the splits based on the file sequence number of the data files */ + public static SerializableComparator fileSequenceNumber() { + return (IcebergSourceSplit o1, IcebergSourceSplit o2) -> { + Preconditions.checkArgument( + o1.task().files().size() == 1 && o2.task().files().size() == 1, + "Could not compare combined task. Please use 'split-open-file-cost' to prevent combining multiple files to a split"); + + Long seq1 = o1.task().files().iterator().next().file().fileSequenceNumber(); + Long seq2 = o2.task().files().iterator().next().file().fileSequenceNumber(); + + Preconditions.checkNotNull( + seq1, + "Invalid file sequence number: null. Doesn't support splits written with V1 format: %s", + o1); + Preconditions.checkNotNull( + seq2, + "IInvalid file sequence number: null. Doesn't support splits written with V1 format: %s", + o2); + + int temp = Long.compare(seq1, seq2); + if (temp != 0) { + return temp; + } else { + return o1.splitId().compareTo(o2.splitId()); + } + }; + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java index 408c281004ed..5efb7413e70c 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java @@ -22,7 +22,6 @@ import java.io.IOException; import java.time.Duration; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -49,6 +48,7 @@ import org.apache.iceberg.TableScan; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.IcebergGenerics; +import org.apache.iceberg.data.InternalRecordWrapper; import org.apache.iceberg.data.Record; import org.apache.iceberg.deletes.EqualityDeleteWriter; import org.apache.iceberg.deletes.PositionDelete; @@ -349,7 +349,10 @@ public static void assertTableRecords(String tablePath, List expected, S public static StructLikeSet expectedRowSet(Table table, Record... records) { StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); - Collections.addAll(set, records); + InternalRecordWrapper wrapper = new InternalRecordWrapper(table.schema().asStruct()); + for (Record record : records) { + set.add(wrapper.copyFor(record)); + } return set; } @@ -361,12 +364,13 @@ public static StructLikeSet actualRowSet(Table table, Long snapshotId, String... throws IOException { table.refresh(); StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); + InternalRecordWrapper wrapper = new InternalRecordWrapper(table.schema().asStruct()); try (CloseableIterable reader = IcebergGenerics.read(table) .useSnapshot(snapshotId == null ? table.currentSnapshot().snapshotId() : snapshotId) .select(columns) .build()) { - reader.forEach(set::add); + reader.forEach(record -> set.add(wrapper.copyFor(record))); } return set; } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java new file mode 100644 index 000000000000..2fbd7627efab --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink; + +import java.io.File; +import java.util.concurrent.TimeUnit; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.Schema; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableDescriptor; +import org.apache.flink.table.api.TableEnvironment; +import org.assertj.core.api.Assertions; +import org.awaitility.Awaitility; +import org.junit.Test; + +public class TestFlinkAnonymousTable extends FlinkTestBase { + + @Test + public void testWriteAnonymousTable() throws Exception { + File warehouseDir = TEMPORARY_FOLDER.newFolder(); + TableEnvironment tEnv = getTableEnv(); + Table table = + tEnv.from( + TableDescriptor.forConnector("datagen") + .schema(Schema.newBuilder().column("f0", DataTypes.STRING()).build()) + .option("number-of-rows", "3") + .build()); + + TableDescriptor descriptor = + TableDescriptor.forConnector("iceberg") + .schema(Schema.newBuilder().column("f0", DataTypes.STRING()).build()) + .option("catalog-name", "hadoop_test") + .option("catalog-type", "hadoop") + .option("catalog-database", "test_db") + .option("catalog-table", "test") + .option("warehouse", warehouseDir.getAbsolutePath()) + .build(); + + table.insertInto(descriptor).execute(); + Awaitility.await() + .atMost(3, TimeUnit.SECONDS) + .untilAsserted( + () -> + Assertions.assertThat( + warehouseDir.toPath().resolve("test_db").resolve("test").toFile()) + .exists()); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java index f97ace69f8f9..47b47cb6262d 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java @@ -27,7 +27,6 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; import org.assertj.core.api.Assertions; import org.junit.After; @@ -74,19 +73,6 @@ public void testCreateNamespace() { "Database should be created", validationNamespaceCatalog.namespaceExists(icebergNamespace)); } - @Test - public void testDefaultDatabase() { - sql("USE CATALOG %s", catalogName); - sql("SHOW TABLES"); - - Assert.assertEquals( - "Should use the current catalog", getTableEnv().getCurrentCatalog(), catalogName); - Assert.assertEquals( - "Should use the configured default namespace", - getTableEnv().getCurrentDatabase(), - "default"); - } - @Test public void testDropEmptyDatabase() { Assert.assertFalse( @@ -174,22 +160,17 @@ public void testListNamespace() { List databases = sql("SHOW DATABASES"); if (isHadoopCatalog) { - Assert.assertEquals("Should have 2 database", 2, databases.size()); - Assert.assertEquals( - "Should have db and default database", - Sets.newHashSet("default", "db"), - Sets.newHashSet(databases.get(0).getField(0), databases.get(1).getField(0))); + Assert.assertEquals("Should have 1 database", 1, databases.size()); + Assert.assertEquals("Should have db database", "db", databases.get(0).getField(0)); if (!baseNamespace.isEmpty()) { // test namespace not belongs to this catalog validationNamespaceCatalog.createNamespace( Namespace.of(baseNamespace.level(0), "UNKNOWN_NAMESPACE")); databases = sql("SHOW DATABASES"); - Assert.assertEquals("Should have 2 database", 2, databases.size()); + Assert.assertEquals("Should have 1 database", 1, databases.size()); Assert.assertEquals( - "Should have db and default database", - Sets.newHashSet("default", "db"), - Sets.newHashSet(databases.get(0).getField(0), databases.get(1).getField(0))); + "Should have db and default database", "db", databases.get(0).getField(0)); } } else { // If there are multiple classes extends FlinkTestBase, TestHiveMetastore may loose the diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java index 184223847c74..ba08b76dd59d 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java @@ -42,7 +42,7 @@ public void before() { } @Test - public void testCreateCreateCatalogHive() { + public void testCreateCatalogHive() { String catalogName = "hiveCatalog"; props.put( FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HIVE); @@ -55,7 +55,7 @@ public void testCreateCreateCatalogHive() { } @Test - public void testCreateCreateCatalogHadoop() { + public void testCreateCatalogHadoop() { String catalogName = "hadoopCatalog"; props.put( FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HADOOP); @@ -68,7 +68,7 @@ public void testCreateCreateCatalogHadoop() { } @Test - public void testCreateCreateCatalogCustom() { + public void testCreateCatalogCustom() { String catalogName = "customCatalog"; props.put(CatalogProperties.CATALOG_IMPL, CustomHadoopCatalog.class.getName()); @@ -80,7 +80,7 @@ public void testCreateCreateCatalogCustom() { } @Test - public void testCreateCreateCatalogCustomWithHiveCatalogTypeSet() { + public void testCreateCatalogCustomWithHiveCatalogTypeSet() { String catalogName = "customCatalog"; props.put(CatalogProperties.CATALOG_IMPL, CustomHadoopCatalog.class.getName()); props.put( diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java index 1f8cbfe19152..4ecbd1c12921 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java @@ -23,31 +23,47 @@ import static org.apache.iceberg.flink.SimpleDataUtil.createRecord; import static org.apache.iceberg.flink.SimpleDataUtil.createUpdateAfter; import static org.apache.iceberg.flink.SimpleDataUtil.createUpdateBefore; +import static org.apache.iceberg.types.Types.NestedField.required; import java.io.File; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; +import java.time.OffsetDateTime; import java.util.Arrays; import java.util.List; import java.util.stream.Collectors; +import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.LocalZonedTimestampType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.RowKind; +import org.apache.iceberg.ContentFile; import org.apache.iceberg.FileContent; import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.RowDelta; +import org.apache.iceberg.Schema; import org.apache.iceberg.SerializableTable; import org.apache.iceberg.TableProperties; import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestTables; +import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.io.TaskWriter; import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.Types; import org.apache.iceberg.util.StructLikeSet; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -79,20 +95,6 @@ public void setupTable() throws IOException { this.metadataDir = new File(tableDir, "metadata"); } - private void initTable(boolean partitioned) { - if (partitioned) { - this.table = create(SCHEMA, PartitionSpec.builderFor(SCHEMA).identity("data").build()); - } else { - this.table = create(SCHEMA, PartitionSpec.unpartitioned()); - } - - table - .updateProperties() - .set(TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, String.valueOf(8 * 1024)) - .defaultFormat(format) - .commit(); - } - private int idFieldId() { return table.schema().findField("id").fieldId(); } @@ -170,18 +172,18 @@ private void testCdcEvents(boolean partitioned) throws IOException { @Test public void testUnpartitioned() throws IOException { - initTable(false); + createAndInitTable(false); testCdcEvents(false); } @Test public void testPartitioned() throws IOException { - initTable(true); + createAndInitTable(true); testCdcEvents(true); } private void testWritePureEqDeletes(boolean partitioned) throws IOException { - initTable(partitioned); + createAndInitTable(partitioned); List equalityFieldIds = Lists.newArrayList(idFieldId()); TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); taskWriterFactory.initialize(1, 1); @@ -210,7 +212,7 @@ public void testPartitionedPureEqDeletes() throws IOException { } private void testAbort(boolean partitioned) throws IOException { - initTable(partitioned); + createAndInitTable(partitioned); List equalityFieldIds = Lists.newArrayList(idFieldId()); TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); taskWriterFactory.initialize(1, 1); @@ -253,7 +255,7 @@ public void testPartitionedAbort() throws IOException { @Test public void testPartitionedTableWithDataAsKey() throws IOException { - initTable(true); + createAndInitTable(true); List equalityFieldIds = Lists.newArrayList(dataFieldId()); TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); taskWriterFactory.initialize(1, 1); @@ -298,7 +300,7 @@ public void testPartitionedTableWithDataAsKey() throws IOException { @Test public void testPartitionedTableWithDataAndIdAsKey() throws IOException { - initTable(true); + createAndInitTable(true); List equalityFieldIds = Lists.newArrayList(dataFieldId(), idFieldId()); TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); taskWriterFactory.initialize(1, 1); @@ -321,6 +323,62 @@ public void testPartitionedTableWithDataAndIdAsKey() throws IOException { "Should have expected records", expectedRowSet(createRecord(1, "aaa")), actualRowSet("*")); } + @Test + public void testEqualityColumnOnCustomPrecisionTSColumn() throws IOException { + Schema tableSchema = + new Schema( + required(3, "id", Types.IntegerType.get()), + required(4, "ts", Types.TimestampType.withZone())); + RowType flinkType = + new RowType( + false, + ImmutableList.of( + new RowType.RowField("id", new IntType()), + new RowType.RowField("ts", new LocalZonedTimestampType(3)))); + + this.table = create(tableSchema, PartitionSpec.unpartitioned()); + initTable(table); + + List equalityIds = ImmutableList.of(table.schema().findField("ts").fieldId()); + TaskWriterFactory taskWriterFactory = createTaskWriterFactory(flinkType, equalityIds); + taskWriterFactory.initialize(1, 1); + + TaskWriter writer = taskWriterFactory.create(); + RowDataSerializer serializer = new RowDataSerializer(flinkType); + OffsetDateTime start = OffsetDateTime.now(); + writer.write( + serializer.toBinaryRow( + GenericRowData.ofKind( + RowKind.INSERT, 1, TimestampData.fromInstant(start.toInstant())))); + writer.write( + serializer.toBinaryRow( + GenericRowData.ofKind( + RowKind.INSERT, 2, TimestampData.fromInstant(start.plusSeconds(1).toInstant())))); + writer.write( + serializer.toBinaryRow( + GenericRowData.ofKind( + RowKind.DELETE, 2, TimestampData.fromInstant(start.plusSeconds(1).toInstant())))); + + WriteResult result = writer.complete(); + // One data file + Assertions.assertThat(result.dataFiles().length).isEqualTo(1); + // One eq delete file + one pos delete file + Assertions.assertThat(result.deleteFiles().length).isEqualTo(2); + Assertions.assertThat( + Arrays.stream(result.deleteFiles()) + .map(ContentFile::content) + .collect(Collectors.toSet())) + .isEqualTo(Sets.newHashSet(FileContent.POSITION_DELETES, FileContent.EQUALITY_DELETES)); + commitTransaction(result); + + Record expectedRecord = GenericRecord.create(tableSchema); + expectedRecord.setField("id", 1); + int cutPrecisionNano = start.getNano() / 1000000 * 1000000; + expectedRecord.setField("ts", start.withNano(cutPrecisionNano)); + + Assertions.assertThat(actualRowSet("*")).isEqualTo(expectedRowSet(expectedRecord)); + } + private void commitTransaction(WriteResult result) { RowDelta rowDelta = table.newRowDelta(); Arrays.stream(result.dataFiles()).forEach(rowDelta::addRows); @@ -349,4 +407,34 @@ private TaskWriterFactory createTaskWriterFactory(List equalit equalityFieldIds, false); } + + private TaskWriterFactory createTaskWriterFactory( + RowType flinkType, List equalityFieldIds) { + return new RowDataTaskWriterFactory( + SerializableTable.copyOf(table), + flinkType, + 128 * 1024 * 1024, + format, + table.properties(), + equalityFieldIds, + true); + } + + private void createAndInitTable(boolean partitioned) { + if (partitioned) { + this.table = create(SCHEMA, PartitionSpec.builderFor(SCHEMA).identity("data").build()); + } else { + this.table = create(SCHEMA, PartitionSpec.unpartitioned()); + } + + initTable(table); + } + + private void initTable(TestTables.TestTable testTable) { + testTable + .updateProperties() + .set(TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, String.valueOf(8 * 1024)) + .defaultFormat(format) + .commit(); + } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java index 8dc68aad10aa..3a8071523b7c 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java @@ -20,19 +20,21 @@ import java.io.File; import java.util.List; -import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.BaseCombinedScanTask; import org.apache.iceberg.FileFormat; import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.ThreadPools; import org.junit.Assert; @@ -40,8 +42,6 @@ public class SplitHelpers { - private static final AtomicLong splitLengthIncrement = new AtomicLong(); - private SplitHelpers() {} /** @@ -54,16 +54,53 @@ private SplitHelpers() {} * *

    Since the table and data files are deleted before this method return, caller shouldn't * attempt to read the data files. + * + *

    By default, v1 Iceberg table is created. For v2 table use {@link + * SplitHelpers#createSplitsFromTransientHadoopTable(TemporaryFolder, int, int, String)} + * + * @param temporaryFolder Folder to place the data to + * @param fileCount The number of files to create and add to the table + * @param filesPerSplit The number of files used for a split */ public static List createSplitsFromTransientHadoopTable( TemporaryFolder temporaryFolder, int fileCount, int filesPerSplit) throws Exception { + return createSplitsFromTransientHadoopTable(temporaryFolder, fileCount, filesPerSplit, "1"); + } + + /** + * This create a list of IcebergSourceSplit from real files + *

  • Create a new Hadoop table under the {@code temporaryFolder} + *
  • write {@code fileCount} number of files to the new Iceberg table + *
  • Discover the splits from the table and partition the splits by the {@code filePerSplit} + * limit + *
  • Delete the Hadoop table + * + *

    Since the table and data files are deleted before this method return, caller shouldn't + * attempt to read the data files. + * + * @param temporaryFolder Folder to place the data to + * @param fileCount The number of files to create and add to the table + * @param filesPerSplit The number of files used for a split + * @param version The table version to create + */ + public static List createSplitsFromTransientHadoopTable( + TemporaryFolder temporaryFolder, int fileCount, int filesPerSplit, String version) + throws Exception { final File warehouseFile = temporaryFolder.newFolder(); Assert.assertTrue(warehouseFile.delete()); final String warehouse = "file:" + warehouseFile; Configuration hadoopConf = new Configuration(); final HadoopCatalog catalog = new HadoopCatalog(hadoopConf, warehouse); + ImmutableMap properties = + ImmutableMap.of(TableProperties.FORMAT_VERSION, version); try { - final Table table = catalog.createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + final Table table = + catalog.createTable( + TestFixtures.TABLE_IDENTIFIER, + TestFixtures.SCHEMA, + PartitionSpec.unpartitioned(), + null, + properties); final GenericAppenderHelper dataAppender = new GenericAppenderHelper(table, FileFormat.PARQUET, temporaryFolder); for (int i = 0; i < fileCount; ++i) { diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestSimpleSplitAssigner.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java similarity index 88% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestSimpleSplitAssigner.java rename to flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java index ee6c5cc3a6c8..f28677ca9d6a 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestSimpleSplitAssigner.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java @@ -30,22 +30,24 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; -public class TestSimpleSplitAssigner { +public abstract class SplitAssignerTestBase { @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); @Test public void testEmptyInitialization() { - SimpleSplitAssigner assigner = new SimpleSplitAssigner(); + SplitAssigner assigner = splitAssigner(); assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); } /** Test a sequence of interactions for StaticEnumerator */ @Test public void testStaticEnumeratorSequence() throws Exception { - SimpleSplitAssigner assigner = new SimpleSplitAssigner(); + SplitAssigner assigner = splitAssigner(); assigner.onDiscoveredSplits( - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 4, 2)); + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 4, 1)); + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); assertSnapshot(assigner, 1); assigner.onUnassignedSplits( @@ -61,7 +63,7 @@ public void testStaticEnumeratorSequence() throws Exception { /** Test a sequence of interactions for ContinuousEnumerator */ @Test public void testContinuousEnumeratorSequence() throws Exception { - SimpleSplitAssigner assigner = new SimpleSplitAssigner(); + SplitAssigner assigner = splitAssigner(); assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); List splits1 = @@ -81,7 +83,7 @@ public void testContinuousEnumeratorSequence() throws Exception { } private void assertAvailableFuture( - SimpleSplitAssigner assigner, int splitCount, Runnable addSplitsRunnable) { + SplitAssigner assigner, int splitCount, Runnable addSplitsRunnable) { // register callback AtomicBoolean futureCompleted = new AtomicBoolean(); CompletableFuture future = assigner.isAvailable(); @@ -102,7 +104,7 @@ private void assertAvailableFuture( assertSnapshot(assigner, 0); } - private void assertGetNext(SimpleSplitAssigner assigner, GetSplitResult.Status expectedStatus) { + protected void assertGetNext(SplitAssigner assigner, GetSplitResult.Status expectedStatus) { GetSplitResult result = assigner.getNext(null); Assert.assertEquals(expectedStatus, result.status()); switch (expectedStatus) { @@ -118,8 +120,10 @@ private void assertGetNext(SimpleSplitAssigner assigner, GetSplitResult.Status e } } - private void assertSnapshot(SimpleSplitAssigner assigner, int splitCount) { + protected void assertSnapshot(SplitAssigner assigner, int splitCount) { Collection stateBeforeGet = assigner.state(); Assert.assertEquals(splitCount, stateBeforeGet.size()); } + + protected abstract SplitAssigner splitAssigner(); } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java new file mode 100644 index 000000000000..8994f3054abe --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.assigner; + +import org.apache.iceberg.flink.source.SplitHelpers; +import org.junit.Test; + +public class TestDefaultSplitAssigner extends SplitAssignerTestBase { + @Override + protected SplitAssigner splitAssigner() { + return new DefaultSplitAssigner(null); + } + + /** Test the assigner when multiple files are in a single split */ + @Test + public void testMultipleFilesInASplit() throws Exception { + SplitAssigner assigner = splitAssigner(); + assigner.onDiscoveredSplits( + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 4, 2)); + + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertSnapshot(assigner, 1); + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + assertSnapshot(assigner, 0); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java new file mode 100644 index 000000000000..8b9e132e0e22 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.assigner; + +import java.util.List; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.flink.source.SplitHelpers; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.SerializableComparator; +import org.apache.iceberg.flink.source.split.SplitComparators; +import org.apache.iceberg.util.SerializationUtil; +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Test; + +public class TestFileSequenceNumberBasedSplitAssigner extends SplitAssignerTestBase { + @Override + protected SplitAssigner splitAssigner() { + return new OrderedSplitAssignerFactory(SplitComparators.fileSequenceNumber()).createAssigner(); + } + + /** Test the assigner when multiple files are in a single split */ + @Test + public void testMultipleFilesInAnIcebergSplit() { + SplitAssigner assigner = splitAssigner(); + Assertions.assertThatThrownBy( + () -> + assigner.onDiscoveredSplits( + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 4, 2, "2")), + "Multiple files in a split is not allowed") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Please use 'split-open-file-cost'"); + } + + /** Test sorted splits */ + @Test + public void testSplitSort() throws Exception { + SplitAssigner assigner = splitAssigner(); + List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 5, 1, "2"); + + assigner.onDiscoveredSplits(splits.subList(3, 5)); + assigner.onDiscoveredSplits(splits.subList(0, 1)); + assigner.onDiscoveredSplits(splits.subList(1, 3)); + + assertGetNext(assigner, 1L); + assertGetNext(assigner, 2L); + assertGetNext(assigner, 3L); + assertGetNext(assigner, 4L); + assertGetNext(assigner, 5L); + + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + } + + @Test + public void testSerializable() { + byte[] bytes = SerializationUtil.serializeToBytes(SplitComparators.fileSequenceNumber()); + SerializableComparator comparator = + SerializationUtil.deserializeFromBytes(bytes); + Assert.assertNotNull(comparator); + } + + protected void assertGetNext(SplitAssigner assigner, Long expectedSequenceNumber) { + GetSplitResult result = assigner.getNext(null); + ContentFile file = result.split().task().files().iterator().next().file(); + Assert.assertEquals(expectedSequenceNumber, file.fileSequenceNumber()); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java index d0ae8fdf7748..349eb11cf549 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java @@ -28,7 +28,7 @@ import org.apache.iceberg.flink.source.ScanContext; import org.apache.iceberg.flink.source.SplitHelpers; import org.apache.iceberg.flink.source.StreamingStartingStrategy; -import org.apache.iceberg.flink.source.assigner.SimpleSplitAssigner; +import org.apache.iceberg.flink.source.assigner.DefaultSplitAssigner; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; @@ -342,7 +342,7 @@ private static ContinuousIcebergEnumerator createEnumerator( ContinuousIcebergEnumerator enumerator = new ContinuousIcebergEnumerator( context, - new SimpleSplitAssigner(Collections.emptyList()), + new DefaultSplitAssigner(null, Collections.emptyList()), scanContext, splitPlanner, null); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java index 0082e25add70..1d12d9f66a8a 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java @@ -40,7 +40,7 @@ public class TestIcebergEnumeratorStateSerializer { @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); private final IcebergEnumeratorStateSerializer serializer = - IcebergEnumeratorStateSerializer.INSTANCE; + new IcebergEnumeratorStateSerializer(true); protected final int version; diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java index 56af0caf1298..def4f436851b 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java @@ -35,6 +35,7 @@ import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.TestHelpers; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.SerializableComparator; import org.apache.iceberg.hadoop.HadoopFileIO; import org.junit.Assert; import org.junit.ClassRule; @@ -55,13 +56,68 @@ public void testReaderMetrics() throws Exception { TestingReaderOutput readerOutput = new TestingReaderOutput<>(); TestingMetricGroup metricGroup = new TestingMetricGroup(); TestingReaderContext readerContext = new TestingReaderContext(new Configuration(), metricGroup); - IcebergSourceReader reader = createReader(metricGroup, readerContext); + IcebergSourceReader reader = createReader(metricGroup, readerContext, null); reader.start(); testOneSplitFetcher(reader, readerOutput, metricGroup, 1); testOneSplitFetcher(reader, readerOutput, metricGroup, 2); } + @Test + public void testReaderOrder() throws Exception { + // Create 2 splits + List> recordBatchList1 = + ReaderUtil.createRecordBatchList(0L, TestFixtures.SCHEMA, 1, 1); + CombinedScanTask task1 = + ReaderUtil.createCombinedScanTask( + recordBatchList1, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory); + + List> recordBatchList2 = + ReaderUtil.createRecordBatchList(1L, TestFixtures.SCHEMA, 1, 1); + CombinedScanTask task2 = + ReaderUtil.createCombinedScanTask( + recordBatchList2, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory); + + // Sort the splits in one way + List rowDataList1 = + read( + Arrays.asList( + IcebergSourceSplit.fromCombinedScanTask(task1), + IcebergSourceSplit.fromCombinedScanTask(task2)), + 2); + + // Reverse the splits + List rowDataList2 = + read( + Arrays.asList( + IcebergSourceSplit.fromCombinedScanTask(task2), + IcebergSourceSplit.fromCombinedScanTask(task1)), + 2); + + // Check that the order of the elements is not changed + Assert.assertEquals(rowDataList1.get(0), rowDataList2.get(0)); + Assert.assertEquals(rowDataList1.get(1), rowDataList2.get(1)); + } + + private List read(List splits, long expected) throws Exception { + TestingMetricGroup metricGroup = new TestingMetricGroup(); + TestingReaderContext readerContext = new TestingReaderContext(new Configuration(), metricGroup); + // Using IdBasedComparator, so we can have a deterministic order of the splits + IcebergSourceReader reader = createReader(metricGroup, readerContext, new IdBasedComparator()); + reader.start(); + + reader.addSplits(splits); + TestingReaderOutput readerOutput = new TestingReaderOutput<>(); + while (readerOutput.getEmittedRecords().size() < expected) { + reader.pollNext(readerOutput); + } + + reader.pollNext(readerOutput); + + Assert.assertEquals(expected, readerOutput.getEmittedRecords().size()); + return readerOutput.getEmittedRecords(); + } + private void testOneSplitFetcher( IcebergSourceReader reader, TestingReaderOutput readerOutput, @@ -96,7 +152,9 @@ private void testOneSplitFetcher( } private IcebergSourceReader createReader( - MetricGroup metricGroup, SourceReaderContext readerContext) { + MetricGroup metricGroup, + SourceReaderContext readerContext, + SerializableComparator splitComparator) { IcebergSourceReaderMetrics readerMetrics = new IcebergSourceReaderMetrics(metricGroup, "db.tbl"); RowDataReaderFunction readerFunction = @@ -109,6 +167,13 @@ private IcebergSourceReader createReader( new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), new PlaintextEncryptionManager(), Collections.emptyList()); - return new IcebergSourceReader<>(readerMetrics, readerFunction, readerContext); + return new IcebergSourceReader<>(readerMetrics, readerFunction, splitComparator, readerContext); + } + + private static class IdBasedComparator implements SerializableComparator { + @Override + public int compare(IcebergSourceSplit o1, IcebergSourceSplit o2) { + return o1.splitId().compareTo(o2.splitId()); + } } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java index 046b0c31ce2e..cd778309f90d 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java @@ -21,7 +21,9 @@ import java.util.List; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; +import org.apache.iceberg.FileScanTask; import org.apache.iceberg.flink.source.SplitHelpers; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.junit.Assert; import org.junit.ClassRule; import org.junit.Test; @@ -31,7 +33,7 @@ public class TestIcebergSourceSplitSerializer { @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - private final IcebergSourceSplitSerializer serializer = IcebergSourceSplitSerializer.INSTANCE; + private final IcebergSourceSplitSerializer serializer = new IcebergSourceSplitSerializer(true); @Test public void testLatestVersion() throws Exception { @@ -81,6 +83,34 @@ private void serializeAndDeserializeV1(int splitCount, int filesPerSplit) throws } } + @Test + public void testV2() throws Exception { + serializeAndDeserializeV2(1, 1); + serializeAndDeserializeV2(10, 2); + } + + private void serializeAndDeserializeV2(int splitCount, int filesPerSplit) throws Exception { + final List splits = + SplitHelpers.createSplitsFromTransientHadoopTable( + TEMPORARY_FOLDER, splitCount, filesPerSplit); + for (IcebergSourceSplit split : splits) { + byte[] result = split.serializeV2(); + IcebergSourceSplit deserialized = IcebergSourceSplit.deserializeV2(result, true); + assertSplitEquals(split, deserialized); + } + } + + @Test + public void testDeserializeV1() throws Exception { + final List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + for (IcebergSourceSplit split : splits) { + byte[] result = split.serializeV1(); + IcebergSourceSplit deserialized = serializer.deserialize(1, result); + assertSplitEquals(split, deserialized); + } + } + @Test public void testCheckpointedPosition() throws Exception { final AtomicInteger index = new AtomicInteger(); @@ -90,9 +120,7 @@ public void testCheckpointedPosition() throws Exception { split -> { IcebergSourceSplit result; if (index.get() % 2 == 0) { - result = - IcebergSourceSplit.fromCombinedScanTask( - split.task(), index.get(), index.get()); + result = IcebergSourceSplit.fromCombinedScanTask(split.task(), 1, 1); } else { result = split; } @@ -115,7 +143,19 @@ public void testCheckpointedPosition() throws Exception { } private void assertSplitEquals(IcebergSourceSplit expected, IcebergSourceSplit actual) { - Assert.assertEquals(expected.splitId(), actual.splitId()); + List expectedTasks = Lists.newArrayList(expected.task().tasks().iterator()); + List actualTasks = Lists.newArrayList(actual.task().tasks().iterator()); + Assert.assertEquals(expectedTasks.size(), actualTasks.size()); + for (int i = 0; i < expectedTasks.size(); ++i) { + FileScanTask expectedTask = expectedTasks.get(i); + FileScanTask actualTask = actualTasks.get(i); + Assert.assertEquals(expectedTask.file().path(), actualTask.file().path()); + Assert.assertEquals(expectedTask.sizeBytes(), actualTask.sizeBytes()); + Assert.assertEquals(expectedTask.filesCount(), actualTask.filesCount()); + Assert.assertEquals(expectedTask.start(), actualTask.start()); + Assert.assertEquals(expectedTask.length(), actualTask.length()); + } + Assert.assertEquals(expected.fileOffset(), actual.fileOffset()); Assert.assertEquals(expected.recordOffset(), actual.recordOffset()); } diff --git a/format/spec.md b/format/spec.md index 58cfc2291114..60c0f99c3f90 100644 --- a/format/spec.md +++ b/format/spec.md @@ -1128,6 +1128,41 @@ Example ] } ] ``` +### Content File (Data and Delete) Serialization + +Content file (data or delete) is serialized as a JSON object according to the following table. + +| Metadata field |JSON representation|Example| +|--------------------------|--- |--- | +| **`spec-id`** |`JSON int`|`1`| +| **`content`** |`JSON string`|`DATA`, `POSITION_DELETES`, `EQUALITY_DELETES`| +| **`file-path`** |`JSON string`|`"s3://b/wh/data.db/table"`| +| **`file-format`** |`JSON string`|`AVRO`, `ORC`, `PARQUET`| +| **`partition`** |`JSON object: Partition data tuple using partition field ids for the struct field ids`|`{"1000":1}`| +| **`record-count`** |`JSON long`|`1`| +| **`file-size-in-bytes`** |`JSON long`|`1024`| +| **`column-sizes`** |`JSON object: Map from column id to the total size on disk of all regions that store the column.`|`{"keys":[3,4],"values":[100,200]}`| +| **`value-counts`** |`JSON object: Map from column id to number of values in the column (including null and NaN values)`|`{"keys":[3,4],"values":[90,180]}`| +| **`null-value-counts`** |`JSON object: Map from column id to number of null values in the column`|`{"keys":[3,4],"values":[10,20]}`| +| **`nan-value-counts`** |`JSON object: Map from column id to number of NaN values in the column`|`{"keys":[3,4],"values":[0,0]}`| +| **`lower-bounds`** |`JSON object: Map from column id to lower bound binary in the column serialized as hexadecimal string`|`{"keys":[3,4],"values":["01000000","02000000"]}`| +| **`upper-bounds`** |`JSON object: Map from column id to upper bound binary in the column serialized as hexadecimal string`|`{"keys":[3,4],"values":["05000000","0A000000"]}`| +| **`key-metadata`** |`JSON string: Encryption key metadata binary serialized as hexadecimal string`|`00000000000000000000000000000000`| +| **`split-offsets`** |`JSON list of long: Split offsets for the data file`|`[128,256]`| +| **`equality-ids`** |`JSON list of int: Field ids used to determine row equality in equality delete files`|`[1]`| +| **`sort-order-id`** |`JSON int`|`1`| + +### File Scan Task Serialization + +File scan task is serialized as a JSON object according to the following table. + +| Metadata field |JSON representation|Example| +|--------------------------|--- |--- | +| **`schema`** |`JSON object`|`See above, read schemas instead`| +| **`spec`** |`JSON object`|`See above, read partition specs instead`| +| **`data-file`** |`JSON object`|`See above, read content file instead`| +| **`delete-files`** |`JSON list of objects`|`See above, read content file instead`| +| **`residual-filter`** |`JSON object: residual filter expression`|`{"type":"eq","term":"id","value":1}`| ## Appendix D: Single-value serialization diff --git a/format/view-spec.md b/format/view-spec.md index a9826a32cf3e..fe23b4efb893 100644 --- a/format/view-spec.md +++ b/format/view-spec.md @@ -75,13 +75,17 @@ Notes: Each version in `versions` is a struct with the following fields: -| Requirement | Field name | Description | -|-------------|-------------------|--------------------------------------------------------------------------| -| _required_ | `version-id` | ID for the version | -| _required_ | `schema-id` | ID of the schema for the view version | -| _required_ | `timestamp-ms` | Timestamp when the version was created (ms from epoch) | -| _required_ | `summary` | A string to string map of [summary metadata](#summary) about the version | -| _required_ | `representations` | A list of [representations](#representations) for the view definition | +| Requirement | Field name | Description | +|-------------|---------------------|-------------------------------------------------------------------------------| +| _required_ | `version-id` | ID for the version | +| _required_ | `schema-id` | ID of the schema for the view version | +| _required_ | `timestamp-ms` | Timestamp when the version was created (ms from epoch) | +| _required_ | `summary` | A string to string map of [summary metadata](#summary) about the version | +| _required_ | `representations` | A list of [representations](#representations) for the view definition | +| _optional_ | `default-catalog` | Catalog name to use when a reference in the SELECT does not contain a catalog | +| _required_ | `default-namespace` | Namespace to use when a reference in the SELECT is a single identifier | + +When `default-catalog` is `null` or not set, the catalog in which the view is stored must be used as the default catalog. #### Summary @@ -117,10 +121,6 @@ A view version can have multiple SQL representations of different dialects, but | _required_ | `type` | `string` | Must be `sql` | | _required_ | `sql` | `string` | A SQL SELECT statement | | _required_ | `dialect` | `string` | The dialect of the `sql` SELECT statement (e.g., "trino" or "spark") | -| _optional_ | `default-catalog` | `string` | Catalog name to use when a reference in the SELECT does not contain a catalog | -| _optional_ | `default-namespace` | `list` | Namespace to use when a reference in the SELECT is a single identifier | -| _optional_ | `field-aliases` | `list` | Column names optionally specified in the create statement | -| _optional_ | `field-comments` | `list` | Column descriptions (COMMENT) optionally specified in the create statement | For example: @@ -144,13 +144,11 @@ This create statement would produce the following `sql` representation metadata: | `type` | `"sql"` | | `sql` | `"SELECT\n COUNT(1), CAST(event_ts AS DATE)\nFROM events\nGROUP BY 2"` | | `dialect` | `"spark"` | -| `default-catalog` | `"prod"` | -| `default-namespace` | `["default"]` | -| `field-aliases` | `["event_count", "event_date"]` | -| `field-comments` | `["Count of events", null]` | If a create statement does not include column names or comments before `AS`, the fields should be omitted. +The `event_count` (with the `Count of events` comment) and `event_date` field aliases must be part of the view version's `schema`. + #### Version log The version log tracks changes to the view's current version. This is the view's history and allows reconstructing what version of the view would have been used at some point in time. @@ -205,6 +203,8 @@ s3://bucket/warehouse/default.db/event_agg/metadata/00001-(uuid).metadata.json "version-id" : 1, "timestamp-ms" : 1573518431292, "schema-id" : 1, + "default-catalog" : "prod", + "default-namespace" : [ "default" ], "summary" : { "operation" : "create", "engine-name" : "Spark", @@ -213,11 +213,7 @@ s3://bucket/warehouse/default.db/event_agg/metadata/00001-(uuid).metadata.json "representations" : [ { "type" : "sql", "sql" : "SELECT\n COUNT(1), CAST(event_ts AS DATE)\nFROM events\nGROUP BY 2", - "dialect" : "spark", - "default-catalog" : "prod", - "default-namespace" : [ "default" ], - "field-aliases" : ["event_count", "event_date"], - "field-comments" : ["Count of events", null] + "dialect" : "spark" } ] } ], "current-schema-id": 1, @@ -226,12 +222,13 @@ s3://bucket/warehouse/default.db/event_agg/metadata/00001-(uuid).metadata.json "type" : "struct", "fields" : [ { "id" : 1, - "name" : "col1", + "name" : "event_count", "required" : false, - "type" : "int" + "type" : "int", + "doc" : "Count of events" }, { "id" : 2, - "name" : "col2", + "name" : "event_date", "required" : false, "type" : "date" } ] @@ -274,6 +271,8 @@ s3://bucket/warehouse/default.db/event_agg/metadata/00002-(uuid).metadata.json "version-id" : 1, "timestamp-ms" : 1573518431292, "schema-id" : 1, + "default-catalog" : "prod", + "default-namespace" : [ "default" ], "summary" : { "operation" : "create", "engine-name" : "Spark", @@ -282,15 +281,14 @@ s3://bucket/warehouse/default.db/event_agg/metadata/00002-(uuid).metadata.json "representations" : [ { "type" : "sql", "sql" : "SELECT\n COUNT(1), CAST(event_ts AS DATE)\nFROM events\nGROUP BY 2", - "dialect" : "spark", - "default-catalog" : "prod", - "default-namespace" : [ "default" ], - "field-aliases" : ["event_count", "event_date"], - "field-comments" : ["Count of events", null] + "dialect" : "spark" } ] }, { "version-id" : 2, "timestamp-ms" : 1573518981593, + "schema-id" : 1, + "default-catalog" : "prod", + "default-namespace" : [ "default" ], "summary" : { "operation" : "create", "engine-name" : "Spark", @@ -299,10 +297,7 @@ s3://bucket/warehouse/default.db/event_agg/metadata/00002-(uuid).metadata.json "representations" : [ { "type" : "sql", "sql" : "SELECT\n COUNT(1), CAST(event_ts AS DATE)\nFROM prod.default.events\nGROUP BY 2", - "dialect" : "spark", - "default-catalog" : "prod", - "default-namespace" : [ "default" ], - "field-aliases" : ["event_count", "event_date"] + "dialect" : "spark" } ] } ], "current-schema-id": 1, @@ -311,12 +306,13 @@ s3://bucket/warehouse/default.db/event_agg/metadata/00002-(uuid).metadata.json "type" : "struct", "fields" : [ { "id" : 1, - "name" : "col1", + "name" : "event_count", "required" : false, - "type" : "int" + "type" : "int", + "doc" : "Count of events" }, { "id" : 2, - "name" : "col2", + "name" : "event_date", "required" : false, "type" : "date" } ] diff --git a/gcp/src/main/java/org/apache/iceberg/gcp/GCPProperties.java b/gcp/src/main/java/org/apache/iceberg/gcp/GCPProperties.java index 457a76313d15..109a4d21ed3c 100644 --- a/gcp/src/main/java/org/apache/iceberg/gcp/GCPProperties.java +++ b/gcp/src/main/java/org/apache/iceberg/gcp/GCPProperties.java @@ -19,6 +19,7 @@ package org.apache.iceberg.gcp; import java.io.Serializable; +import java.util.Date; import java.util.Map; import java.util.Optional; @@ -36,6 +37,9 @@ public class GCPProperties implements Serializable { public static final String GCS_CHANNEL_READ_CHUNK_SIZE = "gcs.channel.read.chunk-size-bytes"; public static final String GCS_CHANNEL_WRITE_CHUNK_SIZE = "gcs.channel.write.chunk-size-bytes"; + public static final String GCS_OAUTH2_TOKEN = "gcs.oauth2.token"; + public static final String GCS_OAUTH2_TOKEN_EXPIRES_AT = "gcs.oauth2.token-expires-at"; + private String projectId; private String clientLibToken; private String serviceHost; @@ -47,6 +51,9 @@ public class GCPProperties implements Serializable { private Integer gcsChannelReadChunkSize; private Integer gcsChannelWriteChunkSize; + private String gcsOAuth2Token; + private Date gcsOAuth2TokenExpiresAt; + public GCPProperties() {} public GCPProperties(Map properties) { @@ -65,6 +72,12 @@ public GCPProperties(Map properties) { if (properties.containsKey(GCS_CHANNEL_WRITE_CHUNK_SIZE)) { gcsChannelWriteChunkSize = Integer.parseInt(properties.get(GCS_CHANNEL_WRITE_CHUNK_SIZE)); } + + gcsOAuth2Token = properties.get(GCS_OAUTH2_TOKEN); + if (properties.containsKey(GCS_OAUTH2_TOKEN_EXPIRES_AT)) { + gcsOAuth2TokenExpiresAt = + new Date(Long.parseLong(properties.get(GCS_OAUTH2_TOKEN_EXPIRES_AT))); + } } public Optional channelReadChunkSize() { @@ -98,4 +111,12 @@ public Optional serviceHost() { public Optional userProject() { return Optional.ofNullable(gcsUserProject); } + + public Optional oauth2Token() { + return Optional.ofNullable(gcsOAuth2Token); + } + + public Optional oauth2TokenExpiresAt() { + return Optional.ofNullable(gcsOAuth2TokenExpiresAt); + } } diff --git a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSFileIO.java b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSFileIO.java index 390896b9923d..54af44e43da8 100644 --- a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSFileIO.java +++ b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSFileIO.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.gcp.gcs; +import com.google.auth.oauth2.AccessToken; +import com.google.auth.oauth2.OAuth2Credentials; import com.google.cloud.storage.BlobId; import com.google.cloud.storage.Storage; import com.google.cloud.storage.StorageOptions; @@ -133,6 +135,15 @@ public void initialize(Map props) { gcpProperties.clientLibToken().ifPresent(builder::setClientLibToken); gcpProperties.serviceHost().ifPresent(builder::setHost); + gcpProperties + .oauth2Token() + .ifPresent( + token -> { + AccessToken accessToken = + new AccessToken(token, gcpProperties.oauth2TokenExpiresAt().orElse(null)); + builder.setCredentials(OAuth2Credentials.create(accessToken)); + }); + // Report Hadoop metrics if Hadoop is available try { DynConstructors.Ctor ctor = diff --git a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSInputStream.java b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSInputStream.java index d8b092e4e70a..fc09bafadbc8 100644 --- a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSInputStream.java +++ b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSInputStream.java @@ -117,7 +117,7 @@ public int read() throws IOException { readBytes.increment(); readOperations.increment(); - return singleByteBuffer.array()[0]; + return singleByteBuffer.array()[0] & 0xFF; } @Override diff --git a/gcp/src/test/java/org/apache/iceberg/gcp/gcs/GCSFileIOTest.java b/gcp/src/test/java/org/apache/iceberg/gcp/gcs/GCSFileIOTest.java index 3e8bc3f4714d..4e91b3e0e753 100644 --- a/gcp/src/test/java/org/apache/iceberg/gcp/gcs/GCSFileIOTest.java +++ b/gcp/src/test/java/org/apache/iceberg/gcp/gcs/GCSFileIOTest.java @@ -27,12 +27,13 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.nio.ByteBuffer; import java.util.Random; import java.util.stream.StreamSupport; -import org.apache.commons.io.IOUtils; import org.apache.iceberg.TestHelpers; import org.apache.iceberg.gcp.GCPProperties; import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.IOUtil; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -62,14 +63,14 @@ public void newInputFile() throws IOException { OutputFile out = io.newOutputFile(location); try (OutputStream os = out.createOrOverwrite()) { - IOUtils.write(expected, os); + IOUtil.writeFully(os, ByteBuffer.wrap(expected)); } assertThat(in.exists()).isTrue(); byte[] actual = new byte[1024 * 1024]; try (InputStream is = in.newStream()) { - IOUtils.readFully(is, actual); + IOUtil.readFully(is, actual, 0, actual.length); } assertThat(expected).isEqualTo(actual); diff --git a/gcp/src/test/java/org/apache/iceberg/gcp/gcs/GCSInputStreamTest.java b/gcp/src/test/java/org/apache/iceberg/gcp/gcs/GCSInputStreamTest.java index 1a5200345b7a..b04891842c92 100644 --- a/gcp/src/test/java/org/apache/iceberg/gcp/gcs/GCSInputStreamTest.java +++ b/gcp/src/test/java/org/apache/iceberg/gcp/gcs/GCSInputStreamTest.java @@ -29,8 +29,8 @@ import java.io.IOException; import java.util.Arrays; import java.util.Random; -import org.apache.commons.io.IOUtils; import org.apache.iceberg.gcp.GCPProperties; +import org.apache.iceberg.io.IOUtil; import org.apache.iceberg.io.SeekableInputStream; import org.apache.iceberg.metrics.MetricsContext; import org.junit.jupiter.api.Test; @@ -78,6 +78,22 @@ public void testRead() throws Exception { } } + @Test + public void testReadSingle() throws Exception { + BlobId uri = BlobId.fromGsUtilUri("gs://bucket/path/to/read.dat"); + int i0 = 1; + int i1 = 255; + byte[] data = {(byte) i0, (byte) i1}; + + writeGCSData(uri, data); + + try (SeekableInputStream in = + new GCSInputStream(storage, uri, gcpProperties, MetricsContext.nullMetrics())) { + assertThat(in.read()).isEqualTo(i0); + assertThat(in.read()).isEqualTo(i1); + } + } + private void readAndCheck( SeekableInputStream in, long rangeStart, int size, byte[] original, boolean buffered) throws IOException { @@ -88,7 +104,7 @@ private void readAndCheck( byte[] actual = new byte[size]; if (buffered) { - IOUtils.readFully(in, actual); + IOUtil.readFully(in, actual, 0, actual.length); } else { int read = 0; while (read < size) { @@ -121,7 +137,7 @@ public void testSeek() throws Exception { in.seek(data.length / 2); byte[] actual = new byte[data.length / 2]; - IOUtils.readFully(in, actual, 0, data.length / 2); + IOUtil.readFully(in, actual, 0, data.length / 2); byte[] expected = Arrays.copyOfRange(data, data.length / 2, data.length); assertThat(actual).isEqualTo(expected); diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java index 272335374105..977aa170cbe4 100644 --- a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java @@ -250,6 +250,7 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) { commitStatus = CommitStatus.SUCCESS; } catch (LockException le) { + commitStatus = CommitStatus.UNKNOWN; throw new CommitStateUnknownException( "Failed to heartbeat for hive lock while " + "committing changes. This can lead to a concurrent commit attempt be able to overwrite this commit. " diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveCreateReplaceTableTest.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveCreateReplaceTableTest.java index dd70795a0bfe..61d3659740b2 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveCreateReplaceTableTest.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveCreateReplaceTableTest.java @@ -20,8 +20,11 @@ import static org.apache.iceberg.PartitionSpec.builderFor; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; +import java.nio.file.Path; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; @@ -33,16 +36,12 @@ import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.exceptions.AlreadyExistsException; import org.apache.iceberg.exceptions.NoSuchTableException; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class HiveCreateReplaceTableTest extends HiveMetastoreTest { @@ -53,23 +52,23 @@ public class HiveCreateReplaceTableTest extends HiveMetastoreTest { required(3, "id", Types.IntegerType.get()), required(4, "data", Types.StringType.get())); private static final PartitionSpec SPEC = builderFor(SCHEMA).identity("id").build(); - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; private String tableLocation; - @Before + @BeforeEach public void createTableLocation() throws IOException { - tableLocation = temp.newFolder("hive-").getPath(); + tableLocation = temp.resolve("hive-").toString(); } - @After + @AfterEach public void cleanup() { catalog.dropTable(TABLE_IDENTIFIER); } @Test public void testCreateTableTxn() { - Assert.assertFalse("Table should not exist", catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should not exist").isFalse(); Transaction txn = catalog.newCreateTableTransaction( @@ -77,17 +76,17 @@ public void testCreateTableTxn() { txn.updateProperties().set("prop", "value").commit(); // verify the table is still not visible before the transaction is committed - Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).isFalse(); txn.commitTransaction(); Table table = catalog.loadTable(TABLE_IDENTIFIER); - Assert.assertEquals("Table props should match", "value", table.properties().get("prop")); + assertThat(table.properties()).as("Table props should match").containsEntry("prop", "value"); } @Test public void testCreateTableTxnTableCreatedConcurrently() { - Assert.assertFalse("Table should not exist", catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should not exist").isFalse(); Transaction txn = catalog.newCreateTableTransaction( @@ -95,16 +94,16 @@ public void testCreateTableTxnTableCreatedConcurrently() { // create the table concurrently catalog.createTable(TABLE_IDENTIFIER, SCHEMA, SPEC); - Assert.assertTrue("Table should be created", catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should be created").isTrue(); - Assertions.assertThatThrownBy(txn::commitTransaction) + assertThatThrownBy(txn::commitTransaction) .isInstanceOf(AlreadyExistsException.class) .hasMessage("Table already exists: hivedb.tbl"); } @Test public void testCreateTableTxnAndAppend() { - Assert.assertFalse("Table should not exist", catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should not exist").isFalse(); Transaction txn = catalog.newCreateTableTransaction( @@ -123,19 +122,20 @@ public void testCreateTableTxnAndAppend() { Table table = catalog.loadTable(TABLE_IDENTIFIER); Snapshot snapshot = table.currentSnapshot(); - Assert.assertTrue( - "Table should have one manifest file", snapshot.allManifests(table.io()).size() == 1); + assertThat(snapshot.allManifests(table.io())) + .as("Table should have one manifest file") + .hasSize(1); } @Test public void testCreateTableTxnTableAlreadyExists() { - Assert.assertFalse("Table should not exist", catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should not exist").isFalse(); // create a table before starting a transaction catalog.createTable(TABLE_IDENTIFIER, SCHEMA, SPEC); - Assert.assertTrue("Table should be created", catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should be created").isTrue(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> catalog.newCreateTableTransaction( TABLE_IDENTIFIER, SCHEMA, SPEC, tableLocation, Maps.newHashMap())) @@ -146,7 +146,7 @@ public void testCreateTableTxnTableAlreadyExists() { @Test public void testReplaceTableTxn() { catalog.createTable(TABLE_IDENTIFIER, SCHEMA, SPEC, tableLocation, Maps.newHashMap()); - Assert.assertTrue("Table should exist", catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should exist").isTrue(); Transaction txn = catalog.newReplaceTableTransaction(TABLE_IDENTIFIER, SCHEMA, false); txn.commitTransaction(); @@ -154,12 +154,14 @@ public void testReplaceTableTxn() { Table table = catalog.loadTable(TABLE_IDENTIFIER); PartitionSpec v1Expected = PartitionSpec.builderFor(table.schema()).alwaysNull("id", "id").withSpecId(1).build(); - Assert.assertEquals("Table should have a spec with one void field", v1Expected, table.spec()); + assertThat(table.spec()) + .as("Table should have a spec with one void field") + .isEqualTo(v1Expected); } @Test public void testReplaceTableTxnTableNotExists() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> catalog.newReplaceTableTransaction(TABLE_IDENTIFIER, SCHEMA, SPEC, false)) .isInstanceOf(NoSuchTableException.class) .hasMessage("Table does not exist: hivedb.tbl"); @@ -168,7 +170,7 @@ public void testReplaceTableTxnTableNotExists() { @Test public void testReplaceTableTxnTableDeletedConcurrently() { catalog.createTable(TABLE_IDENTIFIER, SCHEMA, SPEC, tableLocation, Maps.newHashMap()); - Assert.assertTrue("Table should exist", catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should exist").isTrue(); Transaction txn = catalog.newReplaceTableTransaction(TABLE_IDENTIFIER, SCHEMA, SPEC, false); @@ -176,7 +178,7 @@ public void testReplaceTableTxnTableDeletedConcurrently() { txn.updateProperties().set("prop", "value").commit(); - Assertions.assertThatThrownBy(txn::commitTransaction) + assertThatThrownBy(txn::commitTransaction) .isInstanceOf(NoSuchTableException.class) .hasMessage("No such table: hivedb.tbl"); } @@ -185,7 +187,7 @@ public void testReplaceTableTxnTableDeletedConcurrently() { public void testReplaceTableTxnTableModifiedConcurrently() { Table table = catalog.createTable(TABLE_IDENTIFIER, SCHEMA, SPEC, tableLocation, Maps.newHashMap()); - Assert.assertTrue("Table should exist", catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should exist").isTrue(); Transaction txn = catalog.newReplaceTableTransaction(TABLE_IDENTIFIER, SCHEMA, SPEC, false); @@ -197,26 +199,28 @@ public void testReplaceTableTxnTableModifiedConcurrently() { // the replace should still succeed table = catalog.loadTable(TABLE_IDENTIFIER); - Assert.assertNull("Table props should be updated", table.properties().get("another-prop")); - Assert.assertEquals("Table props should match", "value", table.properties().get("prop")); + assertThat(table.properties()) + .as("Table props should be updated") + .doesNotContainKey("another-prop") + .containsEntry("prop", "value"); } @Test public void testCreateOrReplaceTableTxnTableNotExists() { - Assert.assertFalse("Table should not exist", catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should not exist").isFalse(); Transaction txn = catalog.newReplaceTableTransaction(TABLE_IDENTIFIER, SCHEMA, SPEC, true); txn.updateProperties().set("prop", "value").commit(); txn.commitTransaction(); Table table = catalog.loadTable(TABLE_IDENTIFIER); - Assert.assertEquals("Table props should match", "value", table.properties().get("prop")); + assertThat(table.properties()).as("Table props should match").containsEntry("prop", "value"); } @Test public void testCreateOrReplaceTableTxnTableExists() { catalog.createTable(TABLE_IDENTIFIER, SCHEMA, SPEC, tableLocation, Maps.newHashMap()); - Assert.assertTrue("Table should exist", catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should exist").isTrue(); Transaction txn = catalog.newReplaceTableTransaction(TABLE_IDENTIFIER, SCHEMA, true); txn.commitTransaction(); @@ -224,15 +228,16 @@ public void testCreateOrReplaceTableTxnTableExists() { Table table = catalog.loadTable(TABLE_IDENTIFIER); PartitionSpec v1Expected = PartitionSpec.builderFor(table.schema()).alwaysNull("id", "id").withSpecId(1).build(); - Assert.assertEquals("Table should have a spec with one void field", v1Expected, table.spec()); + assertThat(table.spec()) + .as("Table should have a spec with one void field") + .isEqualTo(v1Expected); } @Test public void testCreateOrReplaceTableTxnTableDeletedConcurrently() { - Assert.assertFalse("Table should not exist", catalog.tableExists(TABLE_IDENTIFIER)); - + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should not exist").isFalse(); catalog.createTable(TABLE_IDENTIFIER, SCHEMA, SPEC); - Assert.assertTrue("Table should be created", catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should be created").isTrue(); Transaction txn = catalog.newReplaceTableTransaction( @@ -251,12 +256,12 @@ public void testCreateOrReplaceTableTxnTableDeletedConcurrently() { txn.commitTransaction(); Table table = catalog.loadTable(TABLE_IDENTIFIER); - Assert.assertEquals("Table props should match", "value", table.properties().get("prop")); + assertThat(table.properties()).as("Table props should match").containsEntry("prop", "value"); } @Test public void testCreateOrReplaceTableTxnTableCreatedConcurrently() { - Assert.assertFalse("Table should not exist", catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should not exist").isFalse(); Transaction txn = catalog.newReplaceTableTransaction( @@ -270,19 +275,21 @@ public void testCreateOrReplaceTableTxnTableCreatedConcurrently() { // create the table concurrently catalog.createTable(TABLE_IDENTIFIER, SCHEMA, SPEC); - Assert.assertTrue("Table should be created", catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should be created").isTrue(); // expect the transaction to succeed anyway txn.commitTransaction(); Table table = catalog.loadTable(TABLE_IDENTIFIER); - Assert.assertEquals("Partition spec should match", PartitionSpec.unpartitioned(), table.spec()); - Assert.assertEquals("Table props should match", "value", table.properties().get("prop")); + assertThat(table.spec()) + .as("Partition spec should match") + .isEqualTo(PartitionSpec.unpartitioned()); + assertThat(table.properties()).as("Table props should match").containsEntry("prop", "value"); } @Test public void testCreateTableTxnWithGlobalTableLocation() { - Assert.assertFalse("Table should not exist", catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should not exist").isFalse(); Transaction txn = catalog.newCreateTableTransaction( @@ -300,6 +307,6 @@ public void testCreateTableTxnWithGlobalTableLocation() { table.newAppend().appendFile(dataFile).commit(); - Assert.assertEquals("Write should succeed", 1, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Write should succeed").hasSize(1); } } diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveMetastoreTest.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveMetastoreTest.java index 8c7adbc1f6c5..e48df0ce9378 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveMetastoreTest.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveMetastoreTest.java @@ -28,8 +28,8 @@ import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.junit.AfterClass; -import org.junit.BeforeClass; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; public abstract class HiveMetastoreTest { @@ -41,7 +41,7 @@ public abstract class HiveMetastoreTest { protected static HiveConf hiveConf; protected static TestHiveMetastore metastore; - @BeforeClass + @BeforeAll public static void startMetastore() throws Exception { startMetastore(Collections.emptyMap()); } @@ -72,7 +72,7 @@ public static void startMetastore(Map hiveConfOverride) throws E hiveConfWithOverrides); } - @AfterClass + @AfterAll public static void stopMetastore() throws Exception { HiveMetastoreTest.catalog = null; diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableBaseTest.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableBaseTest.java index b49c61192aaa..51f4b5953276 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableBaseTest.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableBaseTest.java @@ -34,8 +34,8 @@ import org.apache.iceberg.TableMetadataParser; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.types.Types; -import org.junit.After; -import org.junit.Before; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; public class HiveTableBaseTest extends HiveMetastoreTest { @@ -56,13 +56,13 @@ public class HiveTableBaseTest extends HiveMetastoreTest { private Path tableLocation; - @Before + @BeforeEach public void createTestTable() { this.tableLocation = new Path(catalog.createTable(TABLE_IDENTIFIER, schema, partitionSpec).location()); } - @After + @AfterEach public void dropTestTable() throws Exception { // drop the table data tableLocation.getFileSystem(hiveConf).delete(tableLocation, true); diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableTest.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableTest.java index 5140563f7d4e..0b5edf21aec7 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableTest.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableTest.java @@ -28,9 +28,12 @@ import static org.apache.iceberg.TableMetadataParser.getFileExtension; import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; +import java.nio.file.Path; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -74,16 +77,13 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; import org.apache.thrift.TException; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class HiveTableTest extends HiveTableBaseTest { static final String NON_DEFAULT_DATABASE = "nondefault"; - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); + @TempDir private Path tempFolder; @Test public void testCreate() throws TException { @@ -95,23 +95,23 @@ public void testCreate() throws TException { // check parameters are in expected state Map parameters = table.getParameters(); - Assert.assertNotNull(parameters); - Assert.assertTrue(ICEBERG_TABLE_TYPE_VALUE.equalsIgnoreCase(parameters.get(TABLE_TYPE_PROP))); - Assert.assertTrue("EXTERNAL_TABLE".equalsIgnoreCase(table.getTableType())); + assertThat(parameters).isNotNull(); + assertThat(parameters.get(TABLE_TYPE_PROP)).isEqualToIgnoringCase(ICEBERG_TABLE_TYPE_VALUE); + assertThat(table.getTableType()).isEqualToIgnoringCase("EXTERNAL_TABLE"); // Ensure the table is pointing to empty location - Assert.assertEquals(getTableLocation(tableName), table.getSd().getLocation()); + assertThat(table.getSd().getLocation()).isEqualTo(getTableLocation(tableName)); // Ensure it is stored as unpartitioned table in hive. - Assert.assertEquals(0, table.getPartitionKeysSize()); + assertThat(table.getPartitionKeysSize()).isEqualTo(0); // Only 1 snapshotFile Should exist and no manifests should exist - Assert.assertEquals(1, metadataVersionFiles(tableName).size()); - Assert.assertEquals(0, manifestFiles(tableName).size()); + assertThat(metadataVersionFiles(tableName)).hasSize(1); + assertThat(manifestFiles(tableName)).hasSize(0); final Table icebergTable = catalog.loadTable(TABLE_IDENTIFIER); // Iceberg schema should match the loaded table - Assert.assertEquals(schema.asStruct(), icebergTable.schema().asStruct()); + assertThat(icebergTable.schema().asStruct()).isEqualTo(schema.asStruct()); } @Test @@ -122,25 +122,26 @@ public void testRename() { Table original = catalog.loadTable(TABLE_IDENTIFIER); catalog.renameTable(TABLE_IDENTIFIER, renameTableIdentifier); - Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER)); - Assert.assertTrue(catalog.tableExists(renameTableIdentifier)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).isFalse(); + assertThat(catalog.tableExists(renameTableIdentifier)).isTrue(); Table renamed = catalog.loadTable(renameTableIdentifier); - Assert.assertEquals(original.schema().asStruct(), renamed.schema().asStruct()); - Assert.assertEquals(original.spec(), renamed.spec()); - Assert.assertEquals(original.location(), renamed.location()); - Assert.assertEquals(original.currentSnapshot(), renamed.currentSnapshot()); + assertThat(renamed.schema().asStruct()).isEqualTo(original.schema().asStruct()); + assertThat(renamed.spec()).isEqualTo(original.spec()); + assertThat(renamed.location()).isEqualTo(original.location()); + assertThat(renamed.currentSnapshot()).isEqualTo(original.currentSnapshot()); - Assert.assertTrue(catalog.dropTable(renameTableIdentifier)); + assertThat(catalog.dropTable(renameTableIdentifier)).isTrue(); } @Test public void testDrop() { - Assert.assertTrue("Table should exist", catalog.tableExists(TABLE_IDENTIFIER)); - Assert.assertTrue( - "Drop should return true and drop the table", catalog.dropTable(TABLE_IDENTIFIER)); - Assert.assertFalse("Table should not exist", catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should exist").isTrue(); + assertThat(catalog.dropTable(TABLE_IDENTIFIER)) + .as("Drop should return true and drop the table") + .isTrue(); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should not exist").isFalse(); } @Test @@ -152,13 +153,13 @@ public void testDropWithoutPurgeLeavesTableData() throws IOException { String manifestListLocation = table.currentSnapshot().manifestListLocation().replace("file:", ""); - Assert.assertTrue( - "Drop should return true and drop the table", - catalog.dropTable(TABLE_IDENTIFIER, false /* do not delete underlying files */)); - Assert.assertFalse("Table should not exist", catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.dropTable(TABLE_IDENTIFIER, false /* do not delete underlying files */)) + .as("Drop should return true and drop the table") + .isTrue(); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should not exist").isFalse(); - Assert.assertTrue("Table data files should exist", new File(fileLocation).exists()); - Assert.assertTrue("Table metadata files should exist", new File(manifestListLocation).exists()); + assertThat(new File(fileLocation)).as("Table data files should exist").exists(); + assertThat(new File(manifestListLocation)).as("Table metadata files should exist").exists(); } @Test @@ -214,29 +215,30 @@ public void testDropTable() throws IOException { List manifests = table.currentSnapshot().allManifests(table.io()); - Assert.assertTrue( - "Drop (table and data) should return true and drop the table", - catalog.dropTable(TABLE_IDENTIFIER)); - Assert.assertFalse("Table should not exist", catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.dropTable(TABLE_IDENTIFIER)) + .as("Drop (table and data) should return true and drop the table") + .isTrue(); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should not exist").isFalse(); - Assert.assertFalse("Table data files should not exist", new File(location1).exists()); - Assert.assertFalse("Table data files should not exist", new File(location2).exists()); - Assert.assertFalse( - "Table manifest list files should not exist", new File(manifestListLocation).exists()); + assertThat(new File(location1)).as("Table data files should not exist").doesNotExist(); + assertThat(new File(location2)).as("Table data files should not exist").doesNotExist(); + assertThat(new File(manifestListLocation)) + .as("Table manifest list files should not exist") + .doesNotExist(); for (ManifestFile manifest : manifests) { - Assert.assertFalse( - "Table manifest files should not exist", - new File(manifest.path().replace("file:", "")).exists()); + assertThat(new File(manifest.path().replace("file:", ""))) + .as("Table manifest files should not exist") + .doesNotExist(); } - Assert.assertFalse( - "Table metadata file should not exist", - new File( + assertThat( + new File( ((HasTableOperations) table) .operations() .current() .metadataFileLocation() - .replace("file:", "")) - .exists()); + .replace("file:", ""))) + .as("Table metadata file should not exist") + .doesNotExist(); } @Test @@ -248,9 +250,9 @@ public void testExistingTableUpdate() throws TException { icebergTable = catalog.loadTable(TABLE_IDENTIFIER); // Only 2 snapshotFile Should exist and no manifests should exist - Assert.assertEquals(2, metadataVersionFiles(TABLE_NAME).size()); - Assert.assertEquals(0, manifestFiles(TABLE_NAME).size()); - Assert.assertEquals(altered.asStruct(), icebergTable.schema().asStruct()); + assertThat(metadataVersionFiles(TABLE_NAME)).hasSize(2); + assertThat(manifestFiles(TABLE_NAME)).hasSize(0); + assertThat(icebergTable.schema().asStruct()).isEqualTo(altered.asStruct()); final org.apache.hadoop.hive.metastore.api.Table table = metastoreClient.getTable(DB_NAME, TABLE_NAME); @@ -258,7 +260,7 @@ public void testExistingTableUpdate() throws TException { table.getSd().getCols().stream().map(FieldSchema::getName).collect(Collectors.toList()); final List icebergColumns = altered.columns().stream().map(Types.NestedField::name).collect(Collectors.toList()); - Assert.assertEquals(icebergColumns, hiveColumns); + assertThat(hiveColumns).isEqualTo(icebergColumns); } @Test @@ -285,10 +287,9 @@ public void testColumnTypeChangeInMetastore() throws TException { .addColumn("int", Types.IntegerType.get()) .commit(); - Assert.assertEquals( - "Schema should match expected", - expectedSchema.asStruct(), - icebergTable.schema().asStruct()); + assertThat(icebergTable.schema().asStruct()) + .as("Schema should match expected") + .isEqualTo(expectedSchema.asStruct()); expectedSchema = new Schema( @@ -299,10 +300,9 @@ public void testColumnTypeChangeInMetastore() throws TException { .fields()); icebergTable.updateSchema().deleteColumn("string").commit(); - Assert.assertEquals( - "Schema should match expected", - expectedSchema.asStruct(), - icebergTable.schema().asStruct()); + assertThat(icebergTable.schema().asStruct()) + .as("Schema should match expected") + .isEqualTo(expectedSchema.asStruct()); } @Test @@ -313,7 +313,7 @@ public void testFailure() throws TException { String dummyLocation = "dummylocation"; table.getParameters().put(METADATA_LOCATION_PROP, dummyLocation); metastoreClient.alter_table(DB_NAME, TABLE_NAME, table); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> icebergTable.updateSchema().addColumn("data", Types.LongType.get()).commit()) .isInstanceOf(CommitFailedException.class) .hasMessageContaining("is not same as the current table metadata location 'dummylocation'"); @@ -327,8 +327,8 @@ public void testListTables() throws TException, IOException { .filter(t -> t.namespace().level(0).equals(DB_NAME) && t.name().equals(TABLE_NAME)) .collect(Collectors.toList()); - Assert.assertEquals(1, expectedIdents.size()); - Assert.assertTrue(catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(expectedIdents).hasSize(1); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).isTrue(); // create a hive table String hiveTableName = "test_hive_table"; @@ -337,13 +337,13 @@ public void testListTables() throws TException, IOException { catalog.setListAllTables(false); List tableIdents1 = catalog.listTables(TABLE_IDENTIFIER.namespace()); - Assert.assertEquals("should only 1 iceberg table .", 1, tableIdents1.size()); + assertThat(tableIdents1).as("should only 1 iceberg table .").hasSize(1); catalog.setListAllTables(true); List tableIdents2 = catalog.listTables(TABLE_IDENTIFIER.namespace()); - Assert.assertEquals("should be 2 tables in namespace .", 2, tableIdents2.size()); + assertThat(tableIdents2).as("should be 2 tables in namespace .").hasSize(2); - Assert.assertTrue(catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).isTrue(); metastoreClient.dropTable(DB_NAME, hiveTableName); } @@ -362,7 +362,7 @@ private org.apache.hadoop.hive.metastore.api.Table createHiveTable(String hiveTa StorageDescriptor sd = new StorageDescriptor( Lists.newArrayList(), - tempFolder.newFolder().getAbsolutePath(), + tempFolder.toAbsolutePath().toString(), "org.apache.hadoop.mapred.TextInputFormat", "org.apache.hadoop.mapred.TextOutputFormat", false, @@ -400,14 +400,14 @@ public void testNonDefaultDatabaseLocation() throws IOException, TException { namespace, Collections.singletonMap("location", nonDefaultLocation.getPath())); Map namespaceMeta = catalog.loadNamespaceMetadata(namespace); // Make sure that we are testing a namespace with a non default location :) - Assert.assertEquals(namespaceMeta.get("location"), "file:" + nonDefaultLocation.getPath()); + assertThat("file:" + nonDefaultLocation.getPath()).isEqualTo(namespaceMeta.get("location")); TableIdentifier tableIdentifier = TableIdentifier.of(namespace, TABLE_NAME); catalog.createTable(tableIdentifier, schema); // Let's check the location loaded through the catalog Table table = catalog.loadTable(tableIdentifier); - Assert.assertEquals(namespaceMeta.get("location") + "/" + TABLE_NAME, table.location()); + assertThat(table.location()).isEqualTo(namespaceMeta.get("location") + "/" + TABLE_NAME); // Drop the database and purge the files metastoreClient.dropDatabase(NON_DEFAULT_DATABASE, true, true, true); @@ -419,16 +419,15 @@ public void testRegisterTable() throws TException { metastoreClient.getTable(DB_NAME, TABLE_NAME); Map originalParams = originalTable.getParameters(); - Assert.assertNotNull(originalParams); - Assert.assertTrue( - ICEBERG_TABLE_TYPE_VALUE.equalsIgnoreCase(originalParams.get(TABLE_TYPE_PROP))); - Assert.assertTrue("EXTERNAL_TABLE".equalsIgnoreCase(originalTable.getTableType())); + assertThat(originalParams).isNotNull(); + assertThat(originalParams.get(TABLE_TYPE_PROP)).isEqualToIgnoringCase(ICEBERG_TABLE_TYPE_VALUE); + assertThat(originalTable.getTableType()).isEqualToIgnoringCase("EXTERNAL_TABLE"); catalog.dropTable(TABLE_IDENTIFIER, false); - Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).isFalse(); List metadataVersionFiles = metadataVersionFiles(TABLE_NAME); - Assert.assertEquals(1, metadataVersionFiles.size()); + assertThat(metadataVersionFiles).hasSize(1); catalog.registerTable(TABLE_IDENTIFIER, "file:" + metadataVersionFiles.get(0)); @@ -436,18 +435,17 @@ public void testRegisterTable() throws TException { metastoreClient.getTable(DB_NAME, TABLE_NAME); Map newTableParameters = newTable.getParameters(); - Assert.assertNull(newTableParameters.get(PREVIOUS_METADATA_LOCATION_PROP)); - Assert.assertEquals( - originalParams.get(TABLE_TYPE_PROP), newTableParameters.get(TABLE_TYPE_PROP)); - Assert.assertEquals( - originalParams.get(METADATA_LOCATION_PROP), newTableParameters.get(METADATA_LOCATION_PROP)); - Assert.assertEquals(originalTable.getSd(), newTable.getSd()); + assertThat(newTableParameters) + .doesNotContainKey(PREVIOUS_METADATA_LOCATION_PROP) + .containsEntry(TABLE_TYPE_PROP, originalParams.get(TABLE_TYPE_PROP)) + .containsEntry(METADATA_LOCATION_PROP, originalParams.get(METADATA_LOCATION_PROP)); + assertThat(newTable.getSd()).isEqualTo(originalTable.getSd()); } @Test public void testRegisterHadoopTableToHiveCatalog() throws IOException, TException { // create a hadoop catalog - String tableLocation = tempFolder.newFolder().toString(); + String tableLocation = tempFolder.toString(); HadoopCatalog hadoopCatalog = new HadoopCatalog(new Configuration(), tableLocation); // create table using hadoop catalog TableIdentifier identifier = TableIdentifier.of(DB_NAME, "table1"); @@ -457,8 +455,8 @@ public void testRegisterHadoopTableToHiveCatalog() throws IOException, TExceptio // insert some data String file1Location = appendData(table, "file1"); List tasks = Lists.newArrayList(table.newScan().planFiles()); - Assert.assertEquals("Should scan 1 file", 1, tasks.size()); - Assert.assertEquals(tasks.get(0).file().path(), file1Location); + assertThat(tasks).as("Should scan 1 file").hasSize(1); + assertThat(file1Location).isEqualTo(tasks.get(0).file().path()); // collect metadata file List metadataFiles = @@ -466,31 +464,31 @@ public void testRegisterHadoopTableToHiveCatalog() throws IOException, TExceptio .map(File::getAbsolutePath) .filter(f -> f.endsWith(getFileExtension(TableMetadataParser.Codec.NONE))) .collect(Collectors.toList()); - Assert.assertEquals(2, metadataFiles.size()); + assertThat(metadataFiles).hasSize(2); - Assertions.assertThatThrownBy(() -> metastoreClient.getTable(DB_NAME, "table1")) + assertThatThrownBy(() -> metastoreClient.getTable(DB_NAME, "table1")) .isInstanceOf(NoSuchObjectException.class) .hasMessage("hivedb.table1 table not found"); - Assertions.assertThatThrownBy(() -> catalog.loadTable(identifier)) + assertThatThrownBy(() -> catalog.loadTable(identifier)) .isInstanceOf(NoSuchTableException.class) .hasMessage("Table does not exist: hivedb.table1"); // register the table to hive catalog using the latest metadata file String latestMetadataFile = ((BaseTable) table).operations().current().metadataFileLocation(); catalog.registerTable(identifier, "file:" + latestMetadataFile); - Assert.assertNotNull(metastoreClient.getTable(DB_NAME, "table1")); + assertThat(metastoreClient.getTable(DB_NAME, "table1")).isNotNull(); // load the table in hive catalog table = catalog.loadTable(identifier); - Assert.assertNotNull(table); + assertThat(table).isNotNull(); // insert some data String file2Location = appendData(table, "file2"); tasks = Lists.newArrayList(table.newScan().planFiles()); - Assert.assertEquals("Should scan 2 files", 2, tasks.size()); + assertThat(tasks).as("Should scan 2 files").hasSize(2); Set files = tasks.stream().map(task -> task.file().path().toString()).collect(Collectors.toSet()); - Assert.assertTrue(files.contains(file1Location) && files.contains(file2Location)); + assertThat(files).contains(file1Location, file2Location); } private String appendData(Table table, String fileName) throws IOException { @@ -528,16 +526,15 @@ public void testRegisterExistingTable() throws TException { metastoreClient.getTable(DB_NAME, TABLE_NAME); Map originalParams = originalTable.getParameters(); - Assert.assertNotNull(originalParams); - Assert.assertTrue( - ICEBERG_TABLE_TYPE_VALUE.equalsIgnoreCase(originalParams.get(TABLE_TYPE_PROP))); - Assert.assertTrue("EXTERNAL_TABLE".equalsIgnoreCase(originalTable.getTableType())); + assertThat(originalParams).isNotNull(); + assertThat(originalParams.get(TABLE_TYPE_PROP)).isEqualToIgnoringCase(ICEBERG_TABLE_TYPE_VALUE); + assertThat(originalTable.getTableType()).isEqualToIgnoringCase("EXTERNAL_TABLE"); List metadataVersionFiles = metadataVersionFiles(TABLE_NAME); - Assert.assertEquals(1, metadataVersionFiles.size()); + assertThat(metadataVersionFiles).hasSize(1); // Try to register an existing table - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> catalog.registerTable(TABLE_IDENTIFIER, "file:" + metadataVersionFiles.get(0))) .isInstanceOf(AlreadyExistsException.class) .hasMessage("Table already exists: hivedb.tbl"); @@ -618,35 +615,35 @@ public void testMissingMetadataWontCauseHang() { File realLocation = new File(metadataLocation(TABLE_NAME)); File fakeLocation = new File(metadataLocation(TABLE_NAME) + "_dummy"); - Assert.assertTrue(realLocation.renameTo(fakeLocation)); - Assertions.assertThatThrownBy(() -> catalog.loadTable(TABLE_IDENTIFIER)) + assertThat(realLocation.renameTo(fakeLocation)).isTrue(); + assertThatThrownBy(() -> catalog.loadTable(TABLE_IDENTIFIER)) .isInstanceOf(NotFoundException.class) .hasMessageStartingWith("Failed to open input stream for file"); - Assert.assertTrue(fakeLocation.renameTo(realLocation)); + assertThat(fakeLocation.renameTo(realLocation)).isTrue(); } private void assertHiveEnabled( org.apache.hadoop.hive.metastore.api.Table hmsTable, boolean expected) { if (expected) { - Assert.assertEquals( - "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler", - hmsTable.getParameters().get(hive_metastoreConstants.META_TABLE_STORAGE)); - Assert.assertEquals( - "org.apache.iceberg.mr.hive.HiveIcebergSerDe", - hmsTable.getSd().getSerdeInfo().getSerializationLib()); - Assert.assertEquals( - "org.apache.iceberg.mr.hive.HiveIcebergInputFormat", hmsTable.getSd().getInputFormat()); - Assert.assertEquals( - "org.apache.iceberg.mr.hive.HiveIcebergOutputFormat", hmsTable.getSd().getOutputFormat()); + assertThat(hmsTable.getParameters()) + .containsEntry( + hive_metastoreConstants.META_TABLE_STORAGE, + "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler"); + assertThat(hmsTable.getSd().getSerdeInfo().getSerializationLib()) + .isEqualTo("org.apache.iceberg.mr.hive.HiveIcebergSerDe"); + assertThat(hmsTable.getSd().getInputFormat()) + .isEqualTo("org.apache.iceberg.mr.hive.HiveIcebergInputFormat"); + assertThat(hmsTable.getSd().getOutputFormat()) + .isEqualTo("org.apache.iceberg.mr.hive.HiveIcebergOutputFormat"); } else { - Assert.assertNull(hmsTable.getParameters().get(hive_metastoreConstants.META_TABLE_STORAGE)); - Assert.assertEquals( - "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe", - hmsTable.getSd().getSerdeInfo().getSerializationLib()); - Assert.assertEquals( - "org.apache.hadoop.mapred.FileInputFormat", hmsTable.getSd().getInputFormat()); - Assert.assertEquals( - "org.apache.hadoop.mapred.FileOutputFormat", hmsTable.getSd().getOutputFormat()); + assertThat(hmsTable.getParameters()) + .doesNotContainKey(hive_metastoreConstants.META_TABLE_STORAGE); + assertThat(hmsTable.getSd().getSerdeInfo().getSerializationLib()) + .isEqualTo("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"); + assertThat(hmsTable.getSd().getInputFormat()) + .isEqualTo("org.apache.hadoop.mapred.FileInputFormat"); + assertThat(hmsTable.getSd().getOutputFormat()) + .isEqualTo("org.apache.hadoop.mapred.FileOutputFormat"); } } } diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestCachedClientPool.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestCachedClientPool.java index 2c2d256a4586..19b9b0effbb4 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestCachedClientPool.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestCachedClientPool.java @@ -20,6 +20,8 @@ import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE; import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE_HIVE; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.security.PrivilegedAction; import java.util.Collections; @@ -31,9 +33,7 @@ import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.hive.CachedClientPool.Key; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestCachedClientPool extends HiveMetastoreTest { @@ -41,21 +41,22 @@ public class TestCachedClientPool extends HiveMetastoreTest { public void testClientPoolCleaner() throws InterruptedException { CachedClientPool clientPool = new CachedClientPool(hiveConf, Collections.emptyMap()); HiveClientPool clientPool1 = clientPool.clientPool(); - Assertions.assertThat( + assertThat(clientPool1) + .isSameAs( CachedClientPool.clientPoolCache() - .getIfPresent(CachedClientPool.extractKey(null, hiveConf))) - .isSameAs(clientPool1); + .getIfPresent(CachedClientPool.extractKey(null, hiveConf))); TimeUnit.MILLISECONDS.sleep(EVICTION_INTERVAL - TimeUnit.SECONDS.toMillis(2)); HiveClientPool clientPool2 = clientPool.clientPool(); - Assert.assertSame(clientPool1, clientPool2); + assertThat(clientPool2).isSameAs(clientPool1); TimeUnit.MILLISECONDS.sleep(EVICTION_INTERVAL + TimeUnit.SECONDS.toMillis(5)); - Assert.assertNull( - CachedClientPool.clientPoolCache() - .getIfPresent(CachedClientPool.extractKey(null, hiveConf))); + assertThat( + CachedClientPool.clientPoolCache() + .getIfPresent(CachedClientPool.extractKey(null, hiveConf))) + .isNull(); // The client has been really closed. - Assert.assertTrue(clientPool1.isClosed()); - Assert.assertTrue(clientPool2.isClosed()); + assertThat(clientPool1.isClosed()).isTrue(); + assertThat(clientPool2.isClosed()).isTrue(); } @Test @@ -73,19 +74,22 @@ public void testCacheKey() throws Exception { foo2.doAs( (PrivilegedAction) () -> CachedClientPool.extractKey("conf:key1,user_name", hiveConf)); - Assert.assertEquals("Key elements order shouldn't matter", key1, key2); + assertThat(key2).as("Key elements order shouldn't matter").isEqualTo(key1); key1 = foo1.doAs((PrivilegedAction) () -> CachedClientPool.extractKey("ugi", hiveConf)); key2 = bar.doAs((PrivilegedAction) () -> CachedClientPool.extractKey("ugi", hiveConf)); - Assert.assertNotEquals("Different users are not supposed to be equivalent", key1, key2); + assertThat(key2).as("Different users are not supposed to be equivalent").isNotEqualTo(key1); key2 = foo2.doAs((PrivilegedAction) () -> CachedClientPool.extractKey("ugi", hiveConf)); - Assert.assertNotEquals("Different UGI instances are not supposed to be equivalent", key1, key2); + assertThat(key2) + .as("Different UGI instances are not supposed to be equivalent") + .isNotEqualTo(key1); key1 = CachedClientPool.extractKey("ugi", hiveConf); key2 = CachedClientPool.extractKey("ugi,conf:key1", hiveConf); - Assert.assertNotEquals( - "Keys with different number of elements are not supposed to be equivalent", key1, key2); + assertThat(key2) + .as("Keys with different number of elements are not supposed to be equivalent") + .isNotEqualTo(key1); Configuration conf1 = new Configuration(hiveConf); Configuration conf2 = new Configuration(hiveConf); @@ -93,31 +97,33 @@ public void testCacheKey() throws Exception { conf1.set("key1", "val"); key1 = CachedClientPool.extractKey("conf:key1", conf1); key2 = CachedClientPool.extractKey("conf:key1", conf2); - Assert.assertNotEquals( - "Config with different values are not supposed to be equivalent", key1, key2); + assertThat(key2) + .as("Config with different values are not supposed to be equivalent") + .isNotEqualTo(key1); conf2.set("key1", "val"); conf2.set("key2", "val"); key2 = CachedClientPool.extractKey("conf:key2", conf2); - Assert.assertNotEquals( - "Config with different keys are not supposed to be equivalent", key1, key2); + assertThat(key2) + .as("Config with different keys are not supposed to be equivalent") + .isNotEqualTo(key1); key1 = CachedClientPool.extractKey("conf:key1,ugi", conf1); key2 = CachedClientPool.extractKey("ugi,conf:key1", conf2); - Assert.assertEquals("Config with same key/value should be equivalent", key1, key2); + assertThat(key2).as("Config with same key/value should be equivalent").isEqualTo(key1); conf1.set("key2", "val"); key1 = CachedClientPool.extractKey("conf:key2 ,conf:key1", conf1); key2 = CachedClientPool.extractKey("conf:key2,conf:key1", conf2); - Assert.assertEquals("Config with same key/value should be equivalent", key1, key2); + assertThat(key2).as("Config with same key/value should be equivalent").isEqualTo(key1); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> CachedClientPool.extractKey("ugi,ugi", hiveConf), "Duplicate key elements should result in an error") .isInstanceOf(ValidationException.class) .hasMessageContaining("UGI key element already specified"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> CachedClientPool.extractKey("conf:k1,conf:k2,CONF:k1", hiveConf), "Duplicate conf key elements should result in an error") .isInstanceOf(ValidationException.class) @@ -153,16 +159,16 @@ public void testHmsCatalog() { HiveClientPool pool3 = ((CachedClientPool) catalog3.clientPool()).clientPool(); HiveClientPool pool4 = ((CachedClientPool) catalog4.clientPool()).clientPool(); - Assert.assertSame(pool1, pool2); - Assert.assertNotSame(pool3, pool1); - Assert.assertNotSame(pool3, pool2); - Assert.assertNotSame(pool3, pool4); - Assert.assertNotSame(pool4, pool1); - Assert.assertNotSame(pool4, pool2); + assertThat(pool2).isSameAs(pool1); + assertThat(pool1).isNotSameAs(pool3); + assertThat(pool2).isNotSameAs(pool3); + assertThat(pool4).isNotSameAs(pool3); + assertThat(pool1).isNotSameAs(pool4); + assertThat(pool2).isNotSameAs(pool4); - Assert.assertEquals("foo", pool1.hiveConf().get(HiveCatalog.HIVE_CONF_CATALOG)); - Assert.assertEquals("bar", pool3.hiveConf().get(HiveCatalog.HIVE_CONF_CATALOG)); - Assert.assertNull(pool4.hiveConf().get(HiveCatalog.HIVE_CONF_CATALOG)); + assertThat(pool1.hiveConf().get(HiveCatalog.HIVE_CONF_CATALOG)).isEqualTo("foo"); + assertThat(pool3.hiveConf().get(HiveCatalog.HIVE_CONF_CATALOG)).isEqualTo("bar"); + assertThat(pool4.hiveConf().get(HiveCatalog.HIVE_CONF_CATALOG)).isNull(); pool1.close(); pool3.close(); diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java index c60740c854cb..d4ac49868488 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java @@ -26,14 +26,17 @@ import static org.apache.iceberg.TableProperties.CURRENT_SNAPSHOT_TIMESTAMP; import static org.apache.iceberg.TableProperties.DEFAULT_PARTITION_SPEC; import static org.apache.iceberg.TableProperties.DEFAULT_SORT_ORDER; +import static org.apache.iceberg.TableProperties.SNAPSHOT_COUNT; import static org.apache.iceberg.expressions.Expressions.bucket; import static org.apache.iceberg.types.Types.NestedField.required; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatNoException; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import java.io.IOException; +import java.nio.file.Path; import java.util.List; import java.util.Map; import java.util.Set; @@ -79,11 +82,8 @@ import org.apache.iceberg.types.Types; import org.apache.iceberg.util.JsonUtil; import org.apache.thrift.TException; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.jupiter.api.Assertions; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestHiveCatalog extends HiveMetastoreTest { private static ImmutableMap meta = @@ -92,7 +92,7 @@ public class TestHiveCatalog extends HiveMetastoreTest { "group", "iceberg", "comment", "iceberg hiveCatalog test"); - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; private Schema getTestSchema() { return new Schema( @@ -105,7 +105,7 @@ public void testCreateTableBuilder() throws Exception { Schema schema = getTestSchema(); PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("data", 16).build(); TableIdentifier tableIdent = TableIdentifier.of(DB_NAME, "tbl"); - String location = temp.newFolder("tbl").toString(); + String location = temp.resolve("tbl").toString(); try { Table table = @@ -117,11 +117,11 @@ public void testCreateTableBuilder() throws Exception { .withProperty("key2", "value2") .create(); - Assert.assertEquals(location, table.location()); - Assert.assertEquals(2, table.schema().columns().size()); - Assert.assertEquals(1, table.spec().fields().size()); - Assert.assertEquals("value1", table.properties().get("key1")); - Assert.assertEquals("value2", table.properties().get("key2")); + assertThat(table.location()).isEqualTo(location); + assertThat(table.schema().columns()).hasSize(2); + assertThat(table.spec().fields()).hasSize(1); + assertThat(table.properties()).containsEntry("key1", "value1"); + assertThat(table.properties()).containsEntry("key2", "value2"); } finally { catalog.dropTable(tableIdent); } @@ -132,18 +132,18 @@ public void testCreateTableWithCaching() throws Exception { Schema schema = getTestSchema(); PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("data", 16).build(); TableIdentifier tableIdent = TableIdentifier.of(DB_NAME, "tbl"); - String location = temp.newFolder("tbl").toString(); + String location = temp.resolve("tbl").toString(); ImmutableMap properties = ImmutableMap.of("key1", "value1", "key2", "value2"); Catalog cachingCatalog = CachingCatalog.wrap(catalog); try { Table table = cachingCatalog.createTable(tableIdent, schema, spec, location, properties); - Assert.assertEquals(location, table.location()); - Assert.assertEquals(2, table.schema().columns().size()); - Assert.assertEquals(1, table.spec().fields().size()); - Assert.assertEquals("value1", table.properties().get("key1")); - Assert.assertEquals("value2", table.properties().get("key2")); + assertThat(table.location()).isEqualTo(location); + assertThat(table.schema().columns()).hasSize(2); + assertThat(table.spec().fields()).hasSize(1); + assertThat(table.properties()).containsEntry("key1", "value1"); + assertThat(table.properties()).containsEntry("key2", "value2"); } finally { cachingCatalog.dropTable(tableIdent); } @@ -151,20 +151,22 @@ public void testCreateTableWithCaching() throws Exception { @Test public void testInitialize() { - Assertions.assertDoesNotThrow( - () -> { - HiveCatalog catalog = new HiveCatalog(); - catalog.initialize("hive", Maps.newHashMap()); - }); + assertThatNoException() + .isThrownBy( + () -> { + HiveCatalog catalog = new HiveCatalog(); + catalog.initialize("hive", Maps.newHashMap()); + }); } @Test public void testToStringWithoutSetConf() { - Assertions.assertDoesNotThrow( - () -> { - HiveCatalog catalog = new HiveCatalog(); - catalog.toString(); - }); + assertThatNoException() + .isThrownBy( + () -> { + HiveCatalog catalog = new HiveCatalog(); + catalog.toString(); + }); } @Test @@ -175,16 +177,16 @@ public void testInitializeCatalogWithProperties() { HiveCatalog catalog = new HiveCatalog(); catalog.initialize("hive", properties); - Assert.assertEquals(catalog.getConf().get("hive.metastore.uris"), "thrift://examplehost:9083"); - Assert.assertEquals( - catalog.getConf().get("hive.metastore.warehouse.dir"), "/user/hive/testwarehouse"); + assertThat(catalog.getConf().get("hive.metastore.uris")).isEqualTo("thrift://examplehost:9083"); + assertThat(catalog.getConf().get("hive.metastore.warehouse.dir")) + .isEqualTo("/user/hive/testwarehouse"); } @Test public void testCreateTableTxnBuilder() throws Exception { Schema schema = getTestSchema(); TableIdentifier tableIdent = TableIdentifier.of(DB_NAME, "tbl"); - String location = temp.newFolder("tbl").toString(); + String location = temp.resolve("tbl").toString(); try { Transaction txn = @@ -192,9 +194,9 @@ public void testCreateTableTxnBuilder() throws Exception { txn.commitTransaction(); Table table = catalog.loadTable(tableIdent); - Assert.assertEquals(location, table.location()); - Assert.assertEquals(2, table.schema().columns().size()); - Assert.assertTrue(table.spec().isUnpartitioned()); + assertThat(table.location()).isEqualTo(location); + assertThat(table.schema().columns()).hasSize(2); + assertThat(table.spec().isUnpartitioned()).isTrue(); } finally { catalog.dropTable(tableIdent); } @@ -205,7 +207,7 @@ public void testReplaceTxnBuilder() throws Exception { Schema schema = getTestSchema(); PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("data", 16).build(); TableIdentifier tableIdent = TableIdentifier.of(DB_NAME, "tbl"); - String location = temp.newFolder("tbl").toString(); + String location = temp.resolve("tbl").toString(); try { Transaction createTxn = @@ -218,9 +220,9 @@ public void testReplaceTxnBuilder() throws Exception { createTxn.commitTransaction(); Table table = catalog.loadTable(tableIdent); - Assert.assertEquals(1, table.spec().fields().size()); + assertThat(table.spec().fields()).hasSize(1); - String newLocation = temp.newFolder("tbl-2").toString(); + String newLocation = temp.resolve("tbl-2").toString(); Transaction replaceTxn = catalog @@ -231,17 +233,19 @@ public void testReplaceTxnBuilder() throws Exception { replaceTxn.commitTransaction(); table = catalog.loadTable(tableIdent); - Assert.assertEquals(newLocation, table.location()); - Assert.assertNull(table.currentSnapshot()); + assertThat(table.location()).isEqualTo(newLocation); + assertThat(table.currentSnapshot()).isNull(); PartitionSpec v1Expected = PartitionSpec.builderFor(table.schema()) .alwaysNull("data", "data_bucket") .withSpecId(1) .build(); - Assert.assertEquals("Table should have a spec with one void field", v1Expected, table.spec()); + assertThat(table.spec()) + .as("Table should have a spec with one void field") + .isEqualTo(v1Expected); - Assert.assertEquals("value1", table.properties().get("key1")); - Assert.assertEquals("value2", table.properties().get("key2")); + assertThat(table.properties()).containsEntry("key1", "value1"); + assertThat(table.properties()).containsEntry("key2", "value2"); } finally { catalog.dropTable(tableIdent); } @@ -267,13 +271,13 @@ private void createTableAndVerifyOwner( Schema schema = getTestSchema(); PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("data", 16).build(); TableIdentifier tableIdent = TableIdentifier.of(db, tbl); - String location = temp.newFolder(tbl).toString(); + String location = temp.resolve(tbl).toString(); try { Table table = catalog.createTable(tableIdent, schema, spec, location, properties); org.apache.hadoop.hive.metastore.api.Table hmsTable = metastoreClient.getTable(db, tbl); - Assert.assertEquals(owner, hmsTable.getOwner()); + assertThat(hmsTable.getOwner()).isEqualTo(owner); Map hmsTableParams = hmsTable.getParameters(); - Assert.assertFalse(hmsTableParams.containsKey(HiveCatalog.HMS_TABLE_OWNER)); + assertThat(hmsTableParams).doesNotContainKey(HiveCatalog.HMS_TABLE_OWNER); } finally { catalog.dropTable(tableIdent); } @@ -287,12 +291,12 @@ public void testCreateTableDefaultSortOrder() throws Exception { try { Table table = catalog.createTable(tableIdent, schema, spec); - Assert.assertEquals("Order ID must match", 0, table.sortOrder().orderId()); - Assert.assertTrue("Order must unsorted", table.sortOrder().isUnsorted()); + assertThat(table.sortOrder().orderId()).as("Order ID must match").isEqualTo(0); + assertThat(table.sortOrder().isUnsorted()).as("Order must unsorted").isTrue(); - Assert.assertFalse( - "Must not have default sort order in catalog", - hmsTableParameters().containsKey(DEFAULT_SORT_ORDER)); + assertThat(hmsTableParameters()) + .as("Must not have default sort order in catalog") + .doesNotContainKey(DEFAULT_SORT_ORDER); } finally { catalog.dropTable(tableIdent); } @@ -313,16 +317,19 @@ public void testCreateTableCustomSortOrder() throws Exception { .withSortOrder(order) .create(); SortOrder sortOrder = table.sortOrder(); - Assert.assertEquals("Order ID must match", 1, sortOrder.orderId()); - Assert.assertEquals("Order must have 1 field", 1, sortOrder.fields().size()); - Assert.assertEquals("Direction must match ", ASC, sortOrder.fields().get(0).direction()); - Assert.assertEquals( - "Null order must match ", NULLS_FIRST, sortOrder.fields().get(0).nullOrder()); + assertThat(sortOrder.orderId()).as("Order ID must match").isEqualTo(1); + assertThat(sortOrder.fields()).as("Order must have 1 field").hasSize(1); + assertThat(sortOrder.fields().get(0).direction()).as("Direction must match ").isEqualTo(ASC); + assertThat(sortOrder.fields().get(0).nullOrder()) + .as("Null order must match ") + .isEqualTo(NULLS_FIRST); Transform transform = Transforms.identity(Types.IntegerType.get()); - Assert.assertEquals("Transform must match", transform, sortOrder.fields().get(0).transform()); + assertThat(sortOrder.fields().get(0).transform()) + .as("Transform must match") + .isEqualTo(transform); - Assert.assertEquals( - SortOrderParser.toJson(table.sortOrder()), hmsTableParameters().get(DEFAULT_SORT_ORDER)); + assertThat(hmsTableParameters()) + .containsEntry(DEFAULT_SORT_ORDER, SortOrderParser.toJson(table.sortOrder())); } finally { catalog.dropTable(tableIdent); } @@ -334,18 +341,17 @@ public void testCreateNamespace() throws Exception { catalog.createNamespace(namespace1, meta); Database database1 = metastoreClient.getDatabase(namespace1.toString()); - Assert.assertTrue(database1.getParameters().get("owner").equals("apache")); - Assert.assertTrue(database1.getParameters().get("group").equals("iceberg")); + assertThat(database1.getParameters()).containsEntry("owner", "apache"); + assertThat(database1.getParameters()).containsEntry("group", "iceberg"); - Assert.assertEquals( - "There no same location for db and namespace", - database1.getLocationUri(), - defaultUri(namespace1)); + assertThat(defaultUri(namespace1)) + .as("There no same location for db and namespace") + .isEqualTo(database1.getLocationUri()); assertThatThrownBy(() -> catalog.createNamespace(namespace1)) .isInstanceOf(AlreadyExistsException.class) .hasMessage("Namespace '" + namespace1 + "' already exists!"); - String hiveLocalDir = temp.newFolder().toURI().toString(); + String hiveLocalDir = temp.toFile().toURI().toString(); // remove the trailing slash of the URI hiveLocalDir = hiveLocalDir.substring(0, hiveLocalDir.length() - 1); ImmutableMap newMeta = @@ -357,8 +363,9 @@ public void testCreateNamespace() throws Exception { catalog.createNamespace(namespace2, newMeta); Database database2 = metastoreClient.getDatabase(namespace2.toString()); - Assert.assertEquals( - "There no same location for db and namespace", database2.getLocationUri(), hiveLocalDir); + assertThat(hiveLocalDir) + .as("There no same location for db and namespace") + .isEqualTo(database2.getLocationUri()); } @Test @@ -437,8 +444,8 @@ private void createNamespaceAndVerifyOwnership( catalog.createNamespace(namespace, prop); Database db = metastoreClient.getDatabase(namespace.toString()); - Assert.assertEquals(expectedOwner, db.getOwnerName()); - Assert.assertEquals(expectedOwnerType, db.getOwnerType()); + assertThat(db.getOwnerName()).isEqualTo(expectedOwner); + assertThat(db.getOwnerType()).isEqualTo(expectedOwnerType); } @Test @@ -447,13 +454,13 @@ public void testListNamespace() throws TException { Namespace namespace1 = Namespace.of("dbname1"); catalog.createNamespace(namespace1, meta); namespaces = catalog.listNamespaces(namespace1); - Assert.assertTrue("Hive db not hive the namespace 'dbname1'", namespaces.isEmpty()); + assertThat(namespaces).as("Hive db not hive the namespace 'dbname1'").isEmpty(); Namespace namespace2 = Namespace.of("dbname2"); catalog.createNamespace(namespace2, meta); namespaces = catalog.listNamespaces(); - Assert.assertTrue("Hive db not hive the namespace 'dbname2'", namespaces.contains(namespace2)); + assertThat(namespaces).as("Hive db not hive the namespace 'dbname2'").contains(namespace2); } @Test @@ -463,12 +470,11 @@ public void testLoadNamespaceMeta() throws TException { catalog.createNamespace(namespace, meta); Map nameMata = catalog.loadNamespaceMetadata(namespace); - Assert.assertTrue(nameMata.get("owner").equals("apache")); - Assert.assertTrue(nameMata.get("group").equals("iceberg")); - Assert.assertEquals( - "There no same location for db and namespace", - nameMata.get("location"), - catalog.convertToDatabase(namespace, meta).getLocationUri()); + assertThat(nameMata).containsEntry("owner", "apache"); + assertThat(nameMata).containsEntry("group", "iceberg"); + assertThat(catalog.convertToDatabase(namespace, meta).getLocationUri()) + .as("There no same location for db and namespace") + .isEqualTo(nameMata.get("location")); } @Test @@ -477,10 +483,10 @@ public void testNamespaceExists() throws TException { catalog.createNamespace(namespace, meta); - Assert.assertTrue("Should true to namespace exist", catalog.namespaceExists(namespace)); - Assert.assertTrue( - "Should false to namespace doesn't exist", - !catalog.namespaceExists(Namespace.of("db2", "db2", "ns2"))); + assertThat(catalog.namespaceExists(namespace)).as("Should true to namespace exist").isTrue(); + assertThat(catalog.namespaceExists(Namespace.of("db2", "db2", "ns2"))) + .as("Should false to namespace doesn't exist") + .isFalse(); } @Test @@ -497,9 +503,9 @@ public void testSetNamespaceProperties() throws TException { "comment", "iceberg test")); Database database = metastoreClient.getDatabase(namespace.level(0)); - Assert.assertEquals(database.getParameters().get("owner"), "alter_apache"); - Assert.assertEquals(database.getParameters().get("test"), "test"); - Assert.assertEquals(database.getParameters().get("group"), "iceberg"); + assertThat(database.getParameters()).containsEntry("owner", "alter_apache"); + assertThat(database.getParameters()).containsEntry("test", "test"); + assertThat(database.getParameters()).containsEntry("group", "iceberg"); assertThatThrownBy( () -> catalog.setProperties(Namespace.of("db2", "db2", "ns2"), ImmutableMap.of())) @@ -684,8 +690,8 @@ private void setNamespaceOwnershipAndVerify( catalog.setProperties(Namespace.of(name), propToSet); Database database = metastoreClient.getDatabase(name); - Assert.assertEquals(expectedOwnerPostSet, database.getOwnerName()); - Assert.assertEquals(expectedOwnerTypePostSet, database.getOwnerType()); + assertThat(database.getOwnerName()).isEqualTo(expectedOwnerPostSet); + assertThat(database.getOwnerType()).isEqualTo(expectedOwnerTypePostSet); } @Test @@ -698,8 +704,8 @@ public void testRemoveNamespaceProperties() throws TException { Database database = metastoreClient.getDatabase(namespace.level(0)); - Assert.assertEquals(database.getParameters().get("owner"), null); - Assert.assertEquals(database.getParameters().get("group"), "iceberg"); + assertThat(database.getParameters()).doesNotContainKey("owner"); + assertThat(database.getParameters()).containsEntry("group", "iceberg"); assertThatThrownBy( () -> @@ -830,8 +836,8 @@ private void removeNamespaceOwnershipAndVerify( Database database = metastoreClient.getDatabase(name); - Assert.assertEquals(expectedOwnerPostRemove, database.getOwnerName()); - Assert.assertEquals(expectedOwnerTypePostRemove, database.getOwnerType()); + assertThat(database.getOwnerName()).isEqualTo(expectedOwnerPostRemove); + assertThat(database.getOwnerType()).isEqualTo(expectedOwnerTypePostRemove); } @Test @@ -843,18 +849,19 @@ public void testDropNamespace() throws TException { catalog.createNamespace(namespace, meta); catalog.createTable(identifier, schema); Map nameMata = catalog.loadNamespaceMetadata(namespace); - Assert.assertTrue(nameMata.get("owner").equals("apache")); - Assert.assertTrue(nameMata.get("group").equals("iceberg")); + assertThat(nameMata).containsEntry("owner", "apache"); + assertThat(nameMata).containsEntry("group", "iceberg"); assertThatThrownBy(() -> catalog.dropNamespace(namespace)) .isInstanceOf(NamespaceNotEmptyException.class) .hasMessage("Namespace dbname_drop is not empty. One or more tables exist."); - Assert.assertTrue(catalog.dropTable(identifier, true)); - Assert.assertTrue( - "Should fail to drop namespace if it is not empty", catalog.dropNamespace(namespace)); - Assert.assertFalse( - "Should fail to drop when namespace doesn't exist", - catalog.dropNamespace(Namespace.of("db.ns1"))); + assertThat(catalog.dropTable(identifier, true)).isTrue(); + assertThat(catalog.dropNamespace(namespace)) + .as("Should fail to drop namespace if it is not empty") + .isTrue(); + assertThat(catalog.dropNamespace(Namespace.of("db.ns1"))) + .as("Should fail to drop when namespace doesn't exist") + .isFalse(); assertThatThrownBy(() -> catalog.loadNamespaceMetadata(namespace)) .isInstanceOf(NoSuchNamespaceException.class) .hasMessage("Namespace does not exist: dbname_drop"); @@ -868,7 +875,7 @@ public void testDropTableWithoutMetadataFile() { String metadataFileLocation = catalog.newTableOps(identifier).current().metadataFileLocation(); TableOperations ops = catalog.newTableOps(identifier); ops.io().deleteFile(metadataFileLocation); - Assert.assertTrue(catalog.dropTable(identifier)); + assertThat(catalog.dropTable(identifier)).isTrue(); assertThatThrownBy(() -> catalog.loadTable(identifier)) .isInstanceOf(NoSuchTableException.class) .hasMessageContaining("Table does not exist:"); @@ -884,11 +891,13 @@ public void testTableName() { catalog.buildTable(tableIdent, schema).withPartitionSpec(spec).create(); Table table = catalog.loadTable(tableIdent); - Assert.assertEquals("Name must match", "hive.hivedb.tbl", table.name()); + assertThat(table.name()).as("Name must match").isEqualTo("hive.hivedb.tbl"); TableIdentifier snapshotsTableIdent = TableIdentifier.of(DB_NAME, "tbl", "snapshots"); Table snapshotsTable = catalog.loadTable(snapshotsTableIdent); - Assert.assertEquals("Name must match", "hive.hivedb.tbl.snapshots", snapshotsTable.name()); + assertThat(snapshotsTable.name()) + .as("Name must match") + .isEqualTo("hive.hivedb.tbl.snapshots"); } finally { catalog.dropTable(tableIdent); } @@ -905,12 +914,12 @@ private String defaultUri(Namespace namespace) throws TException { public void testUUIDinTableProperties() throws Exception { Schema schema = getTestSchema(); TableIdentifier tableIdentifier = TableIdentifier.of(DB_NAME, "tbl"); - String location = temp.newFolder("tbl").toString(); + String location = temp.resolve("tbl").toString(); try { catalog.buildTable(tableIdentifier, schema).withLocation(location).create(); - Assert.assertNotNull(hmsTableParameters().get(TableProperties.UUID)); + assertThat(hmsTableParameters()).containsKey(TableProperties.UUID); } finally { catalog.dropTable(tableIdentifier); } @@ -920,17 +929,18 @@ public void testUUIDinTableProperties() throws Exception { public void testSnapshotStatsTableProperties() throws Exception { Schema schema = getTestSchema(); TableIdentifier tableIdentifier = TableIdentifier.of(DB_NAME, "tbl"); - String location = temp.newFolder("tbl").toString(); + String location = temp.resolve("tbl").toString(); try { catalog.buildTable(tableIdentifier, schema).withLocation(location).create(); // check whether parameters are in expected state Map parameters = hmsTableParameters(); - Assert.assertEquals("0", parameters.get(TableProperties.SNAPSHOT_COUNT)); - Assert.assertNull(parameters.get(CURRENT_SNAPSHOT_SUMMARY)); - Assert.assertNull(parameters.get(CURRENT_SNAPSHOT_ID)); - Assert.assertNull(parameters.get(CURRENT_SNAPSHOT_TIMESTAMP)); + assertThat(parameters).containsEntry(SNAPSHOT_COUNT, "0"); + assertThat(parameters) + .doesNotContainKey(CURRENT_SNAPSHOT_SUMMARY) + .doesNotContainKey(CURRENT_SNAPSHOT_ID) + .doesNotContainKey(CURRENT_SNAPSHOT_TIMESTAMP); // create a snapshot Table icebergTable = catalog.loadTable(tableIdentifier); @@ -945,16 +955,16 @@ public void testSnapshotStatsTableProperties() throws Exception { // check whether parameters are in expected state parameters = hmsTableParameters(); - Assert.assertEquals("1", parameters.get(TableProperties.SNAPSHOT_COUNT)); + assertThat(parameters).containsEntry(SNAPSHOT_COUNT, "1"); String summary = JsonUtil.mapper().writeValueAsString(icebergTable.currentSnapshot().summary()); - Assert.assertEquals(summary, parameters.get(CURRENT_SNAPSHOT_SUMMARY)); + assertThat(parameters).containsEntry(CURRENT_SNAPSHOT_SUMMARY, summary); long snapshotId = icebergTable.currentSnapshot().snapshotId(); - Assert.assertEquals(String.valueOf(snapshotId), parameters.get(CURRENT_SNAPSHOT_ID)); - Assert.assertEquals( - String.valueOf(icebergTable.currentSnapshot().timestampMillis()), - parameters.get(CURRENT_SNAPSHOT_TIMESTAMP)); - + assertThat(parameters).containsEntry(CURRENT_SNAPSHOT_ID, String.valueOf(snapshotId)); + assertThat(parameters) + .containsEntry( + CURRENT_SNAPSHOT_TIMESTAMP, + String.valueOf(icebergTable.currentSnapshot().timestampMillis())); } finally { catalog.dropTable(tableIdentifier); } @@ -974,10 +984,10 @@ public void testSetSnapshotSummary() throws Exception { for (int i = 0; i < 100; i++) { summary.put(String.valueOf(i), "value"); } - Assert.assertTrue(JsonUtil.mapper().writeValueAsString(summary).length() < 4000); + assertThat(JsonUtil.mapper().writeValueAsString(summary).length()).isLessThan(4000); Map parameters = Maps.newHashMap(); ops.setSnapshotSummary(parameters, snapshot); - Assert.assertEquals("The snapshot summary must be in parameters", 1, parameters.size()); + assertThat(parameters).as("The snapshot summary must be in parameters").hasSize(1); // create a snapshot summary whose json string size exceeds the limit for (int i = 0; i < 1000; i++) { @@ -985,13 +995,12 @@ public void testSetSnapshotSummary() throws Exception { } long summarySize = JsonUtil.mapper().writeValueAsString(summary).length(); // the limit has been updated to 4000 instead of the default value(32672) - Assert.assertTrue(summarySize > 4000 && summarySize < 32672); + assertThat(summarySize).isGreaterThan(4000).isLessThan(32672); parameters.remove(CURRENT_SNAPSHOT_SUMMARY); ops.setSnapshotSummary(parameters, snapshot); - Assert.assertEquals( - "The snapshot summary must not be in parameters due to the size limit", - 0, - parameters.size()); + assertThat(parameters) + .as("The snapshot summary must not be in parameters due to the size limit") + .isEmpty(); } @Test @@ -1010,18 +1019,19 @@ public void testNotExposeTableProperties() { parameters.put(DEFAULT_SORT_ORDER, "sortOrder"); ops.setSnapshotStats(metadata, parameters); - Assert.assertNull(parameters.get(CURRENT_SNAPSHOT_SUMMARY)); - Assert.assertNull(parameters.get(CURRENT_SNAPSHOT_ID)); - Assert.assertNull(parameters.get(CURRENT_SNAPSHOT_TIMESTAMP)); + assertThat(parameters) + .doesNotContainKey(CURRENT_SNAPSHOT_SUMMARY) + .doesNotContainKey(CURRENT_SNAPSHOT_ID) + .doesNotContainKey(CURRENT_SNAPSHOT_TIMESTAMP); ops.setSchema(metadata, parameters); - Assert.assertNull(parameters.get(CURRENT_SCHEMA)); + assertThat(parameters).doesNotContainKey(CURRENT_SCHEMA); ops.setPartitionSpec(metadata, parameters); - Assert.assertNull(parameters.get(DEFAULT_PARTITION_SPEC)); + assertThat(parameters).doesNotContainKey(DEFAULT_PARTITION_SPEC); ops.setSortOrder(metadata, parameters); - Assert.assertNull(parameters.get(DEFAULT_SORT_ORDER)); + assertThat(parameters).doesNotContainKey(DEFAULT_SORT_ORDER); } @Test @@ -1031,14 +1041,14 @@ public void testSetDefaultPartitionSpec() throws Exception { try { Table table = catalog.buildTable(tableIdent, schema).create(); - Assert.assertFalse( - "Must not have default partition spec", - hmsTableParameters().containsKey(TableProperties.DEFAULT_PARTITION_SPEC)); + assertThat(hmsTableParameters()) + .as("Must not have default partition spec") + .doesNotContainKey(TableProperties.DEFAULT_PARTITION_SPEC); table.updateSpec().addField(bucket("data", 16)).commit(); - Assert.assertEquals( - PartitionSpecParser.toJson(table.spec()), - hmsTableParameters().get(TableProperties.DEFAULT_PARTITION_SPEC)); + assertThat(hmsTableParameters()) + .containsEntry( + TableProperties.DEFAULT_PARTITION_SPEC, PartitionSpecParser.toJson(table.spec())); } finally { catalog.dropTable(tableIdent); } @@ -1052,8 +1062,8 @@ public void testSetCurrentSchema() throws Exception { try { Table table = catalog.buildTable(tableIdent, schema).create(); - Assert.assertEquals( - SchemaParser.toJson(table.schema()), hmsTableParameters().get(CURRENT_SCHEMA)); + assertThat(hmsTableParameters()) + .containsEntry(CURRENT_SCHEMA, SchemaParser.toJson(table.schema())); // add many new fields to make the schema json string exceed the limit UpdateSchema updateSchema = table.updateSchema(); @@ -1062,8 +1072,8 @@ public void testSetCurrentSchema() throws Exception { } updateSchema.commit(); - Assert.assertTrue(SchemaParser.toJson(table.schema()).length() > 32672); - Assert.assertNull(hmsTableParameters().get(CURRENT_SCHEMA)); + assertThat(SchemaParser.toJson(table.schema()).length()).isGreaterThan(32672); + assertThat(hmsTableParameters()).doesNotContainKey(CURRENT_SCHEMA); } finally { catalog.dropTable(tableIdent); } @@ -1081,10 +1091,9 @@ public void testConstructorWarehousePathWithEndSlash() { catalogWithSlash.initialize( "hive_catalog", ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, wareHousePath + "/")); - Assert.assertEquals( - "Should have trailing slash stripped", - wareHousePath, - catalogWithSlash.getConf().get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname)); + assertThat(catalogWithSlash.getConf().get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname)) + .as("Should have trailing slash stripped") + .isEqualTo(wareHousePath); } @Test @@ -1115,28 +1124,25 @@ public void testTablePropsDefinedAtCatalogLevel() { .withProperty("key5", "table-key5") .create(); - Assert.assertEquals( - "Table defaults set for the catalog must be added to the table properties.", - "catalog-default-key1", - table.properties().get("key1")); - Assert.assertEquals( - "Table property must override table default properties set at catalog level.", - "table-key2", - table.properties().get("key2")); - Assert.assertEquals( - "Table property override set at catalog level must override table default" - + " properties set at catalog level and table property specified.", - "catalog-override-key3", - table.properties().get("key3")); - Assert.assertEquals( - "Table override not in table props or defaults should be added to table properties", - "catalog-override-key4", - table.properties().get("key4")); - Assert.assertEquals( - "Table properties without any catalog level default or override should be added to table" - + " properties.", - "table-key5", - table.properties().get("key5")); + assertThat(table.properties()) + .as("Table defaults set for the catalog must be added to the table properties.") + .containsEntry("key1", "catalog-default-key1"); + assertThat(table.properties()) + .as("Table property must override table default properties set at catalog level.") + .containsEntry("key2", "table-key2"); + assertThat(table.properties()) + .as( + "Table property override set at catalog level must override table default" + + " properties set at catalog level and table property specified.") + .containsEntry("key3", "catalog-override-key3"); + assertThat(table.properties()) + .as("Table override not in table props or defaults should be added to table properties") + .containsEntry("key4", "catalog-override-key4"); + assertThat(table.properties()) + .as( + "Table properties without any catalog level default or override should be added to table" + + " properties.") + .containsEntry("key5", "table-key5"); } finally { hiveCatalog.dropTable(tableIdent); } @@ -1153,7 +1159,7 @@ public void testDatabaseLocationWithSlashInWarehouseDir() { Database database = catalog.convertToDatabase(Namespace.of("database"), ImmutableMap.of()); - Assert.assertEquals("s3://bucket/database.db", database.getLocationUri()); + assertThat(database.getLocationUri()).isEqualTo("s3://bucket/database.db"); } @Test diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveClientPool.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveClientPool.java index 54638651866b..5a565d0e983c 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveClientPool.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveClientPool.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.hive; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.ByteArrayInputStream; import java.io.IOException; import java.io.InputStream; @@ -33,11 +36,9 @@ import org.apache.hadoop.hive.metastore.api.PrincipalType; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.thrift.transport.TTransportException; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; public class TestHiveClientPool { @@ -54,13 +55,13 @@ public class TestHiveClientPool { HiveClientPool clients; - @Before + @BeforeEach public void before() { HiveClientPool clientPool = new HiveClientPool(2, new Configuration()); clients = Mockito.spy(clientPool); } - @After + @AfterEach public void after() { clients.close(); clients = null; @@ -74,16 +75,14 @@ public void testConf() { HiveClientPool clientPool = new HiveClientPool(10, conf); HiveConf clientConf = clientPool.hiveConf(); - Assert.assertEquals( - conf.get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname), - clientConf.get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname)); - Assert.assertEquals(10, clientPool.poolSize()); + assertThat(clientConf.get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname)) + .isEqualTo(conf.get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname)); + assertThat(clientPool.poolSize()).isEqualTo(10); // 'hive.metastore.sasl.enabled' should be 'true' as defined in xml - Assert.assertEquals( - conf.get(HiveConf.ConfVars.METASTORE_USE_THRIFT_SASL.varname), - clientConf.get(HiveConf.ConfVars.METASTORE_USE_THRIFT_SASL.varname)); - Assert.assertTrue(clientConf.getBoolVar(HiveConf.ConfVars.METASTORE_USE_THRIFT_SASL)); + assertThat(clientConf.get(HiveConf.ConfVars.METASTORE_USE_THRIFT_SASL.varname)) + .isEqualTo(conf.get(HiveConf.ConfVars.METASTORE_USE_THRIFT_SASL.varname)); + assertThat(clientConf.getBoolVar(HiveConf.ConfVars.METASTORE_USE_THRIFT_SASL)).isTrue(); } private HiveConf createHiveConf() { @@ -100,7 +99,7 @@ private HiveConf createHiveConf() { @Test public void testNewClientFailure() { Mockito.doThrow(new RuntimeException("Connection exception")).when(clients).newClient(); - Assertions.assertThatThrownBy(() -> clients.run(Object::toString)) + assertThatThrownBy(() -> clients.run(Object::toString)) .isInstanceOf(RuntimeException.class) .hasMessage("Connection exception"); } @@ -112,7 +111,7 @@ public void testGetTablesFailsForNonReconnectableException() throws Exception { Mockito.doThrow(new MetaException("Another meta exception")) .when(hmsClient) .getTables(Mockito.anyString(), Mockito.anyString()); - Assertions.assertThatThrownBy(() -> clients.run(client -> client.getTables("default", "t"))) + assertThatThrownBy(() -> clients.run(client -> client.getTables("default", "t"))) .isInstanceOf(MetaException.class) .hasMessage("Another meta exception"); } @@ -132,7 +131,8 @@ public void testConnectionFailureRestoreForMetaException() throws Exception { Mockito.doReturn(databases).when(newClient).getAllDatabases(); // The return is OK when the reconnect method is called. - Assert.assertEquals(databases, clients.run(client -> client.getAllDatabases(), true)); + assertThat((List) clients.run(client -> client.getAllDatabases(), true)) + .isEqualTo(databases); // Verify that the method is called. Mockito.verify(clients).reconnect(hmsClient); @@ -159,8 +159,8 @@ public void testConnectionFailureRestoreForTTransportException() throws Exceptio FunctionType.JAVA, null)); Mockito.doReturn(response).when(newClient).getAllFunctions(); - - Assert.assertEquals(response, clients.run(client -> client.getAllFunctions(), true)); + assertThat((GetAllFunctionsResponse) clients.run(client -> client.getAllFunctions(), true)) + .isEqualTo(response); Mockito.verify(clients).reconnect(hmsClient); Mockito.verify(clients, Mockito.never()).reconnect(newClient); diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommitLocks.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommitLocks.java index 2de728d74b36..9704b9f7220c 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommitLocks.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommitLocks.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.hive; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.mockito.Mockito.any; import static org.mockito.Mockito.atLeastOnce; import static org.mockito.Mockito.doAnswer; @@ -58,12 +60,10 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; import org.apache.thrift.TException; -import org.assertj.core.api.Assertions; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; +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.mockito.AdditionalAnswers; import org.mockito.ArgumentCaptor; import org.mockito.invocation.InvocationOnMock; @@ -85,7 +85,7 @@ public class TestHiveCommitLocks extends HiveTableBaseTest { LockResponse notAcquiredLockResponse = new LockResponse(dummyLockId, LockState.NOT_ACQUIRED); ShowLocksResponse emptyLocks = new ShowLocksResponse(Lists.newArrayList()); - @BeforeClass + @BeforeAll public static void startMetastore() throws Exception { HiveMetastoreTest.startMetastore( ImmutableMap.of(HiveConf.ConfVars.HIVE_TXN_TIMEOUT.varname, "1s")); @@ -114,12 +114,12 @@ public static void startMetastore() throws Exception { spyCachedClientPool = spy(new CachedClientPool(hiveConf, Collections.emptyMap())); when(spyCachedClientPool.clientPool()).thenAnswer(invocation -> spyClientPool); - Assert.assertNotNull(spyClientRef.get()); + assertThat(spyClientRef.get()).isNotNull(); spyClient = spyClientRef.get(); } - @Before + @BeforeEach public void before() throws Exception { Table table = catalog.loadTable(TABLE_IDENTIFIER); ops = (HiveTableOperations) ((HasTableOperations) table).operations(); @@ -134,7 +134,7 @@ public void before() throws Exception { metadataV2 = ops.current(); - Assert.assertEquals(2, ops.current().schema().columns().size()); + assertThat(ops.current().schema().columns()).hasSize(2); spyOps = spy( @@ -148,7 +148,7 @@ public void before() throws Exception { reset(spyClient); } - @AfterClass + @AfterAll public static void cleanup() { try { spyClientPool.close(); @@ -165,7 +165,7 @@ public void testLockAcquisitionAtFirstTime() throws TException, InterruptedExcep spyOps.doCommit(metadataV2, metadataV1); - Assert.assertEquals(1, spyOps.current().schema().columns().size()); // should be 1 again + assertThat(spyOps.current().schema().columns()).hasSize(1); // should be 1 again } @Test @@ -184,7 +184,7 @@ public void testLockAcquisitionAfterRetries() throws TException, InterruptedExce spyOps.doCommit(metadataV2, metadataV1); - Assert.assertEquals(1, spyOps.current().schema().columns().size()); // should be 1 again + assertThat(spyOps.current().schema().columns()).hasSize(1); // should be 1 again } @Test @@ -202,7 +202,7 @@ public void testLockAcquisitionAfterFailedNotFoundLock() throws TException, Inte spyOps.doCommit(metadataV2, metadataV1); - Assert.assertEquals(1, spyOps.current().schema().columns().size()); // should be 1 again + assertThat(spyOps.current().schema().columns()).hasSize(1); // should be 1 again } @Test @@ -226,7 +226,7 @@ public void testLockAcquisitionAfterFailedAndFoundLock() throws TException, Inte spyOps.doCommit(metadataV2, metadataV1); - Assert.assertEquals(1, spyOps.current().schema().columns().size()); // should be 1 again + assertThat(spyOps.current().schema().columns()).hasSize(1); // should be 1 again } @Test @@ -280,7 +280,7 @@ public void testUnLockAfterInterruptedLock() throws TException { doNothing().when(spyClient).unlock(eq(dummyLockId)); doNothing().when(spyClient).heartbeat(eq(0L), eq(dummyLockId)); - Assertions.assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) .isInstanceOf(RuntimeException.class) .hasMessage( "org.apache.iceberg.hive.LockException: " @@ -304,7 +304,7 @@ public void testUnLockAfterInterruptedLockCheck() throws TException { doNothing().when(spyClient).unlock(eq(dummyLockId)); doNothing().when(spyClient).heartbeat(eq(0L), eq(dummyLockId)); - Assertions.assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) .isInstanceOf(RuntimeException.class) .hasMessage( "org.apache.iceberg.hive.LockException: " @@ -328,7 +328,7 @@ public void testUnLockAfterInterruptedGetTable() throws TException { doNothing().when(spyClient).unlock(eq(dummyLockId)); doNothing().when(spyClient).heartbeat(eq(0L), eq(dummyLockId)); - Assertions.assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) .isInstanceOf(RuntimeException.class) .hasMessage("Interrupted during commit"); @@ -363,7 +363,7 @@ public long getLockid() { public void testLockFailureAtFirstTime() throws TException { doReturn(notAcquiredLockResponse).when(spyClient).lock(any()); - Assertions.assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) .isInstanceOf(CommitFailedException.class) .hasMessage( "org.apache.iceberg.hive.LockException: " @@ -381,7 +381,7 @@ public void testLockFailureAfterRetries() throws TException { .when(spyClient) .checkLock(eq(dummyLockId)); - Assertions.assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) .isInstanceOf(CommitFailedException.class) .hasMessage( "org.apache.iceberg.hive.LockException: " @@ -393,7 +393,7 @@ public void testLockTimeoutAfterRetries() throws TException { doReturn(waitLockResponse).when(spyClient).lock(any()); doReturn(waitLockResponse).when(spyClient).checkLock(eq(dummyLockId)); - Assertions.assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) .isInstanceOf(CommitFailedException.class) .hasMessageStartingWith("org.apache.iceberg.hive.LockException") .hasMessageContaining("Timed out after") @@ -408,7 +408,7 @@ public void testPassThroughThriftExceptions() throws TException { .when(spyClient) .checkLock(eq(dummyLockId)); - Assertions.assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) .isInstanceOf(RuntimeException.class) .hasMessage( "org.apache.iceberg.hive.LockException: Metastore operation failed for hivedb.tbl"); @@ -427,7 +427,7 @@ public void testPassThroughInterruptions() throws TException { .when(spyClient) .checkLock(eq(dummyLockId)); - Assertions.assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) .isInstanceOf(CommitFailedException.class) .hasMessage( "org.apache.iceberg.hive.LockException: " @@ -488,7 +488,7 @@ public void testLockHeartbeatFailureDuringCommit() throws TException, Interrupte .when(spyClient) .heartbeat(eq(0L), eq(dummyLockId)); - Assertions.assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) .isInstanceOf(CommitFailedException.class) .hasMessage( "org.apache.iceberg.hive.LockException: " @@ -527,9 +527,10 @@ public void testNoLockCallsWithNoLock() throws TException { // Make sure that the expected parameter context values are set Map context = contextCaptor.getValue().getProperties(); - Assert.assertEquals(3, context.size()); - Assert.assertEquals( - context.get("expected_parameter_key"), HiveTableOperations.METADATA_LOCATION_PROP); - Assert.assertEquals(context.get("expected_parameter_value"), metadataV2.metadataFileLocation()); + assertThat(context).hasSize(3); + assertThat(HiveTableOperations.METADATA_LOCATION_PROP) + .isEqualTo(context.get("expected_parameter_key")); + assertThat(metadataV2.metadataFileLocation()) + .isEqualTo(context.get("expected_parameter_value")); } } diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommits.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommits.java index 887235a4ed09..aaa659042118 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommits.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommits.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.hive; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyBoolean; import static org.mockito.Mockito.doAnswer; @@ -39,9 +41,7 @@ import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.types.Types; import org.apache.thrift.TException; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestHiveCommits extends HiveTableBaseTest { @@ -58,7 +58,7 @@ public void testSuppressUnlockExceptions() { TableMetadata metadataV2 = ops.current(); - Assert.assertEquals(2, ops.current().schema().columns().size()); + assertThat(ops.current().schema().columns()).hasSize(2); HiveTableOperations spyOps = spy(ops); @@ -83,7 +83,7 @@ public void testSuppressUnlockExceptions() { ops.refresh(); // the commit must succeed - Assert.assertEquals(1, ops.current().schema().columns().size()); + assertThat(ops.current().schema().columns()).hasSize(1); } /** @@ -105,24 +105,24 @@ public void testThriftExceptionUnknownStateIfNotInHistoryFailureOnCommit() TableMetadata metadataV2 = ops.current(); - Assert.assertEquals(2, ops.current().schema().columns().size()); + assertThat(ops.current().schema().columns()).hasSize(2); HiveTableOperations spyOps = spy(ops); failCommitAndThrowException(spyOps); - Assertions.assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) .isInstanceOf(CommitStateUnknownException.class) .hasMessageStartingWith("Datacenter on fire"); ops.refresh(); - Assert.assertEquals("Current metadata should not have changed", metadataV2, ops.current()); - Assert.assertTrue("Current metadata should still exist", metadataFileExists(metadataV2)); - Assert.assertEquals( - "New metadata files should still exist, new location not in history but" - + " the commit may still succeed", - 3, - metadataFileCount(ops.current())); + assertThat(ops.current()).as("Current metadata should not have changed").isEqualTo(metadataV2); + assertThat(metadataFileExists(metadataV2)).as("Current metadata should still exist").isTrue(); + assertThat(metadataFileCount(ops.current())) + .as( + "New metadata files should still exist, new location not in history but" + + " the commit may still succeed") + .isEqualTo(3); } /** Pretends we throw an error while persisting that actually does commit serverside */ @@ -139,7 +139,7 @@ public void testThriftExceptionSuccessOnCommit() throws TException, InterruptedE TableMetadata metadataV2 = ops.current(); - Assert.assertEquals(2, ops.current().schema().columns().size()); + assertThat(ops.current().schema().columns()).hasSize(2); HiveTableOperations spyOps = spy(ops); @@ -151,13 +151,13 @@ public void testThriftExceptionSuccessOnCommit() throws TException, InterruptedE spyOps.commit(metadataV2, metadataV1); ops.refresh(); - Assert.assertNotEquals("Current metadata should have changed", metadataV2, ops.current()); - Assert.assertTrue( - "Current metadata file should still exist", metadataFileExists(ops.current())); - Assert.assertEquals( - "Commit should have been successful and new metadata file should be made", - 3, - metadataFileCount(ops.current())); + assertThat(ops.current()).as("Current metadata should have changed").isNotEqualTo(metadataV2); + assertThat(metadataFileExists(ops.current())) + .as("Current metadata file should still exist") + .isTrue(); + assertThat(metadataFileCount(ops.current())) + .as("Commit should have been successful and new metadata file should be made") + .isEqualTo(3); } /** @@ -177,26 +177,26 @@ public void testThriftExceptionUnknownFailedCommit() throws TException, Interrup TableMetadata metadataV2 = ops.current(); - Assert.assertEquals(2, ops.current().schema().columns().size()); + assertThat(ops.current().schema().columns()).hasSize(2); HiveTableOperations spyOps = spy(ops); failCommitAndThrowException(spyOps); breakFallbackCatalogCommitCheck(spyOps); - Assertions.assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) .isInstanceOf(CommitStateUnknownException.class) .hasMessageStartingWith("Datacenter on fire"); ops.refresh(); - Assert.assertEquals("Current metadata should not have changed", metadataV2, ops.current()); - Assert.assertTrue( - "Current metadata file should still exist", metadataFileExists(ops.current())); - Assert.assertEquals( - "Client could not determine outcome so new metadata file should also exist", - 3, - metadataFileCount(ops.current())); + assertThat(ops.current()).as("Current metadata should not have changed").isEqualTo(metadataV2); + assertThat(metadataFileExists(ops.current())) + .as("Current metadata file should still exist") + .isTrue(); + assertThat(metadataFileCount(ops.current())) + .as("Client could not determine outcome so new metadata file should also exist") + .isEqualTo(3); } /** @@ -216,22 +216,23 @@ public void testThriftExceptionsUnknownSuccessCommit() throws TException, Interr TableMetadata metadataV2 = ops.current(); - Assert.assertEquals(2, ops.current().schema().columns().size()); + assertThat(ops.current().schema().columns()).hasSize(2); HiveTableOperations spyOps = spy(ops); commitAndThrowException(ops, spyOps); breakFallbackCatalogCommitCheck(spyOps); - Assertions.assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) .isInstanceOf(CommitStateUnknownException.class) .hasMessageStartingWith("Datacenter on fire"); ops.refresh(); - Assert.assertFalse("Current metadata should have changed", ops.current().equals(metadataV2)); - Assert.assertTrue( - "Current metadata file should still exist", metadataFileExists(ops.current())); + assertThat(ops.current()).as("Current metadata should have changed").isNotEqualTo(metadataV2); + assertThat(metadataFileExists(ops.current())) + .as("Current metadata file should still exist") + .isTrue(); } /** @@ -265,7 +266,7 @@ public void testThriftExceptionConcurrentCommit() throws TException, Interrupted TableMetadata metadataV2 = ops.current(); - Assert.assertEquals(2, ops.current().schema().columns().size()); + assertThat(ops.current().schema().columns()).hasSize(2); HiveTableOperations spyOps = spy(ops); @@ -287,30 +288,29 @@ public void testThriftExceptionConcurrentCommit() throws TException, Interrupted spyOps.commit(metadataV2, metadataV1); ops.refresh(); - Assert.assertNotEquals("Current metadata should have changed", metadataV2, ops.current()); - Assert.assertTrue( - "Current metadata file should still exist", metadataFileExists(ops.current())); - Assert.assertEquals( - "The column addition from the concurrent commit should have been successful", - 2, - ops.current().schema().columns().size()); + assertThat(ops.current()).as("Current metadata should have changed").isNotEqualTo(metadataV2); + assertThat(metadataFileExists(ops.current())) + .as("Current metadata file should still exist") + .isTrue(); + assertThat(ops.current().schema().columns()) + .as("The column addition from the concurrent commit should have been successful") + .hasSize(2); } @Test public void testInvalidObjectException() { TableIdentifier badTi = TableIdentifier.of(DB_NAME, "`tbl`"); - Assert.assertThrows( - String.format("Invalid table name for %s.%s", DB_NAME, "`tbl`"), - ValidationException.class, - () -> catalog.createTable(badTi, schema, PartitionSpec.unpartitioned())); + assertThatThrownBy(() -> catalog.createTable(badTi, schema, PartitionSpec.unpartitioned())) + .isInstanceOf(ValidationException.class) + .hasMessage(String.format("Invalid Hive object for %s.%s", DB_NAME, "`tbl`")); } @Test public void testAlreadyExistsException() { - Assert.assertThrows( - String.format("Table already exists: %s.%s", DB_NAME, TABLE_NAME), - AlreadyExistsException.class, - () -> catalog.createTable(TABLE_IDENTIFIER, schema, PartitionSpec.unpartitioned())); + assertThatThrownBy( + () -> catalog.createTable(TABLE_IDENTIFIER, schema, PartitionSpec.unpartitioned())) + .isInstanceOf(AlreadyExistsException.class) + .hasMessage(String.format("Table already exists: %s.%s", DB_NAME, TABLE_NAME)); } /** Uses NoLock and pretends we throw an error because of a concurrent commit */ @@ -327,7 +327,7 @@ public void testNoLockThriftExceptionConcurrentCommit() throws TException, Inter TableMetadata metadataV2 = ops.current(); - Assert.assertEquals(2, ops.current().schema().columns().size()); + assertThat(ops.current().schema().columns()).hasSize(2); HiveTableOperations spyOps = spy(ops); @@ -342,14 +342,59 @@ public void testNoLockThriftExceptionConcurrentCommit() throws TException, Inter .persistTable(any(), anyBoolean(), any()); // Should throw a CommitFailedException so the commit could be retried - Assertions.assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) .isInstanceOf(CommitFailedException.class) .hasMessage("The table hivedb.tbl has been modified concurrently"); ops.refresh(); - Assert.assertEquals("Current metadata should not have changed", metadataV2, ops.current()); - Assert.assertTrue("Current metadata should still exist", metadataFileExists(metadataV2)); - Assert.assertEquals("New metadata files should not exist", 2, metadataFileCount(ops.current())); + assertThat(ops.current()).as("Current metadata should not have changed").isEqualTo(metadataV2); + assertThat(metadataFileExists(metadataV2)).as("Current metadata should still exist").isTrue(); + assertThat(metadataFileCount(ops.current())) + .as("New metadata files should not exist") + .isEqualTo(2); + } + + @Test + public void testLockExceptionUnknownSuccessCommit() throws TException, InterruptedException { + Table table = catalog.loadTable(TABLE_IDENTIFIER); + HiveTableOperations ops = (HiveTableOperations) ((HasTableOperations) table).operations(); + + TableMetadata metadataV1 = ops.current(); + + table.updateSchema().addColumn("n", Types.IntegerType.get()).commit(); + + ops.refresh(); + + TableMetadata metadataV2 = ops.current(); + + assertThat(ops.current().schema().columns()).hasSize(2); + + HiveTableOperations spyOps = spy(ops); + + // Simulate a communication error after a successful commit + doAnswer( + i -> { + org.apache.hadoop.hive.metastore.api.Table tbl = + i.getArgument(0, org.apache.hadoop.hive.metastore.api.Table.class); + String location = i.getArgument(2, String.class); + ops.persistTable(tbl, true, location); + throw new LockException("Datacenter on fire"); + }) + .when(spyOps) + .persistTable(any(), anyBoolean(), any()); + + assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) + .hasMessageContaining("Failed to heartbeat for hive lock while") + .isInstanceOf(CommitStateUnknownException.class); + + ops.refresh(); + + assertThat(ops.current().location()) + .as("Current metadata should have changed to metadata V1") + .isEqualTo(metadataV1.location()); + assertThat(metadataFileExists(ops.current())) + .as("Current metadata file should still exist") + .isTrue(); } private void commitAndThrowException( diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveMetastore.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveMetastore.java index 08a70e95a1ea..ef8bd7ee0ae3 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveMetastore.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveMetastore.java @@ -21,6 +21,7 @@ import static java.nio.file.Files.createTempDirectory; import static java.nio.file.attribute.PosixFilePermissions.asFileAttribute; import static java.nio.file.attribute.PosixFilePermissions.fromString; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; @@ -52,7 +53,6 @@ import org.apache.thrift.server.TThreadPoolServer; import org.apache.thrift.transport.TServerSocket; import org.apache.thrift.transport.TTransportFactory; -import org.junit.Assert; public class TestHiveMetastore { @@ -112,7 +112,7 @@ public class TestHiveMetastore { FileSystem fs = Util.getFs(localDirPath, new Configuration()); String errMsg = "Failed to delete " + localDirPath; try { - Assert.assertTrue(errMsg, fs.delete(localDirPath, true)); + assertThat(fs.delete(localDirPath, true)).as(errMsg).isTrue(); } catch (IOException e) { throw new RuntimeException(errMsg, e); } diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveSchemaUtil.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveSchemaUtil.java index c444df37b6ed..84d11d03a741 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveSchemaUtil.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveSchemaUtil.java @@ -19,6 +19,8 @@ package org.apache.iceberg.hive; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.Arrays; import java.util.List; @@ -32,9 +34,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestHiveSchemaUtil { private static final Schema SIMPLE_ICEBERG_SCHEMA = @@ -113,8 +113,8 @@ public class TestHiveSchemaUtil { @Test public void testSimpleSchemaConvertToIcebergSchema() { - Assert.assertEquals( - SIMPLE_ICEBERG_SCHEMA.asStruct(), HiveSchemaUtil.convert(SIMPLE_HIVE_SCHEMA).asStruct()); + assertThat(HiveSchemaUtil.convert(SIMPLE_HIVE_SCHEMA).asStruct()) + .isEqualTo(SIMPLE_ICEBERG_SCHEMA.asStruct()); } @Test @@ -127,27 +127,26 @@ public void testSimpleSchemaConvertToIcebergSchemaFromNameAndTypeLists() { .collect(Collectors.toList()); List comments = SIMPLE_HIVE_SCHEMA.stream().map(FieldSchema::getComment).collect(Collectors.toList()); - Assert.assertEquals( - SIMPLE_ICEBERG_SCHEMA.asStruct(), - HiveSchemaUtil.convert(names, types, comments).asStruct()); + assertThat(HiveSchemaUtil.convert(names, types, comments).asStruct()) + .isEqualTo(SIMPLE_ICEBERG_SCHEMA.asStruct()); } @Test public void testComplexSchemaConvertToIcebergSchema() { - Assert.assertEquals( - COMPLEX_ICEBERG_SCHEMA.asStruct(), HiveSchemaUtil.convert(COMPLEX_HIVE_SCHEMA).asStruct()); + assertThat(HiveSchemaUtil.convert(COMPLEX_HIVE_SCHEMA).asStruct()) + .isEqualTo(COMPLEX_ICEBERG_SCHEMA.asStruct()); } @Test public void testSchemaConvertToIcebergSchemaForEveryPrimitiveType() { Schema schemaWithEveryType = HiveSchemaUtil.convert(getSupportedFieldSchemas()); - Assert.assertEquals(getSchemaWithSupportedTypes().asStruct(), schemaWithEveryType.asStruct()); + assertThat(schemaWithEveryType.asStruct()).isEqualTo(getSchemaWithSupportedTypes().asStruct()); } @Test public void testNotSupportedTypes() { for (FieldSchema notSupportedField : getNotSupportedFieldSchemas()) { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> HiveSchemaUtil.convert(Lists.newArrayList(Arrays.asList(notSupportedField)))) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Unsupported Hive type"); @@ -156,12 +155,12 @@ public void testNotSupportedTypes() { @Test public void testSimpleSchemaConvertToHiveSchema() { - Assert.assertEquals(SIMPLE_HIVE_SCHEMA, HiveSchemaUtil.convert(SIMPLE_ICEBERG_SCHEMA)); + assertThat(HiveSchemaUtil.convert(SIMPLE_ICEBERG_SCHEMA)).isEqualTo(SIMPLE_HIVE_SCHEMA); } @Test public void testComplexSchemaConvertToHiveSchema() { - Assert.assertEquals(COMPLEX_HIVE_SCHEMA, HiveSchemaUtil.convert(COMPLEX_ICEBERG_SCHEMA)); + assertThat(HiveSchemaUtil.convert(COMPLEX_ICEBERG_SCHEMA)).isEqualTo(COMPLEX_HIVE_SCHEMA); } @Test @@ -200,7 +199,7 @@ public void testConversionWithoutLastComment() { TypeInfoUtils.getTypeInfoFromTypeString(serdeConstants.STRING_TYPE_NAME)), Arrays.asList("customer comment")); - Assert.assertEquals(expected.asStruct(), schema.asStruct()); + assertThat(schema.asStruct()).isEqualTo(expected.asStruct()); } protected List getSupportedFieldSchemas() { @@ -252,7 +251,7 @@ protected Schema getSchemaWithSupportedTypes() { */ private void checkConvert(TypeInfo typeInfo, Type type) { // Convert to TypeInfo - Assert.assertEquals(typeInfo, HiveSchemaUtil.convert(type)); + assertThat(HiveSchemaUtil.convert(type)).isEqualTo(typeInfo); // Convert to Type assertEquals(type, HiveSchemaUtil.convert(typeInfo)); } @@ -265,13 +264,13 @@ private void checkConvert(TypeInfo typeInfo, Type type) { */ private void assertEquals(Type expected, Type actual) { if (actual.isPrimitiveType()) { - Assert.assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); } else { List expectedFields = ((Type.NestedType) expected).fields(); List actualFields = ((Type.NestedType) actual).fields(); for (int i = 0; i < expectedFields.size(); ++i) { assertEquals(expectedFields.get(i).type(), actualFields.get(i).type()); - Assert.assertEquals(expectedFields.get(i).name(), actualFields.get(i).name()); + assertThat(actualFields.get(i).name()).isEqualTo(expectedFields.get(i).name()); } } } diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveTableConcurrency.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveTableConcurrency.java index b8630364e837..e7608962cb85 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveTableConcurrency.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveTableConcurrency.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS; import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS; import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES; +import static org.assertj.core.api.Assertions.assertThat; import java.util.UUID; import java.util.concurrent.ExecutorService; @@ -32,11 +33,9 @@ import org.apache.iceberg.DataFiles; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Table; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors; import org.apache.iceberg.util.Tasks; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestHiveTableConcurrency extends HiveTableBaseTest { @@ -78,7 +77,7 @@ public synchronized void testConcurrentFastAppends() { }); icebergTable.refresh(); - Assert.assertEquals(20, icebergTable.currentSnapshot().allManifests(icebergTable.io()).size()); + assertThat(icebergTable.currentSnapshot().allManifests(icebergTable.io())).hasSize(20); } @Test @@ -109,7 +108,7 @@ public synchronized void testConcurrentConnections() throws InterruptedException } executorService.shutdown(); - Assert.assertTrue("Timeout", executorService.awaitTermination(3, TimeUnit.MINUTES)); - Assert.assertEquals(7, Iterables.size(icebergTable.snapshots())); + assertThat(executorService.awaitTermination(3, TimeUnit.MINUTES)).as("Timeout").isTrue(); + assertThat(icebergTable.snapshots()).hasSize(7); } } diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestLoadHiveCatalog.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestLoadHiveCatalog.java index 06bd79914fac..4886140cde86 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestLoadHiveCatalog.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestLoadHiveCatalog.java @@ -18,29 +18,30 @@ */ package org.apache.iceberg.hive; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.Collections; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; public class TestLoadHiveCatalog { private static TestHiveMetastore metastore; - @BeforeClass + @BeforeAll public static void startMetastore() throws Exception { HiveConf hiveConf = new HiveConf(TestLoadHiveCatalog.class); metastore = new TestHiveMetastore(); metastore.start(hiveConf); } - @AfterClass + @AfterAll public static void stopMetastore() throws Exception { if (metastore != null) { metastore.stop(); @@ -67,7 +68,7 @@ public void testCustomCacheKeys() throws Exception { CachedClientPool clientPool1 = (CachedClientPool) hiveCatalog1.clientPool(); CachedClientPool clientPool2 = (CachedClientPool) hiveCatalog2.clientPool(); - Assert.assertSame(clientPool1.clientPool(), clientPool2.clientPool()); + assertThat(clientPool2.clientPool()).isSameAs(clientPool1.clientPool()); Configuration conf1 = new Configuration(metastore.hiveConf()); Configuration conf2 = new Configuration(metastore.hiveConf()); @@ -89,7 +90,7 @@ public void testCustomCacheKeys() throws Exception { conf2); clientPool1 = (CachedClientPool) hiveCatalog1.clientPool(); clientPool2 = (CachedClientPool) hiveCatalog2.clientPool(); - Assert.assertSame(clientPool1.clientPool(), clientPool2.clientPool()); + assertThat(clientPool2.clientPool()).isSameAs(clientPool1.clientPool()); conf2.set("any.key", "any.value2"); hiveCatalog2 = @@ -100,6 +101,6 @@ public void testCustomCacheKeys() throws Exception { ImmutableMap.of(CatalogProperties.CLIENT_POOL_CACHE_KEYS, "conf:any.key"), conf2); clientPool2 = (CachedClientPool) hiveCatalog2.clientPool(); - Assert.assertNotSame(clientPool1.clientPool(), clientPool2.clientPool()); + assertThat(clientPool2.clientPool()).isNotSameAs(clientPool1.clientPool()); } } diff --git a/hive3/build.gradle b/hive3/build.gradle index c94e46596c7d..4b04a5e03836 100644 --- a/hive3/build.gradle +++ b/hive3/build.gradle @@ -19,7 +19,7 @@ // overriding default versions for Hive3 def hiveVersion = "3.1.3" -def hadoopVersion = "3.3.5" +def hadoopVersion = "3.3.6" def tezVersion = "0.10.2" project(':iceberg-hive3') { diff --git a/hive3/src/main/java/org/apache/iceberg/mr/hive/vector/CompatibilityHiveVectorUtils.java b/hive3/src/main/java/org/apache/iceberg/mr/hive/vector/CompatibilityHiveVectorUtils.java index 48207a1b3a99..619e858daac9 100644 --- a/hive3/src/main/java/org/apache/iceberg/mr/hive/vector/CompatibilityHiveVectorUtils.java +++ b/hive3/src/main/java/org/apache/iceberg/mr/hive/vector/CompatibilityHiveVectorUtils.java @@ -21,7 +21,6 @@ import java.nio.charset.StandardCharsets; import java.sql.Date; import java.sql.Timestamp; -import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.hive.common.type.HiveDecimal; import org.apache.hadoop.hive.common.type.HiveIntervalDayTime; import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth; @@ -67,7 +66,7 @@ public static MapWork findMapWork(JobConf job) { LOG.debug("Initializing for input {}", inputName); } String prefixes = job.get(DagUtils.TEZ_MERGE_WORK_FILE_PREFIXES); - if (prefixes != null && !StringUtils.isBlank(prefixes)) { + if (prefixes != null && !prefixes.trim().isEmpty()) { // Currently SMB is broken, so we cannot check if it's compatible with IO elevator. // So, we don't use the below code that would get the correct MapWork. See HIVE-16985. return null; diff --git a/hive3/src/main/java/org/apache/iceberg/mr/hive/vector/HiveVectorizedReader.java b/hive3/src/main/java/org/apache/iceberg/mr/hive/vector/HiveVectorizedReader.java index e6f9b8f8cc98..0817060b811b 100644 --- a/hive3/src/main/java/org/apache/iceberg/mr/hive/vector/HiveVectorizedReader.java +++ b/hive3/src/main/java/org/apache/iceberg/mr/hive/vector/HiveVectorizedReader.java @@ -21,7 +21,6 @@ import java.io.IOException; import java.util.List; import java.util.Map; -import org.apache.commons.lang3.ArrayUtils; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; import org.apache.hadoop.hive.ql.io.IOConstants; @@ -101,7 +100,7 @@ public static CloseableIterable reader( } } - partitionColIndices = ArrayUtils.toPrimitive(partitionColIndicesList.toArray(new Integer[0])); + partitionColIndices = partitionColIndicesList.stream().mapToInt(Integer::intValue).toArray(); partitionValues = partitionValuesList.toArray(new Object[0]); ColumnProjectionUtils.setReadColumns(job, readColumnIds); diff --git a/nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java b/nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java index 0c6f2d4c5911..0c91cfb1dbe1 100644 --- a/nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java +++ b/nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java @@ -44,6 +44,7 @@ import org.projectnessie.client.NessieClientBuilder; import org.projectnessie.client.NessieConfigConstants; import org.projectnessie.client.api.NessieApiV1; +import org.projectnessie.client.api.NessieApiV2; import org.projectnessie.client.http.HttpClientBuilder; import org.projectnessie.model.ContentKey; import org.projectnessie.model.TableReference; @@ -88,11 +89,28 @@ public void initialize(String name, Map options) { options.get(removePrefix.apply(NessieConfigConstants.CONF_NESSIE_REF)); String requestedHash = options.get(removePrefix.apply(NessieConfigConstants.CONF_NESSIE_REF_HASH)); - NessieApiV1 api = + + NessieClientBuilder nessieClientBuilder = createNessieClientBuilder( options.get(NessieConfigConstants.CONF_NESSIE_CLIENT_BUILDER_IMPL)) - .fromConfig(x -> options.get(removePrefix.apply(x))) - .build(NessieApiV1.class); + .fromConfig(x -> options.get(removePrefix.apply(x))); + // default version is set to v1. + final String apiVersion = + options.getOrDefault(removePrefix.apply(NessieUtil.CLIENT_API_VERSION), "1"); + NessieApiV1 api; + switch (apiVersion) { + case "1": + api = nessieClientBuilder.build(NessieApiV1.class); + break; + case "2": + api = nessieClientBuilder.build(NessieApiV2.class); + break; + default: + throw new IllegalArgumentException( + String.format( + "Unsupported %s: %s. Can only be 1 or 2", + removePrefix.apply(NessieUtil.CLIENT_API_VERSION), apiVersion)); + } initialize( name, diff --git a/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java b/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java index 8b17194acb66..930674f35f77 100644 --- a/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java +++ b/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java @@ -85,10 +85,14 @@ public NessieApiV1 getApi() { return api; } - public UpdateableReference getRef() { + UpdateableReference getRef() { return reference.get(); } + public Reference getReference() { + return reference.get().getReference(); + } + public void refresh() throws NessieNotFoundException { getRef().refresh(api); } @@ -326,8 +330,8 @@ public void renameTable(TableIdentifier from, TableIdentifier to) { NessieUtil.buildCommitMetadata( String.format("Iceberg rename table from '%s' to '%s'", from, to), catalogOptions)) - .operation(Operation.Put.of(NessieUtil.toKey(to), existingFromTable, existingFromTable)) - .operation(Operation.Delete.of(NessieUtil.toKey(from))); + .operation(Operation.Delete.of(NessieUtil.toKey(from))) + .operation(Operation.Put.of(NessieUtil.toKey(to), existingFromTable)); try { Tasks.foreach(operations) diff --git a/nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java b/nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java index 43a453c78aa5..863f90b3b5a8 100644 --- a/nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java +++ b/nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java @@ -21,10 +21,8 @@ import java.util.List; import java.util.Map; import org.apache.iceberg.BaseMetastoreTableOperations; -import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableMetadataParser; -import org.apache.iceberg.TableProperties; import org.apache.iceberg.exceptions.AlreadyExistsException; import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.exceptions.CommitStateUnknownException; @@ -32,7 +30,6 @@ import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.projectnessie.client.http.HttpClientException; import org.projectnessie.error.NessieConflictException; import org.projectnessie.error.NessieNotFoundException; @@ -81,47 +78,6 @@ protected String tableName() { return key.toString(); } - private TableMetadata loadTableMetadata(String metadataLocation, Reference reference) { - // Update the TableMetadata with the Content of NessieTableState. - TableMetadata deserialized = TableMetadataParser.read(io(), metadataLocation); - Map newProperties = Maps.newHashMap(deserialized.properties()); - newProperties.put(NESSIE_COMMIT_ID_PROPERTY, reference.getHash()); - // To prevent accidental deletion of files that are still referenced by other branches/tags, - // setting GC_ENABLED to false. So that all Iceberg's gc operations like expire_snapshots, - // remove_orphan_files, drop_table with purge will fail with an error. - // Nessie CLI will provide a reference aware GC functionality for the expired/unreferenced - // files. - newProperties.put(TableProperties.GC_ENABLED, "false"); - - boolean metadataCleanupEnabled = - newProperties - .getOrDefault(TableProperties.METADATA_DELETE_AFTER_COMMIT_ENABLED, "false") - .equalsIgnoreCase("true"); - if (metadataCleanupEnabled) { - newProperties.put(TableProperties.METADATA_DELETE_AFTER_COMMIT_ENABLED, "false"); - LOG.warn( - "Automatic table metadata files cleanup was requested, but disabled because " - + "the Nessie catalog can use historical metadata files from other references. " - + "Use the 'nessie-gc' tool for history-aware GC"); - } - - TableMetadata.Builder builder = - TableMetadata.buildFrom(deserialized) - .setPreviousFileLocation(null) - .setCurrentSchema(table.getSchemaId()) - .setDefaultSortOrder(table.getSortOrderId()) - .setDefaultPartitionSpec(table.getSpecId()) - .withMetadataLocation(metadataLocation) - .setProperties(newProperties); - if (table.getSnapshotId() != -1) { - builder.setBranchSnapshot(table.getSnapshotId(), SnapshotRef.MAIN_BRANCH); - } - LOG.info( - "loadTableMetadata for '{}' from location '{}' at '{}'", key, metadataLocation, reference); - - return builder.discardChanges().build(); - } - @Override protected void doRefresh() { try { @@ -159,7 +115,17 @@ protected void doRefresh() { throw new NoSuchTableException(ex, "No such table '%s'", key); } } - refreshFromMetadataLocation(metadataLocation, null, 2, l -> loadTableMetadata(l, reference)); + refreshFromMetadataLocation( + metadataLocation, + null, + 2, + location -> + NessieUtil.updateTableMetadataWithNessieSpecificProperties( + TableMetadataParser.read(fileIO, location), + location, + table, + key.toString(), + reference)); } @Override diff --git a/nessie/src/main/java/org/apache/iceberg/nessie/NessieUtil.java b/nessie/src/main/java/org/apache/iceberg/nessie/NessieUtil.java index 0e57d8ccc365..4a7a73e40895 100644 --- a/nessie/src/main/java/org/apache/iceberg/nessie/NessieUtil.java +++ b/nessie/src/main/java/org/apache/iceberg/nessie/NessieUtil.java @@ -24,19 +24,31 @@ import java.util.Optional; import javax.annotation.Nullable; import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableProperties; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.projectnessie.model.CommitMeta; import org.projectnessie.model.ContentKey; +import org.projectnessie.model.IcebergTable; import org.projectnessie.model.ImmutableCommitMeta; +import org.projectnessie.model.Reference; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public final class NessieUtil { + private static final Logger LOG = LoggerFactory.getLogger(NessieUtil.class); + public static final String NESSIE_CONFIG_PREFIX = "nessie."; static final String APPLICATION_TYPE = "application-type"; + public static final String CLIENT_API_VERSION = "nessie.client-api-version"; + private NessieUtil() {} static TableIdentifier removeCatalogName(TableIdentifier to, String name) { @@ -89,4 +101,52 @@ private static String commitAuthor(Map catalogOptions) { return Optional.ofNullable(catalogOptions.get(CatalogProperties.USER)) .orElseGet(() -> System.getProperty("user.name")); } + + public static TableMetadata updateTableMetadataWithNessieSpecificProperties( + TableMetadata tableMetadata, + String metadataLocation, + IcebergTable table, + String identifier, + Reference reference) { + // Update the TableMetadata with the Content of NessieTableState. + Map newProperties = Maps.newHashMap(tableMetadata.properties()); + newProperties.put(NessieTableOperations.NESSIE_COMMIT_ID_PROPERTY, reference.getHash()); + // To prevent accidental deletion of files that are still referenced by other branches/tags, + // setting GC_ENABLED to false. So that all Iceberg's gc operations like expire_snapshots, + // remove_orphan_files, drop_table with purge will fail with an error. + // Nessie CLI will provide a reference aware GC functionality for the expired/unreferenced + // files. + newProperties.put(TableProperties.GC_ENABLED, "false"); + + boolean metadataCleanupEnabled = + newProperties + .getOrDefault(TableProperties.METADATA_DELETE_AFTER_COMMIT_ENABLED, "false") + .equalsIgnoreCase("true"); + if (metadataCleanupEnabled) { + newProperties.put(TableProperties.METADATA_DELETE_AFTER_COMMIT_ENABLED, "false"); + LOG.warn( + "Automatic table metadata files cleanup was requested, but disabled because " + + "the Nessie catalog can use historical metadata files from other references. " + + "Use the 'nessie-gc' tool for history-aware GC"); + } + + TableMetadata.Builder builder = + TableMetadata.buildFrom(tableMetadata) + .setPreviousFileLocation(null) + .setCurrentSchema(table.getSchemaId()) + .setDefaultSortOrder(table.getSortOrderId()) + .setDefaultPartitionSpec(table.getSpecId()) + .withMetadataLocation(metadataLocation) + .setProperties(newProperties); + if (table.getSnapshotId() != -1) { + builder.setBranchSnapshot(table.getSnapshotId(), SnapshotRef.MAIN_BRANCH); + } + LOG.info( + "loadTableMetadata for '{}' from location '{}' at '{}'", + identifier, + metadataLocation, + reference); + + return builder.discardChanges().build(); + } } diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java b/nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java index ae4947a98cb7..8566bce67c12 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java @@ -60,27 +60,23 @@ import org.projectnessie.model.Branch; import org.projectnessie.model.Reference; import org.projectnessie.model.Tag; -import org.projectnessie.versioned.persist.adapter.DatabaseAdapter; -import org.projectnessie.versioned.persist.inmem.InmemoryDatabaseAdapterFactory; -import org.projectnessie.versioned.persist.inmem.InmemoryTestConnectionProviderSource; -import org.projectnessie.versioned.persist.tests.extension.DatabaseAdapterExtension; -import org.projectnessie.versioned.persist.tests.extension.NessieDbAdapter; -import org.projectnessie.versioned.persist.tests.extension.NessieDbAdapterName; -import org.projectnessie.versioned.persist.tests.extension.NessieExternalDatabase; +import org.projectnessie.versioned.storage.common.persist.Persist; +import org.projectnessie.versioned.storage.inmemory.InmemoryBackendTestFactory; +import org.projectnessie.versioned.storage.testextension.NessieBackend; +import org.projectnessie.versioned.storage.testextension.NessiePersist; +import org.projectnessie.versioned.storage.testextension.PersistExtension; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -@ExtendWith(DatabaseAdapterExtension.class) -@NessieDbAdapterName(InmemoryDatabaseAdapterFactory.NAME) -@NessieExternalDatabase(InmemoryTestConnectionProviderSource.class) -@NessieApiVersions(versions = NessieApiVersion.V1) +@ExtendWith(PersistExtension.class) +@NessieBackend(InmemoryBackendTestFactory.class) +@NessieApiVersions // test all versions public abstract class BaseTestIceberg { - @NessieDbAdapter static DatabaseAdapter databaseAdapter; + @NessiePersist static Persist persist; @RegisterExtension - static NessieJaxRsExtension server = - NessieJaxRsExtension.jaxRsExtensionForDatabaseAdapter(() -> databaseAdapter); + static NessieJaxRsExtension server = NessieJaxRsExtension.jaxRsExtension(() -> persist); private static final Logger LOG = LoggerFactory.getLogger(BaseTestIceberg.class); @@ -88,6 +84,7 @@ public abstract class BaseTestIceberg { protected NessieCatalog catalog; protected NessieApiV1 api; + protected String apiVersion; protected Configuration hadoopConfig; protected final String branch; private String initialHashOfDefaultBranch; @@ -122,6 +119,7 @@ public void beforeEach(NessieClientFactory clientFactory, @NessieClientUri URI n throws IOException { this.uri = nessieUri.toASCIIString(); this.api = clientFactory.make(); + this.apiVersion = clientFactory.apiVersion() == NessieApiVersion.V2 ? "2" : "1"; Branch defaultBranch = api.getDefaultBranch(); initialHashOfDefaultBranch = defaultBranch.getHash(); @@ -145,7 +143,8 @@ NessieCatalog initCatalog(String ref, String hash) { .put("ref", ref) .put(CatalogProperties.URI, uri) .put("auth-type", "NONE") - .put(CatalogProperties.WAREHOUSE_LOCATION, temp.toUri().toString()); + .put(CatalogProperties.WAREHOUSE_LOCATION, temp.toUri().toString()) + .put("client-api-version", apiVersion); if (null != hash) { options.put("ref.hash", hash); } diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestCustomNessieClient.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestCustomNessieClient.java index 2a3c0c85e000..40341358073f 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestCustomNessieClient.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestCustomNessieClient.java @@ -48,7 +48,9 @@ public void testNoCustomClient() { CatalogProperties.WAREHOUSE_LOCATION, temp.toUri().toString(), CatalogProperties.URI, - uri)); + uri, + "client-api-version", + apiVersion)); } @Test @@ -62,7 +64,9 @@ public void testUnnecessaryDefaultCustomClient() { CatalogProperties.URI, uri, NessieConfigConstants.CONF_NESSIE_CLIENT_BUILDER_IMPL, - HttpClientBuilder.class.getName())); + HttpClientBuilder.class.getName(), + "client-api-version", + apiVersion)); } @Test diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieCatalog.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieCatalog.java index f8ed3987c1ea..760ff0701160 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieCatalog.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieCatalog.java @@ -43,30 +43,27 @@ import org.projectnessie.model.Branch; import org.projectnessie.model.Reference; import org.projectnessie.model.Tag; -import org.projectnessie.versioned.persist.adapter.DatabaseAdapter; -import org.projectnessie.versioned.persist.inmem.InmemoryDatabaseAdapterFactory; -import org.projectnessie.versioned.persist.inmem.InmemoryTestConnectionProviderSource; -import org.projectnessie.versioned.persist.tests.extension.DatabaseAdapterExtension; -import org.projectnessie.versioned.persist.tests.extension.NessieDbAdapter; -import org.projectnessie.versioned.persist.tests.extension.NessieDbAdapterName; -import org.projectnessie.versioned.persist.tests.extension.NessieExternalDatabase; - -@ExtendWith(DatabaseAdapterExtension.class) -@NessieDbAdapterName(InmemoryDatabaseAdapterFactory.NAME) -@NessieExternalDatabase(InmemoryTestConnectionProviderSource.class) -@NessieApiVersions(versions = NessieApiVersion.V1) +import org.projectnessie.versioned.storage.common.persist.Persist; +import org.projectnessie.versioned.storage.inmemory.InmemoryBackendTestFactory; +import org.projectnessie.versioned.storage.testextension.NessieBackend; +import org.projectnessie.versioned.storage.testextension.NessiePersist; +import org.projectnessie.versioned.storage.testextension.PersistExtension; + +@ExtendWith(PersistExtension.class) +@NessieBackend(InmemoryBackendTestFactory.class) +@NessieApiVersions // test all versions public class TestNessieCatalog extends CatalogTests { - @NessieDbAdapter static DatabaseAdapter databaseAdapter; + @NessiePersist static Persist persist; @RegisterExtension - static NessieJaxRsExtension server = - NessieJaxRsExtension.jaxRsExtensionForDatabaseAdapter(() -> databaseAdapter); + static NessieJaxRsExtension server = NessieJaxRsExtension.jaxRsExtension(() -> persist); @TempDir public Path temp; private NessieCatalog catalog; private NessieApiV1 api; + private NessieApiVersion apiVersion; private Configuration hadoopConfig; private String initialHashOfDefaultBranch; private String uri; @@ -75,6 +72,7 @@ public class TestNessieCatalog extends CatalogTests { public void setUp(NessieClientFactory clientFactory, @NessieClientUri URI nessieUri) throws NessieNotFoundException { api = clientFactory.make(); + apiVersion = clientFactory.apiVersion(); initialHashOfDefaultBranch = api.getDefaultBranch().getHash(); uri = nessieUri.toASCIIString(); hadoopConfig = new Configuration(); @@ -115,17 +113,17 @@ private void resetData() throws NessieConflictException, NessieNotFoundException private NessieCatalog initNessieCatalog(String ref) { NessieCatalog newCatalog = new NessieCatalog(); newCatalog.setConf(hadoopConfig); - newCatalog.initialize( - "nessie", + ImmutableMap options = ImmutableMap.of( "ref", ref, CatalogProperties.URI, uri, - "auth-type", - "NONE", CatalogProperties.WAREHOUSE_LOCATION, - temp.toUri().toString())); + temp.toUri().toString(), + "client-api-version", + apiVersion == NessieApiVersion.V2 ? "2" : "1"); + newCatalog.initialize("nessie", options); return newCatalog; } diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieIcebergClient.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieIcebergClient.java index d6643be91e24..12fa807dcfb6 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieIcebergClient.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieIcebergClient.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.nessie; +import java.io.IOException; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.assertj.core.api.Assertions; @@ -89,4 +90,16 @@ public void testWithReferenceAfterRecreatingBranch() .isEqualTo(ref); Assertions.assertThat(client.withReference(branch, null)).isNotEqualTo(client); } + + @Test + public void testInvalidClientApiVersion() throws IOException { + try (NessieCatalog newCatalog = new NessieCatalog()) { + newCatalog.setConf(hadoopConfig); + ImmutableMap.Builder options = + ImmutableMap.builder().put("client-api-version", "3"); + Assertions.assertThatThrownBy(() -> newCatalog.initialize("nessie", options.buildOrThrow())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Unsupported client-api-version: 3. Can only be 1 or 2"); + } + } } diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java index be3f31ac3d7f..f9ec85053e27 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java @@ -25,16 +25,18 @@ import java.io.File; import java.io.IOException; import java.net.URI; +import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; import java.util.Arrays; +import java.util.Comparator; import java.util.List; import java.util.Objects; import java.util.Set; import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecordBuilder; -import org.apache.commons.io.FileUtils; import org.apache.iceberg.BaseTable; import org.apache.iceberg.DataFile; import org.apache.iceberg.HasTableOperations; @@ -107,7 +109,9 @@ public void beforeEach(NessieClientFactory clientFactory, @NessieClientUri URI n public void afterEach() throws Exception { // drop the table data if (tableLocation != null) { - FileUtils.deleteDirectory(new File(tableLocation)); + try (Stream walk = Files.walk(Paths.get(tableLocation))) { + walk.sorted(Comparator.reverseOrder()).map(Path::toFile).forEach(File::delete); + } catalog.dropTable(TABLE_IDENTIFIER, false); } @@ -542,7 +546,7 @@ public void testFailure() throws NessieNotFoundException, NessieConflictExceptio IcebergTable table = getTable(BRANCH, KEY); - IcebergTable value = IcebergTable.of("dummytable.metadata.json", 42, 42, 42, 42, "cid"); + IcebergTable value = IcebergTable.of("dummytable.metadata.json", 42, 42, 42, 42, table.getId()); api.commitMultipleOperations() .branch(branch) .operation(Operation.Put.of(KEY, value)) diff --git a/open-api/Makefile b/open-api/Makefile index d69324a714b7..d60496cc4b94 100644 --- a/open-api/Makefile +++ b/open-api/Makefile @@ -23,6 +23,7 @@ lint: generate: datamodel-codegen \ + --enum-field-as-literal all \ --target-python-version 3.8 \ --use-schema-description \ --field-constraints \ diff --git a/open-api/rest-catalog-open-api.py b/open-api/rest-catalog-open-api.py index 65cccc139533..5fc9fe6e688b 100644 --- a/open-api/rest-catalog-open-api.py +++ b/open-api/rest-catalog-open-api.py @@ -17,8 +17,7 @@ from __future__ import annotations -from enum import Enum -from typing import Any, Dict, List, Optional, Union +from typing import Any, Dict, List, Literal, Optional, Union from pydantic import BaseModel, Extra, Field @@ -85,18 +84,6 @@ class PrimitiveType(BaseModel): __root__: str = Field(..., example=['long', 'string', 'fixed[16]', 'decimal(10,2)']) -class Type1(Enum): - struct = 'struct' - - -class Type2(Enum): - list = 'list' - - -class Type3(Enum): - map = 'map' - - class ExpressionType(BaseModel): __root__: str = Field( ..., @@ -126,10 +113,6 @@ class Reference(BaseModel): __root__: str = Field(..., example=['column-name']) -class Type4(Enum): - transform = 'transform' - - class Transform(BaseModel): __root__: str = Field( ..., @@ -157,14 +140,12 @@ class PartitionSpec(BaseModel): fields: List[PartitionField] -class SortDirection(Enum): - asc = 'asc' - desc = 'desc' +class SortDirection(BaseModel): + __root__: Literal['asc', 'desc'] -class NullOrder(Enum): - nulls_first = 'nulls-first' - nulls_last = 'nulls-last' +class NullOrder(BaseModel): + __root__: Literal['nulls-first', 'nulls-last'] class SortField(BaseModel): @@ -179,15 +160,8 @@ class SortOrder(BaseModel): fields: List[SortField] -class Operation(Enum): - append = 'append' - replace = 'replace' - overwrite = 'overwrite' - delete = 'delete' - - class Summary(BaseModel): - operation: Operation + operation: Literal['append', 'replace', 'overwrite', 'delete'] additionalProperties: Optional[str] = None @@ -205,13 +179,8 @@ class Snapshot(BaseModel): schema_id: Optional[int] = Field(None, alias='schema-id') -class Type5(Enum): - tag = 'tag' - branch = 'branch' - - class SnapshotReference(BaseModel): - type: Type5 + type: Literal['tag', 'branch'] snapshot_id: int = Field(..., alias='snapshot-id') max_ref_age_ms: Optional[int] = Field(None, alias='max-ref-age-ms') max_snapshot_age_ms: Optional[int] = Field(None, alias='max-snapshot-age-ms') @@ -240,25 +209,23 @@ class MetadataLog(BaseModel): __root__: List[MetadataLogItem] -class Action(Enum): - upgrade_format_version = 'upgrade-format-version' - add_schema = 'add-schema' - set_current_schema = 'set-current-schema' - add_spec = 'add-spec' - set_default_spec = 'set-default-spec' - add_sort_order = 'add-sort-order' - set_default_sort_order = 'set-default-sort-order' - add_snapshot = 'add-snapshot' - set_snapshot_ref = 'set-snapshot-ref' - remove_snapshots = 'remove-snapshots' - remove_snapshot_ref = 'remove-snapshot-ref' - set_location = 'set-location' - set_properties = 'set-properties' - remove_properties = 'remove-properties' - - class BaseUpdate(BaseModel): - action: Action + action: Literal[ + 'upgrade-format-version', + 'add-schema', + 'set-current-schema', + 'add-spec', + 'set-default-spec', + 'add-sort-order', + 'set-default-sort-order', + 'add-snapshot', + 'set-snapshot-ref', + 'remove-snapshots', + 'remove-snapshot-ref', + 'set-location', + 'set-properties', + 'remove-properties', + ] class UpgradeFormatVersionUpdate(BaseUpdate): @@ -325,17 +292,6 @@ class RemovePropertiesUpdate(BaseUpdate): removals: List[str] -class Type6(Enum): - assert_create = 'assert-create' - assert_table_uuid = 'assert-table-uuid' - assert_ref_snapshot_id = 'assert-ref-snapshot-id' - assert_last_assigned_field_id = 'assert-last-assigned-field-id' - assert_current_schema_id = 'assert-current-schema-id' - assert_last_assigned_partition_id = 'assert-last-assigned-partition-id' - assert_default_spec_id = 'assert-default-spec-id' - assert_default_sort_order_id = 'assert-default-sort-order-id' - - class TableRequirement(BaseModel): """ Assertions from the client that must be valid for the commit to succeed. Assertions are identified by `type` - @@ -349,7 +305,16 @@ class TableRequirement(BaseModel): - `assert-default-sort-order-id` - the table's default sort order id must match the requirement's `default-sort-order-id` """ - type: Type6 + type: Literal[ + 'assert-create', + 'assert-table-uuid', + 'assert-ref-snapshot-id', + 'assert-last-assigned-field-id', + 'assert-current-schema-id', + 'assert-last-assigned-partition-id', + 'assert-default-spec-id', + 'assert-default-sort-order-id', + ] ref: Optional[str] = None uuid: Optional[str] = None snapshot_id: Optional[int] = Field(None, alias='snapshot-id') @@ -362,29 +327,23 @@ class TableRequirement(BaseModel): default_sort_order_id: Optional[int] = Field(None, alias='default-sort-order-id') -class TokenType(Enum): - """ - Token type identifier, from RFC 8693 Section 3 +class RegisterTableRequest(BaseModel): + name: str + metadata_location: str = Field(..., alias='metadata-location') - See https://datatracker.ietf.org/doc/html/rfc8693#section-3 - """ - urn_ietf_params_oauth_token_type_access_token = ( - 'urn:ietf:params:oauth:token-type:access_token' - ) - urn_ietf_params_oauth_token_type_refresh_token = ( - 'urn:ietf:params:oauth:token-type:refresh_token' - ) - urn_ietf_params_oauth_token_type_id_token = ( - 'urn:ietf:params:oauth:token-type:id_token' +class TokenType(BaseModel): + __root__: Literal[ + 'urn:ietf:params:oauth:token-type:access_token', + 'urn:ietf:params:oauth:token-type:refresh_token', + 'urn:ietf:params:oauth:token-type:id_token', + 'urn:ietf:params:oauth:token-type:saml1', + 'urn:ietf:params:oauth:token-type:saml2', + 'urn:ietf:params:oauth:token-type:jwt', + ] = Field( + ..., + description='Token type identifier, from RFC 8693 Section 3\n\nSee https://datatracker.ietf.org/doc/html/rfc8693#section-3', ) - urn_ietf_params_oauth_token_type_saml1 = 'urn:ietf:params:oauth:token-type:saml1' - urn_ietf_params_oauth_token_type_saml2 = 'urn:ietf:params:oauth:token-type:saml2' - urn_ietf_params_oauth_token_type_jwt = 'urn:ietf:params:oauth:token-type:jwt' - - -class GrantType(Enum): - client_credentials = 'client_credentials' class OAuthClientCredentialsRequest(BaseModel): @@ -394,7 +353,7 @@ class OAuthClientCredentialsRequest(BaseModel): See https://datatracker.ietf.org/doc/html/rfc6749#section-4.4 """ - grant_type: GrantType + grant_type: Literal['client_credentials'] scope: Optional[str] = None client_id: str = Field( ..., @@ -406,12 +365,6 @@ class OAuthClientCredentialsRequest(BaseModel): ) -class GrantType1(Enum): - urn_ietf_params_oauth_grant_type_token_exchange = ( - 'urn:ietf:params:oauth:grant-type:token-exchange' - ) - - class OAuthTokenExchangeRequest(BaseModel): """ OAuth2 token exchange request @@ -419,7 +372,7 @@ class OAuthTokenExchangeRequest(BaseModel): See https://datatracker.ietf.org/doc/html/rfc8693 """ - grant_type: GrantType1 + grant_type: Literal['urn:ietf:params:oauth:grant-type:token-exchange'] scope: Optional[str] = None requested_token_type: Optional[TokenType] = None subject_token: str = Field( @@ -464,38 +417,24 @@ class CommitReport(BaseModel): metadata: Optional[Dict[str, str]] = None -class Error(Enum): - invalid_request = 'invalid_request' - invalid_client = 'invalid_client' - invalid_grant = 'invalid_grant' - unauthorized_client = 'unauthorized_client' - unsupported_grant_type = 'unsupported_grant_type' - invalid_scope = 'invalid_scope' - - class OAuthError(BaseModel): - error: Error + error: Literal[ + 'invalid_request', + 'invalid_client', + 'invalid_grant', + 'unauthorized_client', + 'unsupported_grant_type', + 'invalid_scope', + ] error_description: Optional[str] = None error_uri: Optional[str] = None -class TokenType1(Enum): - """ - Access token type for client credentials or token exchange - - See https://datatracker.ietf.org/doc/html/rfc6749#section-7.1 - """ - - bearer = 'bearer' - mac = 'mac' - N_A = 'N_A' - - class OAuthTokenResponse(BaseModel): access_token: str = Field( ..., description='The access token, for client credentials or token exchange' ) - token_type: TokenType1 = Field( + token_type: Literal['bearer', 'mac', 'N_A'] = Field( ..., description='Access token type for client credentials or token exchange\n\nSee https://datatracker.ietf.org/doc/html/rfc6749#section-7.1', ) @@ -573,7 +512,7 @@ class RenameTableRequest(BaseModel): class TransformTerm(BaseModel): - type: Type4 + type: Literal['transform'] transform: Transform term: Reference @@ -613,19 +552,19 @@ class StructField(BaseModel): class StructType(BaseModel): - type: Type1 + type: Literal['struct'] fields: List[StructField] class ListType(BaseModel): - type: Type2 + type: Literal['list'] element_id: int = Field(..., alias='element-id') element: Type element_required: bool = Field(..., alias='element-required') class MapType(BaseModel): - type: Type3 + type: Literal['map'] key_id: int = Field(..., alias='key-id') key: Type value_id: int = Field(..., alias='value-id') @@ -677,6 +616,7 @@ class TableMetadata(BaseModel): snapshots: Optional[List[Snapshot]] = None refs: Optional[SnapshotReferences] = None current_snapshot_id: Optional[int] = Field(None, alias='current-snapshot-id') + last_sequence_number: Optional[int] = Field(None, alias='last-sequence-number') snapshot_log: Optional[SnapshotLog] = Field(None, alias='snapshot-log') metadata_log: Optional[MetadataLog] = Field(None, alias='metadata-log') @@ -748,10 +688,18 @@ class LoadTableResult(BaseModel): class CommitTableRequest(BaseModel): + identifier: Optional[TableIdentifier] = Field( + None, + description='Table identifier to update; must be present for CommitTransactionRequest', + ) requirements: List[TableRequirement] updates: List[TableUpdate] +class CommitTransactionRequest(BaseModel): + table_changes: List[CommitTableRequest] = Field(..., alias='table-changes') + + class CreateTableRequest(BaseModel): name: str location: Optional[str] = None diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index 8025ca966fbb..4075ec9af570 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -498,6 +498,58 @@ paths: 5XX: $ref: '#/components/responses/ServerErrorResponse' + /v1/{prefix}/namespaces/{namespace}/register: + parameters: + - $ref: '#/components/parameters/prefix' + - $ref: '#/components/parameters/namespace' + + post: + tags: + - Catalog API + summary: Register a table in the given namespace using given metadata file location + description: + Register a table using given metadata file location. + + operationId: registerTable + requestBody: + content: + application/json: + schema: + $ref: '#/components/schemas/RegisterTableRequest' + responses: + 200: + $ref: '#/components/responses/LoadTableResponse' + 400: + $ref: '#/components/responses/BadRequestErrorResponse' + 401: + $ref: '#/components/responses/UnauthorizedResponse' + 403: + $ref: '#/components/responses/ForbiddenResponse' + 404: + description: Not Found - The namespace specified does not exist + content: + application/json: + schema: + $ref: '#/components/schemas/ErrorModel' + examples: + NamespaceNotFound: + $ref: '#/components/examples/NoSuchNamespaceError' + 409: + description: Conflict - The table already exists + content: + application/json: + schema: + $ref: '#/components/schemas/ErrorModel' + examples: + NamespaceAlreadyExists: + $ref: '#/components/examples/TableAlreadyExistsError' + 419: + $ref: '#/components/responses/AuthenticationTimeoutResponse' + 503: + $ref: '#/components/responses/ServiceUnavailableResponse' + 5XX: + $ref: '#/components/responses/ServerErrorResponse' + /v1/{prefix}/namespaces/{namespace}/tables/{table}: parameters: - $ref: '#/components/parameters/prefix' @@ -727,8 +779,8 @@ paths: description: Check if a table exists within a given namespace. This request does not return a response body. responses: - 200: - description: OK - Table Exists + 204: + description: Success, no content 400: description: Bad Request 401: @@ -765,8 +817,8 @@ paths: $ref: '#/components/examples/RenameTableSameNamespace' required: true responses: - 200: - description: OK + 204: + description: Success, no content 400: $ref: '#/components/responses/BadRequestErrorResponse' 401: @@ -848,6 +900,121 @@ paths: 5XX: $ref: '#/components/responses/ServerErrorResponse' + /v1/{prefix}/transactions/commit: + parameters: + - $ref: '#/components/parameters/prefix' + + post: + tags: + - Catalog API + summary: Commit updates to multiple tables in an atomic operation + operationId: commitTransaction + requestBody: + description: + Commit updates to multiple tables in an atomic operation + + + A commit for a single table consists of a table identifier with requirements and updates. + Requirements are assertions that will be validated before attempting to make and commit changes. + For example, `assert-ref-snapshot-id` will check that a named ref's snapshot ID has a certain value. + + + Updates are changes to make to table metadata. For example, after asserting that the current main ref + is at the expected snapshot, a commit may add a new child snapshot and set the ref to the new + snapshot id. + content: + application/json: + schema: + $ref: '#/components/schemas/CommitTransactionRequest' + required: true + responses: + 204: + description: Success, no content + 400: + $ref: '#/components/responses/BadRequestErrorResponse' + 401: + $ref: '#/components/responses/UnauthorizedResponse' + 403: + $ref: '#/components/responses/ForbiddenResponse' + 404: + description: + Not Found - NoSuchTableException, table to load does not exist + content: + application/json: + schema: + $ref: '#/components/schemas/ErrorModel' + examples: + TableToUpdateDoesNotExist: + $ref: '#/components/examples/NoSuchTableError' + 409: + description: + Conflict - CommitFailedException, one or more requirements failed. The client may retry. + content: + application/json: + schema: + $ref: '#/components/schemas/ErrorModel' + 419: + $ref: '#/components/responses/AuthenticationTimeoutResponse' + 500: + description: + An unknown server-side problem occurred; the commit state is unknown. + content: + application/json: + schema: + $ref: '#/components/schemas/ErrorModel' + example: { + "error": { + "message": "Internal Server Error", + "type": "CommitStateUnknownException", + "code": 500 + } + } + 503: + $ref: '#/components/responses/ServiceUnavailableResponse' + 502: + description: + A gateway or proxy received an invalid response from the upstream server; the commit state is unknown. + content: + application/json: + schema: + $ref: '#/components/schemas/ErrorModel' + example: { + "error": { + "message": "Invalid response from the upstream server", + "type": "CommitStateUnknownException", + "code": 502 + } + } + 504: + description: + A server-side gateway timeout occurred; the commit state is unknown. + content: + application/json: + schema: + $ref: '#/components/schemas/ErrorModel' + example: { + "error": { + "message": "Gateway timed out during commit", + "type": "CommitStateUnknownException", + "code": 504 + } + } + 5XX: + description: + A server-side problem that might not be addressable on the client. + content: + application/json: + schema: + $ref: '#/components/schemas/ErrorModel' + example: { + "error": { + "message": "Bad Gateway", + "type": "InternalServerError", + "code": 502 + } + } + + components: ####################################################### # Common Parameter Definitions Used In Several Routes # @@ -1454,6 +1621,9 @@ components: current-snapshot-id: type: integer format: int64 + last-sequence-number: + type: integer + format: int64 # logs snapshot-log: $ref: '#/components/schemas/SnapshotLog' @@ -1840,6 +2010,9 @@ components: - requirements - updates properties: + identifier: + description: Table identifier to update; must be present for CommitTransactionRequest + $ref: '#/components/schemas/TableIdentifier' requirements: type: array items: @@ -1849,6 +2022,17 @@ components: items: $ref: '#/components/schemas/TableUpdate' + CommitTransactionRequest: + type: object + required: + - table-changes + properties: + table-changes: + type: array + items: + description: Table commit request; must provide an `identifier` + $ref: '#/components/schemas/CommitTableRequest' + CreateTableRequest: type: object required: @@ -1872,6 +2056,17 @@ components: additionalProperties: type: string + RegisterTableRequest: + type: object + required: + - name + - metadata-location + properties: + name: + type: string + metadata-location: + type: string + TokenType: type: string enum: diff --git a/orc/src/test/java/org/apache/iceberg/orc/TestBloomFilter.java b/orc/src/test/java/org/apache/iceberg/orc/TestBloomFilter.java index 7d28ef77c290..ca932f205bff 100644 --- a/orc/src/test/java/org/apache/iceberg/orc/TestBloomFilter.java +++ b/orc/src/test/java/org/apache/iceberg/orc/TestBloomFilter.java @@ -42,10 +42,8 @@ import org.apache.orc.impl.RecordReaderImpl; import org.apache.orc.impl.WriterImpl; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestBloomFilter { private static final Schema DATA_SCHEMA = @@ -54,12 +52,11 @@ public class TestBloomFilter { required(101, "name", Types.StringType.get()), required(102, "price", Types.DoubleType.get())); - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private File testFile; @Test public void testWriteOption() throws Exception { - File testFile = temp.newFile(); - Assert.assertTrue("Delete should succeed", testFile.delete()); + Assertions.assertThat(testFile.delete()).as("Delete should succeed").isTrue(); OutputFile outFile = Files.localOutput(testFile); try (FileAppender writer = @@ -84,9 +81,9 @@ public void testWriteOption() throws Exception { double bloomFilterFpp = (double) bloomFilterFppField.get(orcWriter); // Validate whether the bloom filters are set in ORC SDK or not - Assert.assertTrue(bloomFilterColumns[1]); - Assert.assertTrue(bloomFilterColumns[2]); - Assert.assertEquals(0.04, bloomFilterFpp, 1e-15); + Assertions.assertThat(bloomFilterColumns[1]).isTrue(); + Assertions.assertThat(bloomFilterColumns[2]).isTrue(); + Assertions.assertThat(bloomFilterFpp).isCloseTo(0.04, Assertions.offset(1e-15)); Record recordTemplate = GenericRecord.create(DATA_SCHEMA); Record record1 = recordTemplate.copy("id", 1L, "name", "foo", "price", 1.0); @@ -132,8 +129,7 @@ public void testWriteOption() throws Exception { @Test public void testInvalidFppOption() throws Exception { - File testFile = temp.newFile(); - Assert.assertTrue("Delete should succeed", testFile.delete()); + Assertions.assertThat(testFile.delete()).as("Delete should succeed").isTrue(); Assertions.assertThatThrownBy( () -> diff --git a/orc/src/test/java/org/apache/iceberg/orc/TestBuildOrcProjection.java b/orc/src/test/java/org/apache/iceberg/orc/TestBuildOrcProjection.java index ec42b26f1bc9..b8f17f3364bc 100644 --- a/orc/src/test/java/org/apache/iceberg/orc/TestBuildOrcProjection.java +++ b/orc/src/test/java/org/apache/iceberg/orc/TestBuildOrcProjection.java @@ -20,13 +20,12 @@ import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; -import static org.junit.Assert.assertEquals; import org.apache.iceberg.Schema; import org.apache.iceberg.types.Types; import org.apache.orc.TypeDescription; import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; /** Test projections on ORC types. */ public class TestBuildOrcProjection { @@ -39,11 +38,13 @@ public void testProjectionPrimitiveNoOp() { // Original mapping (stored in ORC) TypeDescription orcSchema = ORCSchemaUtil.convert(originalSchema); - assertEquals(2, orcSchema.getChildren().size()); - assertEquals(1, orcSchema.findSubtype("a").getId()); - assertEquals(TypeDescription.Category.INT, orcSchema.findSubtype("a").getCategory()); - assertEquals(2, orcSchema.findSubtype("b").getId()); - assertEquals(TypeDescription.Category.STRING, orcSchema.findSubtype("b").getCategory()); + Assertions.assertThat(orcSchema.getChildren()).hasSize(2); + Assertions.assertThat(orcSchema.findSubtype("a").getId()).isEqualTo(1); + Assertions.assertThat(orcSchema.findSubtype("a").getCategory()) + .isEqualTo(TypeDescription.Category.INT); + Assertions.assertThat(orcSchema.findSubtype("b").getId()).isEqualTo(2); + Assertions.assertThat(orcSchema.findSubtype("b").getCategory()) + .isEqualTo(TypeDescription.Category.STRING); } @Test @@ -63,11 +64,13 @@ public void testProjectionPrimitive() { ); TypeDescription newOrcSchema = ORCSchemaUtil.buildOrcProjection(evolveSchema, orcSchema); - assertEquals(2, newOrcSchema.getChildren().size()); - assertEquals(1, newOrcSchema.findSubtype("b").getId()); - assertEquals(TypeDescription.Category.STRING, newOrcSchema.findSubtype("b").getCategory()); - assertEquals(2, newOrcSchema.findSubtype("c_r3").getId()); - assertEquals(TypeDescription.Category.DATE, newOrcSchema.findSubtype("c_r3").getCategory()); + Assertions.assertThat(newOrcSchema.getChildren()).hasSize(2); + Assertions.assertThat(newOrcSchema.findSubtype("b").getId()).isEqualTo(1); + Assertions.assertThat(newOrcSchema.findSubtype("b").getCategory()) + .isEqualTo(TypeDescription.Category.STRING); + Assertions.assertThat(newOrcSchema.findSubtype("c_r3").getId()).isEqualTo(2); + Assertions.assertThat(newOrcSchema.findSubtype("c_r3").getCategory()) + .isEqualTo(TypeDescription.Category.DATE); } @Test @@ -81,13 +84,16 @@ public void testProjectionNestedNoOp() { TypeDescription orcSchema = ORCSchemaUtil.convert(originalSchema); TypeDescription newOrcSchema = ORCSchemaUtil.buildOrcProjection(originalSchema, orcSchema); - assertEquals(1, newOrcSchema.getChildren().size()); - assertEquals(TypeDescription.Category.STRUCT, newOrcSchema.findSubtype("a").getCategory()); + Assertions.assertThat(newOrcSchema.getChildren()).hasSize(1); + Assertions.assertThat(newOrcSchema.findSubtype("a").getCategory()) + .isEqualTo(TypeDescription.Category.STRUCT); TypeDescription nestedCol = newOrcSchema.findSubtype("a"); - assertEquals(2, nestedCol.findSubtype("b").getId()); - assertEquals(TypeDescription.Category.STRING, nestedCol.findSubtype("b").getCategory()); - assertEquals(3, nestedCol.findSubtype("c").getId()); - assertEquals(TypeDescription.Category.DATE, nestedCol.findSubtype("c").getCategory()); + Assertions.assertThat(nestedCol.findSubtype("b").getId()).isEqualTo(2); + Assertions.assertThat(nestedCol.findSubtype("b").getCategory()) + .isEqualTo(TypeDescription.Category.STRING); + Assertions.assertThat(nestedCol.findSubtype("c").getId()).isEqualTo(3); + Assertions.assertThat(nestedCol.findSubtype("c").getCategory()) + .isEqualTo(TypeDescription.Category.DATE); } @Test @@ -107,13 +113,16 @@ public void testProjectionNested() { Schema evolveSchema = new Schema(optional(1, "aa", newNestedStructType)); TypeDescription newOrcSchema = ORCSchemaUtil.buildOrcProjection(evolveSchema, orcSchema); - assertEquals(1, newOrcSchema.getChildren().size()); - assertEquals(TypeDescription.Category.STRUCT, newOrcSchema.findSubtype("a").getCategory()); + Assertions.assertThat(newOrcSchema.getChildren()).hasSize(1); + Assertions.assertThat(newOrcSchema.findSubtype("a").getCategory()) + .isEqualTo(TypeDescription.Category.STRUCT); TypeDescription nestedCol = newOrcSchema.findSubtype("a"); - assertEquals(2, nestedCol.findSubtype("c").getId()); - assertEquals(TypeDescription.Category.DATE, nestedCol.findSubtype("c").getCategory()); - assertEquals(3, nestedCol.findSubtype("b").getId()); - assertEquals(TypeDescription.Category.STRING, nestedCol.findSubtype("b").getCategory()); + Assertions.assertThat(nestedCol.findSubtype("c").getId()).isEqualTo(2); + Assertions.assertThat(nestedCol.findSubtype("c").getCategory()) + .isEqualTo(TypeDescription.Category.DATE); + Assertions.assertThat(nestedCol.findSubtype("b").getId()).isEqualTo(3); + Assertions.assertThat(nestedCol.findSubtype("b").getCategory()) + .isEqualTo(TypeDescription.Category.STRING); } @Test @@ -127,13 +136,16 @@ public void testEvolutionAddContainerField() { optional(2, "b", Types.StructType.of(required(3, "c", Types.LongType.get())))); TypeDescription newOrcSchema = ORCSchemaUtil.buildOrcProjection(evolvedSchema, baseOrcSchema); - assertEquals(2, newOrcSchema.getChildren().size()); - assertEquals(TypeDescription.Category.INT, newOrcSchema.findSubtype("a").getCategory()); - assertEquals(2, newOrcSchema.findSubtype("b_r2").getId()); - assertEquals(TypeDescription.Category.STRUCT, newOrcSchema.findSubtype("b_r2").getCategory()); + Assertions.assertThat(newOrcSchema.getChildren()).hasSize(2); + Assertions.assertThat(newOrcSchema.findSubtype("a").getCategory()) + .isEqualTo(TypeDescription.Category.INT); + Assertions.assertThat(newOrcSchema.findSubtype("b_r2").getId()).isEqualTo(2); + Assertions.assertThat(newOrcSchema.findSubtype("b_r2").getCategory()) + .isEqualTo(TypeDescription.Category.STRUCT); TypeDescription nestedCol = newOrcSchema.findSubtype("b_r2"); - assertEquals(3, nestedCol.findSubtype("c_r3").getId()); - assertEquals(TypeDescription.Category.LONG, nestedCol.findSubtype("c_r3").getCategory()); + Assertions.assertThat(nestedCol.findSubtype("c_r3").getId()).isEqualTo(3); + Assertions.assertThat(nestedCol.findSubtype("c_r3").getCategory()) + .isEqualTo(TypeDescription.Category.LONG); } @Test diff --git a/orc/src/test/java/org/apache/iceberg/orc/TestEstimateOrcAvgWidthVisitor.java b/orc/src/test/java/org/apache/iceberg/orc/TestEstimateOrcAvgWidthVisitor.java index 46736bcf11c9..1aa7dda04761 100644 --- a/orc/src/test/java/org/apache/iceberg/orc/TestEstimateOrcAvgWidthVisitor.java +++ b/orc/src/test/java/org/apache/iceberg/orc/TestEstimateOrcAvgWidthVisitor.java @@ -24,8 +24,8 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.types.Types; import org.apache.orc.TypeDescription; -import org.junit.Assert; -import org.junit.Test; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; public class TestEstimateOrcAvgWidthVisitor { @@ -78,7 +78,9 @@ public void testEstimateIntegerWidth() { Schema integerSchema = new Schema(ID_FIELD); TypeDescription integerOrcSchema = ORCSchemaUtil.convert(integerSchema); long estimateLength = getEstimateLength(integerOrcSchema); - Assert.assertEquals("Estimated average length of integer must be 8.", 8, estimateLength); + Assertions.assertThat(estimateLength) + .as("Estimated average length of integer must be 8.") + .isEqualTo(8); } @Test @@ -86,7 +88,9 @@ public void testEstimateStringWidth() { Schema stringSchema = new Schema(DATA_FIELD); TypeDescription stringOrcSchema = ORCSchemaUtil.convert(stringSchema); long estimateLength = getEstimateLength(stringOrcSchema); - Assert.assertEquals("Estimated average length of string must be 128.", 128, estimateLength); + Assertions.assertThat(estimateLength) + .as("Estimated average length of string must be 128.") + .isEqualTo(128); } @Test @@ -94,7 +98,9 @@ public void testEstimateFloatWidth() { Schema floatSchema = new Schema(FLOAT_FIELD); TypeDescription floatOrcSchema = ORCSchemaUtil.convert(floatSchema); long estimateLength = getEstimateLength(floatOrcSchema); - Assert.assertEquals("Estimated average length of float must be 8.", 8, estimateLength); + Assertions.assertThat(estimateLength) + .as("Estimated average length of float must be 8.") + .isEqualTo(8); } @Test @@ -102,7 +108,9 @@ public void testEstimateDoubleWidth() { Schema doubleSchema = new Schema(DOUBLE_FIELD); TypeDescription doubleOrcSchema = ORCSchemaUtil.convert(doubleSchema); long estimateLength = getEstimateLength(doubleOrcSchema); - Assert.assertEquals("Estimated average length of double must be 8.", 8, estimateLength); + Assertions.assertThat(estimateLength) + .as("Estimated average length of double must be 8.") + .isEqualTo(8); } @Test @@ -110,7 +118,9 @@ public void testEstimateDecimalWidth() { Schema decimalSchema = new Schema(DECIMAL_FIELD); TypeDescription decimalOrcSchema = ORCSchemaUtil.convert(decimalSchema); long estimateLength = getEstimateLength(decimalOrcSchema); - Assert.assertEquals("Estimated average length of decimal must be 7.", 7, estimateLength); + Assertions.assertThat(estimateLength) + .as("Estimated average length of decimal must be 7.") + .isEqualTo(7); } @Test @@ -118,7 +128,9 @@ public void testEstimateFixedWidth() { Schema fixedSchema = new Schema(FIXED_FIELD); TypeDescription fixedOrcSchema = ORCSchemaUtil.convert(fixedSchema); long estimateLength = getEstimateLength(fixedOrcSchema); - Assert.assertEquals("Estimated average length of fixed must be 128.", 128, estimateLength); + Assertions.assertThat(estimateLength) + .as("Estimated average length of fixed must be 128.") + .isEqualTo(128); } @Test @@ -126,7 +138,9 @@ public void testEstimateBinaryWidth() { Schema binarySchema = new Schema(BINARY_FIELD); TypeDescription binaryOrcSchema = ORCSchemaUtil.convert(binarySchema); long estimateLength = getEstimateLength(binaryOrcSchema); - Assert.assertEquals("Estimated average length of binary must be 128.", 128, estimateLength); + Assertions.assertThat(estimateLength) + .as("Estimated average length of binary must be 128.") + .isEqualTo(128); } @Test @@ -134,7 +148,9 @@ public void testEstimateListWidth() { Schema listSchema = new Schema(FLOAT_LIST_FIELD); TypeDescription listOrcSchema = ORCSchemaUtil.convert(listSchema); long estimateLength = getEstimateLength(listOrcSchema); - Assert.assertEquals("Estimated average length of list must be 8.", 8, estimateLength); + Assertions.assertThat(estimateLength) + .as("Estimated average length of list must be 8.") + .isEqualTo(8); } @Test @@ -142,7 +158,9 @@ public void testEstimateLongWidth() { Schema longSchema = new Schema(LONG_FIELD); TypeDescription longOrcSchema = ORCSchemaUtil.convert(longSchema); long estimateLength = getEstimateLength(longOrcSchema); - Assert.assertEquals("Estimated average length of long must be 8.", 8, estimateLength); + Assertions.assertThat(estimateLength) + .as("Estimated average length of long must be 8.") + .isEqualTo(8); } @Test @@ -150,7 +168,9 @@ public void testEstimateBooleanWidth() { Schema booleanSchema = new Schema(BOOLEAN_FIELD); TypeDescription booleanOrcSchema = ORCSchemaUtil.convert(booleanSchema); long estimateLength = getEstimateLength(booleanOrcSchema); - Assert.assertEquals("Estimated average length of boolean must be 8.", 8, estimateLength); + Assertions.assertThat(estimateLength) + .as("Estimated average length of boolean must be 8.") + .isEqualTo(8); } @Test @@ -158,13 +178,16 @@ public void testEstimateTimestampWidth() { Schema timestampZoneSchema = new Schema(TIMESTAMP_ZONE_FIELD); TypeDescription timestampZoneOrcSchema = ORCSchemaUtil.convert(timestampZoneSchema); long estimateLength = getEstimateLength(timestampZoneOrcSchema); - Assert.assertEquals( - "Estimated average length of timestamps with zone must be 12.", 12, estimateLength); + Assertions.assertThat(estimateLength) + .as("Estimated average length of timestamps with zone must be 12.") + .isEqualTo(12); Schema timestampSchema = new Schema(TIMESTAMP_FIELD); TypeDescription timestampOrcSchema = ORCSchemaUtil.convert(timestampSchema); estimateLength = getEstimateLength(timestampOrcSchema); - Assert.assertEquals("Estimated average length of timestamp must be 12.", 12, estimateLength); + Assertions.assertThat(estimateLength) + .as("Estimated average length of timestamp must be 12.") + .isEqualTo(12); } @Test @@ -172,7 +195,9 @@ public void testEstimateDateWidth() { Schema dateSchema = new Schema(DATE_FIELD); TypeDescription dateOrcSchema = ORCSchemaUtil.convert(dateSchema); long estimateLength = getEstimateLength(dateOrcSchema); - Assert.assertEquals("Estimated average length of date must be 8.", 8, estimateLength); + Assertions.assertThat(estimateLength) + .as("Estimated average length of date must be 8.") + .isEqualTo(8); } @Test @@ -180,7 +205,9 @@ public void testEstimateUUIDWidth() { Schema uuidSchema = new Schema(UUID_FIELD); TypeDescription uuidOrcSchema = ORCSchemaUtil.convert(uuidSchema); long estimateLength = getEstimateLength(uuidOrcSchema); - Assert.assertEquals("Estimated average length of uuid must be 128.", 128, estimateLength); + Assertions.assertThat(estimateLength) + .as("Estimated average length of uuid must be 128.") + .isEqualTo(128); } @Test @@ -188,7 +215,9 @@ public void testEstimateMapWidth() { Schema mapSchema = new Schema(MAP_FIELD_1); TypeDescription mapOrcSchema = ORCSchemaUtil.convert(mapSchema); long estimateLength = getEstimateLength(mapOrcSchema); - Assert.assertEquals("Estimated average length of map must be 136.", 136, estimateLength); + Assertions.assertThat(estimateLength) + .as("Estimated average length of map must be 136.") + .isEqualTo(136); } @Test @@ -196,7 +225,9 @@ public void testEstimateStructWidth() { Schema structSchema = new Schema(STRUCT_FIELD); TypeDescription structOrcSchema = ORCSchemaUtil.convert(structSchema); long estimateLength = getEstimateLength(structOrcSchema); - Assert.assertEquals("Estimated average length of struct must be 28.", 28, estimateLength); + Assertions.assertThat(estimateLength) + .as("Estimated average length of struct must be 28.") + .isEqualTo(28); } @Test @@ -217,7 +248,9 @@ public void testEstimateFullWidth() { STRUCT_FIELD); TypeDescription fullOrcSchema = ORCSchemaUtil.convert(fullSchema); long estimateLength = getEstimateLength(fullOrcSchema); - Assert.assertEquals("Estimated average length of the row must be 611.", 611, estimateLength); + Assertions.assertThat(estimateLength) + .as("Estimated average length of the row must be 611.") + .isEqualTo(611); } private Integer getEstimateLength(TypeDescription orcSchemaWithDate) { diff --git a/orc/src/test/java/org/apache/iceberg/orc/TestExpressionToSearchArgument.java b/orc/src/test/java/org/apache/iceberg/orc/TestExpressionToSearchArgument.java index 4af8768786bf..037f77dacb43 100644 --- a/orc/src/test/java/org/apache/iceberg/orc/TestExpressionToSearchArgument.java +++ b/orc/src/test/java/org/apache/iceberg/orc/TestExpressionToSearchArgument.java @@ -58,8 +58,8 @@ import org.apache.orc.storage.ql.io.sarg.SearchArgument.TruthValue; import org.apache.orc.storage.ql.io.sarg.SearchArgumentFactory; import org.apache.orc.storage.serde2.io.HiveDecimalWritable; -import org.junit.Assert; -import org.junit.Test; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; public class TestExpressionToSearchArgument { @@ -134,7 +134,7 @@ public void testPrimitiveTypes() { SearchArgument actual = ExpressionToSearchArgument.convert(boundFilter, ORCSchemaUtil.convert(schema)); - Assert.assertEquals(expected.toString(), actual.toString()); + Assertions.assertThat(actual.toString()).isEqualTo(expected.toString()); } @Test @@ -174,7 +174,7 @@ public void testTimezoneSensitiveTypes() { SearchArgument actual = ExpressionToSearchArgument.convert(boundFilter, ORCSchemaUtil.convert(schema)); - Assert.assertEquals(expected.toString(), actual.toString()); + Assertions.assertThat(actual.toString()).isEqualTo(expected.toString()); } } finally { TimeZone.setDefault(currentTz); @@ -212,7 +212,7 @@ public void testUnsupportedTypes() { SearchArgument actual = ExpressionToSearchArgument.convert(boundFilter, ORCSchemaUtil.convert(schema)); - Assert.assertEquals(expected.toString(), actual.toString()); + Assertions.assertThat(actual.toString()).isEqualTo(expected.toString()); } @Test @@ -261,7 +261,7 @@ public void testNestedPrimitives() { SearchArgument actual = ExpressionToSearchArgument.convert(boundFilter, ORCSchemaUtil.convert(schema)); - Assert.assertEquals(expected.toString(), actual.toString()); + Assertions.assertThat(actual.toString()).isEqualTo(expected.toString()); } @Test @@ -291,7 +291,7 @@ public void testSpecialCharacters() { SearchArgument actual = ExpressionToSearchArgument.convert(boundFilter, ORCSchemaUtil.convert(schema)); - Assert.assertEquals(expected.toString(), actual.toString()); + Assertions.assertThat(actual.toString()).isEqualTo(expected.toString()); } @Test @@ -315,7 +315,7 @@ public void testEvolvedSchema() { SearchArgumentFactory.newBuilder().equals("`int`", Type.LONG, 1L).build(); SearchArgument actual = ExpressionToSearchArgument.convert(boundFilter, readSchema); - Assert.assertEquals(expected.toString(), actual.toString()); + Assertions.assertThat(actual.toString()).isEqualTo(expected.toString()); // for columns not in the file, buildOrcProjection will append field names with _r // this will be passed down to ORC, but ORC will handle such cases and return a TruthValue @@ -326,7 +326,7 @@ public void testEvolvedSchema() { SearchArgumentFactory.newBuilder().equals("`float_added_r3`", Type.FLOAT, 1.0).build(); actual = ExpressionToSearchArgument.convert(boundFilter, readSchema); - Assert.assertEquals(expected.toString(), actual.toString()); + Assertions.assertThat(actual.toString()).isEqualTo(expected.toString()); } @Test @@ -352,7 +352,7 @@ public void testOriginalSchemaNameMapping() { .build(); SearchArgument actual = ExpressionToSearchArgument.convert(boundFilter, readSchema); - Assert.assertEquals(expected.toString(), actual.toString()); + Assertions.assertThat(actual.toString()).isEqualTo(expected.toString()); } @Test @@ -379,7 +379,7 @@ public void testModifiedSimpleSchemaNameMapping() { SearchArgumentFactory.newBuilder().equals("`int`", Type.LONG, 1L).build(); SearchArgument actual = ExpressionToSearchArgument.convert(boundFilter, readSchema); - Assert.assertEquals(expected.toString(), actual.toString()); + Assertions.assertThat(actual.toString()).isEqualTo(expected.toString()); // for columns not in the file, buildOrcProjection will append field names with _r // this will be passed down to ORC, but ORC will handle such cases and return a TruthValue @@ -390,7 +390,7 @@ public void testModifiedSimpleSchemaNameMapping() { SearchArgumentFactory.newBuilder().equals("`new_float_field_r3`", Type.FLOAT, 1.0).build(); actual = ExpressionToSearchArgument.convert(boundFilter, readSchema); - Assert.assertEquals(expected.toString(), actual.toString()); + Assertions.assertThat(actual.toString()).isEqualTo(expected.toString()); } @Test @@ -474,6 +474,6 @@ public void testModifiedComplexSchemaNameMapping() { .build(); SearchArgument actual = ExpressionToSearchArgument.convert(boundFilter, readSchema); - Assert.assertEquals(expected.toString(), actual.toString()); + Assertions.assertThat(actual.toString()).isEqualTo(expected.toString()); } } diff --git a/orc/src/test/java/org/apache/iceberg/orc/TestIdToOrcName.java b/orc/src/test/java/org/apache/iceberg/orc/TestIdToOrcName.java index 8d1ada5bd668..a323499b42d2 100644 --- a/orc/src/test/java/org/apache/iceberg/orc/TestIdToOrcName.java +++ b/orc/src/test/java/org/apache/iceberg/orc/TestIdToOrcName.java @@ -23,8 +23,8 @@ import java.util.Map; import org.apache.iceberg.Schema; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Test; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; public class TestIdToOrcName { @@ -70,31 +70,31 @@ public void testIdToQuotedColumnName() { required(26, "colWith`Quotes`", Types.LongType.get())); Map actual = ORCSchemaUtil.idToOrcName(schema); - Assert.assertEquals("`long`", actual.get(1)); - Assert.assertEquals("`struct`", actual.get(2)); - Assert.assertEquals("`struct`.`long`", actual.get(3)); - Assert.assertEquals("`listOfLongs`", actual.get(4)); - Assert.assertEquals("`listOfLongs`.`_elem`", actual.get(5)); - Assert.assertEquals("`listOfStructs`", actual.get(6)); - Assert.assertEquals("`listOfStructs`.`_elem`", actual.get(7)); - Assert.assertEquals("`listOfStructs`.`_elem`.`long`", actual.get(8)); - Assert.assertEquals("`map`", actual.get(9)); - Assert.assertEquals("`map`.`_key`", actual.get(10)); - Assert.assertEquals("`map`.`_value`", actual.get(11)); - Assert.assertEquals("`mapOfStructs`", actual.get(12)); - Assert.assertEquals("`mapOfStructs`.`_key`", actual.get(13)); - Assert.assertEquals("`mapOfStructs`.`_value`", actual.get(14)); - Assert.assertEquals("`mapOfStructs`.`_key`.`long`", actual.get(15)); - Assert.assertEquals("`mapOfStructs`.`_value`.`long`", actual.get(16)); - Assert.assertEquals("`listOfMapsOfStruct`", actual.get(17)); - Assert.assertEquals("`listOfMapsOfStruct`.`_elem`", actual.get(18)); - Assert.assertEquals("`listOfMapsOfStruct`.`_elem`.`_key`", actual.get(19)); - Assert.assertEquals("`listOfMapsOfStruct`.`_elem`.`_value`", actual.get(20)); - Assert.assertEquals("`listOfMapsOfStruct`.`_elem`.`_key`.`long`", actual.get(21)); - Assert.assertEquals("`listOfMapsOfStruct`.`_elem`.`_value`.`long`", actual.get(22)); - Assert.assertEquals("`col.with.dots`", actual.get(23)); - Assert.assertEquals("`col.with.dots`.`inner.col.with.dots`", actual.get(24)); - Assert.assertEquals("`colW!th$peci@lCh@rs`", actual.get(25)); - Assert.assertEquals("`colWith``Quotes```", actual.get(26)); + Assertions.assertThat(actual.get(1)).isEqualTo("`long`"); + Assertions.assertThat(actual.get(2)).isEqualTo("`struct`"); + Assertions.assertThat(actual.get(3)).isEqualTo("`struct`.`long`"); + Assertions.assertThat(actual.get(4)).isEqualTo("`listOfLongs`"); + Assertions.assertThat(actual.get(5)).isEqualTo("`listOfLongs`.`_elem`"); + Assertions.assertThat(actual.get(6)).isEqualTo("`listOfStructs`"); + Assertions.assertThat(actual.get(7)).isEqualTo("`listOfStructs`.`_elem`"); + Assertions.assertThat(actual.get(8)).isEqualTo("`listOfStructs`.`_elem`.`long`"); + Assertions.assertThat(actual.get(9)).isEqualTo("`map`"); + Assertions.assertThat(actual.get(10)).isEqualTo("`map`.`_key`"); + Assertions.assertThat(actual.get(11)).isEqualTo("`map`.`_value`"); + Assertions.assertThat(actual.get(12)).isEqualTo("`mapOfStructs`"); + Assertions.assertThat(actual.get(13)).isEqualTo("`mapOfStructs`.`_key`"); + Assertions.assertThat(actual.get(14)).isEqualTo("`mapOfStructs`.`_value`"); + Assertions.assertThat(actual.get(15)).isEqualTo("`mapOfStructs`.`_key`.`long`"); + Assertions.assertThat(actual.get(16)).isEqualTo("`mapOfStructs`.`_value`.`long`"); + Assertions.assertThat(actual.get(17)).isEqualTo("`listOfMapsOfStruct`"); + Assertions.assertThat(actual.get(18)).isEqualTo("`listOfMapsOfStruct`.`_elem`"); + Assertions.assertThat(actual.get(19)).isEqualTo("`listOfMapsOfStruct`.`_elem`.`_key`"); + Assertions.assertThat(actual.get(20)).isEqualTo("`listOfMapsOfStruct`.`_elem`.`_value`"); + Assertions.assertThat(actual.get(21)).isEqualTo("`listOfMapsOfStruct`.`_elem`.`_key`.`long`"); + Assertions.assertThat(actual.get(22)).isEqualTo("`listOfMapsOfStruct`.`_elem`.`_value`.`long`"); + Assertions.assertThat(actual.get(23)).isEqualTo("`col.with.dots`"); + Assertions.assertThat(actual.get(24)).isEqualTo("`col.with.dots`.`inner.col.with.dots`"); + Assertions.assertThat(actual.get(25)).isEqualTo("`colW!th$peci@lCh@rs`"); + Assertions.assertThat(actual.get(26)).isEqualTo("`colWith``Quotes```"); } } diff --git a/orc/src/test/java/org/apache/iceberg/orc/TestORCFileIOProxies.java b/orc/src/test/java/org/apache/iceberg/orc/TestORCFileIOProxies.java index 91e0152dee40..9b9eb688aa2a 100644 --- a/orc/src/test/java/org/apache/iceberg/orc/TestORCFileIOProxies.java +++ b/orc/src/test/java/org/apache/iceberg/orc/TestORCFileIOProxies.java @@ -18,9 +18,6 @@ */ package org.apache.iceberg.orc; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; - import java.io.File; import java.io.IOException; import java.io.InputStream; @@ -31,7 +28,7 @@ import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.OutputFile; import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestORCFileIOProxies { @Test @@ -42,7 +39,7 @@ public void testInputFileSystem() throws IOException { InputFile localFile = Files.localInput(inputFile); FileIOFSUtil.InputFileSystem ifs = new FileIOFSUtil.InputFileSystem(localFile); InputStream is = ifs.open(new Path(localFile.location())); - assertNotNull(is); + Assertions.assertThat(is).isNotNull(); // Cannot use the filesystem for any other operation Assertions.assertThatThrownBy(() -> ifs.getFileStatus(new Path(localFile.location()))) @@ -76,10 +73,10 @@ public void testOutputFileSystem() throws IOException { FileSystem ifs = new FileIOFSUtil.InputFileSystem(outputFile.toInputFile()); try (InputStream is = ifs.open(new Path(outputFile.location()))) { - assertEquals('O', is.read()); - assertEquals('R', is.read()); - assertEquals('C', is.read()); - assertEquals(-1, is.read()); + Assertions.assertThat(is.read()).isEqualTo('O'); + Assertions.assertThat(is.read()).isEqualTo('R'); + Assertions.assertThat(is.read()).isEqualTo('C'); + Assertions.assertThat(is.read()).isEqualTo(-1); } } } diff --git a/orc/src/test/java/org/apache/iceberg/orc/TestORCSchemaUtil.java b/orc/src/test/java/org/apache/iceberg/orc/TestORCSchemaUtil.java index 68f4a8f7c9c6..24a376cd71d3 100644 --- a/orc/src/test/java/org/apache/iceberg/orc/TestORCSchemaUtil.java +++ b/orc/src/test/java/org/apache/iceberg/orc/TestORCSchemaUtil.java @@ -22,9 +22,6 @@ import static org.apache.iceberg.orc.ORCSchemaUtil.ICEBERG_REQUIRED_ATTRIBUTE; import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; import java.util.Collections; import java.util.List; @@ -37,7 +34,7 @@ import org.apache.iceberg.types.Types; import org.apache.orc.TypeDescription; import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestORCSchemaUtil { @@ -65,7 +62,8 @@ public class TestORCSchemaUtil { @Test public void testRoundtripConversionPrimitive() { TypeDescription orcSchema = ORCSchemaUtil.convert(new Schema(SUPPORTED_PRIMITIVES.fields())); - assertEquals(SUPPORTED_PRIMITIVES, ORCSchemaUtil.convert(orcSchema).asStruct()); + Assertions.assertThat(ORCSchemaUtil.convert(orcSchema).asStruct()) + .isEqualTo(SUPPORTED_PRIMITIVES); } @Test @@ -191,7 +189,8 @@ public void testRoundtripConversionNested() { Types.ListType.ofRequired( 1250, nestedStructTypeForStruct)))))))); TypeDescription orcSchema = ORCSchemaUtil.convert(expectedSchema); - assertEquals(expectedSchema.asStruct(), ORCSchemaUtil.convert(orcSchema).asStruct()); + Assertions.assertThat(ORCSchemaUtil.convert(orcSchema).asStruct()) + .isEqualTo(expectedSchema.asStruct()); } @Test @@ -213,16 +212,18 @@ public void testTypePromotions() { optional(3, "c", Types.DecimalType.of(15, 2))); TypeDescription newOrcSchema = ORCSchemaUtil.buildOrcProjection(evolveSchema, orcSchema); - assertEquals(3, newOrcSchema.getChildren().size()); - assertEquals(1, newOrcSchema.findSubtype("a").getId()); - assertEquals(TypeDescription.Category.LONG, newOrcSchema.findSubtype("a").getCategory()); - assertEquals(2, newOrcSchema.findSubtype("b").getId()); - assertEquals(TypeDescription.Category.DOUBLE, newOrcSchema.findSubtype("b").getCategory()); + Assertions.assertThat(newOrcSchema.getChildren()).hasSize(3); + Assertions.assertThat(newOrcSchema.findSubtype("a").getId()).isEqualTo(1); + Assertions.assertThat(newOrcSchema.findSubtype("a").getCategory()) + .isEqualTo(TypeDescription.Category.LONG); + Assertions.assertThat(newOrcSchema.findSubtype("b").getId()).isEqualTo(2); + Assertions.assertThat(newOrcSchema.findSubtype("b").getCategory()) + .isEqualTo(TypeDescription.Category.DOUBLE); TypeDescription decimalC = newOrcSchema.findSubtype("c"); - assertEquals(3, decimalC.getId()); - assertEquals(TypeDescription.Category.DECIMAL, decimalC.getCategory()); - assertEquals(15, decimalC.getPrecision()); - assertEquals(2, decimalC.getScale()); + Assertions.assertThat(decimalC.getId()).isEqualTo(3); + Assertions.assertThat(decimalC.getCategory()).isEqualTo(TypeDescription.Category.DECIMAL); + Assertions.assertThat(decimalC.getPrecision()).isEqualTo(15); + Assertions.assertThat(decimalC.getScale()).isEqualTo(2); } @Test @@ -267,7 +268,9 @@ public void testSkipNonIcebergColumns() { 5, "mapCol", Types.MapType.ofOptional(3, 4, Types.StringType.get(), Types.BooleanType.get()))); - assertEquals("Schemas must match.", expectedSchema.asStruct(), icebergSchema.asStruct()); + Assertions.assertThat(icebergSchema.asStruct()) + .as("Schemas must match.") + .isEqualTo(expectedSchema.asStruct()); TypeDescription structCol = TypeDescription.createStruct(); structCol.setAttribute(ICEBERG_ID_ATTRIBUTE, "7"); @@ -301,7 +304,9 @@ public void testSkipNonIcebergColumns() { required(6, "doubleCol", Types.DoubleType.get()) // Skipped mapCol2 since value has no Iceberg ID ))); - assertEquals("Schemas must match.", expectedSchema2.asStruct(), icebergSchema2.asStruct()); + Assertions.assertThat(icebergSchema2.asStruct()) + .as("Schemas must match.") + .isEqualTo(expectedSchema2.asStruct()); } @Test @@ -322,13 +327,15 @@ public void testHasIds() { optional(4, "listCol", Types.ListType.ofOptional(40, Types.DoubleType.get()))); TypeDescription orcSchema = ORCSchemaUtil.removeIds(ORCSchemaUtil.convert(schema)); - assertFalse("Should not have Ids", ORCSchemaUtil.hasIds(orcSchema)); + Assertions.assertThat(ORCSchemaUtil.hasIds(orcSchema)).as("Should not have Ids").isFalse(); TypeDescription map2Col = TypeDescription.createMap(TypeDescription.createString(), TypeDescription.createBinary()); map2Col.setAttribute(ICEBERG_ID_ATTRIBUTE, "4"); orcSchema.addField("map2Col", map2Col); - assertTrue("Should have Ids after adding one type with Id", ORCSchemaUtil.hasIds(orcSchema)); + Assertions.assertThat(ORCSchemaUtil.hasIds(orcSchema)) + .as("Should have Ids after adding one type with Id") + .isTrue(); } @Test @@ -396,9 +403,10 @@ public void testAssignIdsByNameMapping() { ORCSchemaUtil.applyNameMapping( ORCSchemaUtil.removeIds(typeDescriptionWithIds), nameMapping); - assertTrue( - "TypeDescription schemas should be equal, including IDs", - equalsWithIds(typeDescriptionWithIds, typeDescriptionWithIdsFromNameMapping)); + Assertions.assertThat( + equalsWithIds(typeDescriptionWithIds, typeDescriptionWithIdsFromNameMapping)) + .as("TypeDescription schemas should be equal, including IDs") + .isTrue(); } @Test @@ -504,14 +512,16 @@ public void testAssignIdsByNameMappingAndProject() { longField.setAttribute(ICEBERG_ID_ATTRIBUTE, "40"); expected.addField("long_r40", longField); - assertTrue( - "ORC Schema must have the same structure, but one has Iceberg IDs", - typeDescriptionWithIdsFromNameMapping.equals(fileSchema, false)); + Assertions.assertThat(typeDescriptionWithIdsFromNameMapping.equals(fileSchema, false)) + .as("ORC Schema must have the same structure, but one has Iceberg IDs") + .isTrue(); TypeDescription projectedOrcSchema = ORCSchemaUtil.buildOrcProjection(mappingSchema, typeDescriptionWithIdsFromNameMapping); - assertTrue( - "Schema should be the prunned by projection", equalsWithIds(expected, projectedOrcSchema)); + + Assertions.assertThat(equalsWithIds(expected, projectedOrcSchema)) + .as("Schema should be the prunned by projection") + .isTrue(); } private static boolean equalsWithIds(TypeDescription first, TypeDescription second) { diff --git a/orc/src/test/java/org/apache/iceberg/orc/TestOrcDataReader.java b/orc/src/test/java/org/apache/iceberg/orc/TestOrcDataReader.java index 7131fdcc47b7..e9121c1c1e37 100644 --- a/orc/src/test/java/org/apache/iceberg/orc/TestOrcDataReader.java +++ b/orc/src/test/java/org/apache/iceberg/orc/TestOrcDataReader.java @@ -43,13 +43,13 @@ import org.apache.iceberg.types.Types; import org.apache.orc.OrcConf; import org.assertj.core.api.WithAssertions; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestOrcDataReader implements WithAssertions { - @ClassRule public static TemporaryFolder temp = new TemporaryFolder(); + + @TempDir private static File temp; private static final Schema SCHEMA = new Schema( @@ -61,7 +61,7 @@ public class TestOrcDataReader implements WithAssertions { private static DataFile dataFile; private static OutputFile outputFile; - @BeforeClass + @BeforeAll public static void createDataFile() throws IOException { GenericRecord bufferRecord = GenericRecord.create(SCHEMA); @@ -80,7 +80,7 @@ public static void createDataFile() throws IOException { bufferRecord.copy( ImmutableMap.of("id", 5L, "data", "e", "array", Arrays.asList(5, 6, 7, 8, 9)))); - outputFile = Files.localOutput(File.createTempFile("test", ".orc", temp.getRoot())); + outputFile = Files.localOutput(File.createTempFile("test", ".orc", new File(temp.getPath()))); DataWriter dataWriter = ORC.writeData(outputFile) diff --git a/orc/src/test/java/org/apache/iceberg/orc/TestOrcDataWriter.java b/orc/src/test/java/org/apache/iceberg/orc/TestOrcDataWriter.java index 58b48f25b56b..f95884dfa708 100644 --- a/orc/src/test/java/org/apache/iceberg/orc/TestOrcDataWriter.java +++ b/orc/src/test/java/org/apache/iceberg/orc/TestOrcDataWriter.java @@ -18,12 +18,12 @@ */ package org.apache.iceberg.orc; +import java.io.File; import java.io.IOException; import java.util.List; import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.UnsupportedFileSystemException; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileContent; import org.apache.iceberg.FileFormat; @@ -48,11 +48,9 @@ import org.apache.orc.OrcFile; import org.apache.orc.StripeInformation; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestOrcDataWriter { private static final Schema SCHEMA = @@ -63,9 +61,9 @@ public class TestOrcDataWriter { private List records; - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private File temp; - @Before + @BeforeEach public void createRecords() { GenericRecord record = GenericRecord.create(SCHEMA); @@ -92,7 +90,7 @@ private List stripeOffsetsFromReader(DataFile dataFile, OrcFile.ReaderOpti @Test public void testDataWriter() throws IOException { - OutputFile file = Files.localOutput(temp.newFile()); + OutputFile file = Files.localOutput(temp); SortOrder sortOrder = SortOrder.builderFor(SCHEMA).withOrderId(10).asc("id").build(); @@ -114,15 +112,13 @@ public void testDataWriter() throws IOException { } DataFile dataFile = dataWriter.toDataFile(); - Assert.assertEquals(dataFile.splitOffsets(), stripeOffsetsFromReader(dataFile)); - Assert.assertEquals("Format should be ORC", FileFormat.ORC, dataFile.format()); - Assert.assertEquals("Should be data file", FileContent.DATA, dataFile.content()); - Assert.assertEquals("Record count should match", records.size(), dataFile.recordCount()); - Assert.assertEquals("Partition should be empty", 0, dataFile.partition().size()); - Assert.assertEquals( - "Sort order should match", sortOrder.orderId(), (int) dataFile.sortOrderId()); - Assert.assertNull("Key metadata should be null", dataFile.keyMetadata()); - + Assertions.assertThat(dataFile.splitOffsets()).isEqualTo(stripeOffsetsFromReader(dataFile)); + Assertions.assertThat(dataFile.format()).isEqualTo(FileFormat.ORC); + Assertions.assertThat(dataFile.content()).isEqualTo(FileContent.DATA); + Assertions.assertThat(dataFile.recordCount()).isEqualTo(records.size()); + Assertions.assertThat(dataFile.partition().size()).isEqualTo(0); + Assertions.assertThat(dataFile.sortOrderId()).isEqualTo(sortOrder.orderId()); + Assertions.assertThat(dataFile.keyMetadata()).isNull(); List writtenRecords; try (CloseableIterable reader = ORC.read(file.toInputFile()) @@ -131,8 +127,7 @@ public void testDataWriter() throws IOException { .build()) { writtenRecords = Lists.newArrayList(reader); } - - Assert.assertEquals("Written records should match", records, writtenRecords); + Assertions.assertThat(writtenRecords).as("Written records should match").isEqualTo(records); } @Test @@ -140,12 +135,13 @@ public void testUsingFileIO() throws IOException { // When files other than HadoopInputFile and HadoopOutputFile are supplied the location // is used to determine the corresponding FileSystem class based on the scheme in case of // local files that would be the LocalFileSystem. To prevent this we use the Proxy classes to - // use a scheme `dummy` that is not handled. - ProxyOutputFile outFile = new ProxyOutputFile(Files.localOutput(temp.newFile())); + // use a scheme `dummy` that is not handled. Note that Hadoop 2.7.3 throws IOException + // while latest Hadoop versions throw UnsupportedFileSystemException (extends IOException) + ProxyOutputFile outFile = new ProxyOutputFile(Files.localOutput(temp)); Assertions.assertThatThrownBy( () -> new Path(outFile.location()).getFileSystem(new Configuration())) - .isInstanceOf(UnsupportedFileSystemException.class) - .hasMessageStartingWith("No FileSystem for scheme \"dummy\""); + .isInstanceOf(IOException.class) + .hasMessageStartingWith("No FileSystem for scheme"); // Given that FileIO is now handled there is no determination of FileSystem based on scheme // but instead operations are handled by the InputFileSystem and OutputFileSystem that wrap @@ -171,13 +167,13 @@ public void testUsingFileIO() throws IOException { OrcFile.readerOptions(new Configuration()) .filesystem(new FileIOFSUtil.InputFileSystem(outFile.toInputFile())) .maxLength(outFile.toInputFile().getLength()); - Assert.assertEquals(dataFile.splitOffsets(), stripeOffsetsFromReader(dataFile, options)); - Assert.assertEquals("Format should be ORC", FileFormat.ORC, dataFile.format()); - Assert.assertEquals("Should be data file", FileContent.DATA, dataFile.content()); - Assert.assertEquals("Record count should match", records.size(), dataFile.recordCount()); - Assert.assertEquals("Partition should be empty", 0, dataFile.partition().size()); - Assert.assertNull("Key metadata should be null", dataFile.keyMetadata()); - + Assertions.assertThat(dataFile.splitOffsets()) + .isEqualTo(stripeOffsetsFromReader(dataFile, options)); + Assertions.assertThat(dataFile.format()).isEqualTo(FileFormat.ORC); + Assertions.assertThat(dataFile.content()).isEqualTo(FileContent.DATA); + Assertions.assertThat(dataFile.recordCount()).isEqualTo(records.size()); + Assertions.assertThat(dataFile.partition().size()).isEqualTo(0); + Assertions.assertThat(dataFile.keyMetadata()).isNull(); List writtenRecords; try (CloseableIterable reader = ORC.read(outFile.toInputFile()) @@ -186,8 +182,7 @@ public void testUsingFileIO() throws IOException { .build()) { writtenRecords = Lists.newArrayList(reader); } - - Assert.assertEquals("Written records should match", records, writtenRecords); + Assertions.assertThat(writtenRecords).as("Written records should match").isEqualTo(records); } private static class ProxyInputFile implements InputFile { diff --git a/orc/src/test/java/org/apache/iceberg/orc/TestOrcDeleteWriters.java b/orc/src/test/java/org/apache/iceberg/orc/TestOrcDeleteWriters.java index 0ea6a8b5437d..3e0c13313624 100644 --- a/orc/src/test/java/org/apache/iceberg/orc/TestOrcDeleteWriters.java +++ b/orc/src/test/java/org/apache/iceberg/orc/TestOrcDeleteWriters.java @@ -41,11 +41,10 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestOrcDeleteWriters { private static final Schema SCHEMA = @@ -55,9 +54,9 @@ public class TestOrcDeleteWriters { private List records; - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private File temp; - @Before + @BeforeEach public void createDeleteRecords() { GenericRecord record = GenericRecord.create(SCHEMA); @@ -73,9 +72,7 @@ public void createDeleteRecords() { @Test public void testEqualityDeleteWriter() throws IOException { - File deleteFile = temp.newFile(); - - OutputFile out = Files.localOutput(deleteFile); + OutputFile out = Files.localOutput(temp); EqualityDeleteWriter deleteWriter = ORC.writeDeletes(out) .createWriterFunc(GenericOrcWriter::buildWriter) @@ -90,12 +87,11 @@ public void testEqualityDeleteWriter() throws IOException { } DeleteFile metadata = deleteWriter.toDeleteFile(); - Assert.assertEquals("Format should be ORC", FileFormat.ORC, metadata.format()); - Assert.assertEquals( - "Should be equality deletes", FileContent.EQUALITY_DELETES, metadata.content()); - Assert.assertEquals("Record count should be correct", records.size(), metadata.recordCount()); - Assert.assertEquals("Partition should be empty", 0, metadata.partition().size()); - Assert.assertNull("Key metadata should be null", metadata.keyMetadata()); + Assertions.assertThat(metadata.format()).isEqualTo(FileFormat.ORC); + Assertions.assertThat(metadata.content()).isEqualTo(FileContent.EQUALITY_DELETES); + Assertions.assertThat(metadata.recordCount()).isEqualTo(records.size()); + Assertions.assertThat(metadata.partition().size()).isEqualTo(0); + Assertions.assertThat(metadata.keyMetadata()).isNull(); List deletedRecords; try (CloseableIterable reader = @@ -106,13 +102,13 @@ public void testEqualityDeleteWriter() throws IOException { deletedRecords = Lists.newArrayList(reader); } - Assert.assertEquals("Deleted records should match expected", records, deletedRecords); + Assertions.assertThat(deletedRecords) + .as("Deleted records should match expected") + .isEqualTo(records); } @Test public void testPositionDeleteWriter() throws IOException { - File deleteFile = temp.newFile(); - Schema deleteSchema = new Schema( MetadataColumns.DELETE_FILE_PATH, @@ -124,7 +120,7 @@ public void testPositionDeleteWriter() throws IOException { GenericRecord posDelete = GenericRecord.create(deleteSchema); List expectedDeleteRecords = Lists.newArrayList(); - OutputFile out = Files.localOutput(deleteFile); + OutputFile out = Files.localOutput(temp); PositionDeleteWriter deleteWriter = ORC.writeDeletes(out) .createWriterFunc(GenericOrcWriter::buildWriter) @@ -147,12 +143,11 @@ public void testPositionDeleteWriter() throws IOException { } DeleteFile metadata = deleteWriter.toDeleteFile(); - Assert.assertEquals("Format should be ORC", FileFormat.ORC, metadata.format()); - Assert.assertEquals( - "Should be position deletes", FileContent.POSITION_DELETES, metadata.content()); - Assert.assertEquals("Record count should be correct", records.size(), metadata.recordCount()); - Assert.assertEquals("Partition should be empty", 0, metadata.partition().size()); - Assert.assertNull("Key metadata should be null", metadata.keyMetadata()); + Assertions.assertThat(metadata.format()).isEqualTo(FileFormat.ORC); + Assertions.assertThat(metadata.content()).isEqualTo(FileContent.POSITION_DELETES); + Assertions.assertThat(metadata.recordCount()).isEqualTo(records.size()); + Assertions.assertThat(metadata.partition().size()).isEqualTo(0); + Assertions.assertThat(metadata.keyMetadata()).isNull(); List deletedRecords; try (CloseableIterable reader = @@ -163,14 +158,13 @@ public void testPositionDeleteWriter() throws IOException { deletedRecords = Lists.newArrayList(reader); } - Assert.assertEquals( - "Deleted records should match expected", expectedDeleteRecords, deletedRecords); + Assertions.assertThat(deletedRecords) + .as("Deleted records should match expected") + .isEqualTo(expectedDeleteRecords); } @Test public void testPositionDeleteWriterWithEmptyRow() throws IOException { - File deleteFile = temp.newFile(); - Schema deleteSchema = new Schema(MetadataColumns.DELETE_FILE_PATH, MetadataColumns.DELETE_FILE_POS); @@ -178,7 +172,7 @@ public void testPositionDeleteWriterWithEmptyRow() throws IOException { GenericRecord posDelete = GenericRecord.create(deleteSchema); List expectedDeleteRecords = Lists.newArrayList(); - OutputFile out = Files.localOutput(deleteFile); + OutputFile out = Files.localOutput(temp); PositionDeleteWriter deleteWriter = ORC.writeDeletes(out) .createWriterFunc(GenericOrcWriter::buildWriter) @@ -202,12 +196,11 @@ public void testPositionDeleteWriterWithEmptyRow() throws IOException { } DeleteFile metadata = deleteWriter.toDeleteFile(); - Assert.assertEquals("Format should be ORC", FileFormat.ORC, metadata.format()); - Assert.assertEquals( - "Should be position deletes", FileContent.POSITION_DELETES, metadata.content()); - Assert.assertEquals("Record count should be correct", records.size(), metadata.recordCount()); - Assert.assertEquals("Partition should be empty", 0, metadata.partition().size()); - Assert.assertNull("Key metadata should be null", metadata.keyMetadata()); + Assertions.assertThat(metadata.format()).isEqualTo(FileFormat.ORC); + Assertions.assertThat(metadata.content()).isEqualTo(FileContent.POSITION_DELETES); + Assertions.assertThat(metadata.recordCount()).isEqualTo(records.size()); + Assertions.assertThat(metadata.partition().size()).isEqualTo(0); + Assertions.assertThat(metadata.keyMetadata()).isNull(); List deletedRecords; try (CloseableIterable reader = @@ -217,8 +210,8 @@ public void testPositionDeleteWriterWithEmptyRow() throws IOException { .build()) { deletedRecords = Lists.newArrayList(reader); } - - Assert.assertEquals( - "Deleted records should match expected", expectedDeleteRecords, deletedRecords); + Assertions.assertThat(deletedRecords) + .as("Deleted records should match expected") + .isEqualTo(expectedDeleteRecords); } } diff --git a/orc/src/test/java/org/apache/iceberg/orc/TestTableProperties.java b/orc/src/test/java/org/apache/iceberg/orc/TestTableProperties.java index 09fe8ff915e2..339e885e581e 100644 --- a/orc/src/test/java/org/apache/iceberg/orc/TestTableProperties.java +++ b/orc/src/test/java/org/apache/iceberg/orc/TestTableProperties.java @@ -38,10 +38,9 @@ import org.apache.orc.CompressionKind; import org.apache.orc.OrcConf; import org.apache.orc.OrcFile.CompressionStrategy; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestTableProperties { @@ -50,7 +49,8 @@ public class TestTableProperties { Types.NestedField.optional(1, "id", Types.IntegerType.get()), Types.NestedField.optional(2, "data", Types.StringType.get())); - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @TempDir private File folder; + @TempDir private File testFile; @Test public void testOrcTableProperties() throws Exception { @@ -71,17 +71,16 @@ public void testOrcTableProperties() throws Exception { TableProperties.ORC_COMPRESSION_STRATEGY, strategyAsString, TableProperties.DEFAULT_FILE_FORMAT, FileFormat.ORC.name()); - File folder = TEMPORARY_FOLDER.newFolder(); - String warehouse = folder.getAbsolutePath(); String tablePath = warehouse.concat("/test"); - Assert.assertTrue("Should create the table path correctly.", new File(tablePath).mkdir()); + Assertions.assertThat(new File(tablePath).mkdir()) + .as("Should create the table path correctly.") + .isTrue(); PartitionSpec spec = PartitionSpec.unpartitioned(); Table table = new HadoopTables().create(SCHEMA, spec, properties, tablePath); - File testFile = TEMPORARY_FOLDER.newFile(); - Assert.assertTrue(testFile.delete()); + Assertions.assertThat(testFile.delete()).isTrue(); FileAppender writer = ORC.write(Files.localOutput(testFile)) @@ -93,12 +92,13 @@ public void testOrcTableProperties() throws Exception { DynFields.builder().hiddenImpl(writer.getClass(), "conf").build(writer); Configuration configuration = confField.get(); - Assert.assertEquals(blockSizeBytes, OrcConf.BLOCK_SIZE.getLong(configuration)); - Assert.assertEquals(stripeSizeBytes, OrcConf.STRIPE_SIZE.getLong(configuration)); - Assert.assertEquals(codecAsString, OrcConf.COMPRESS.getString(configuration)); - Assert.assertEquals(strategyAsString, OrcConf.COMPRESSION_STRATEGY.getString(configuration)); - Assert.assertEquals( - FileFormat.ORC.name(), configuration.get(TableProperties.DEFAULT_FILE_FORMAT)); + Assertions.assertThat(OrcConf.BLOCK_SIZE.getLong(configuration)).isEqualTo(blockSizeBytes); + Assertions.assertThat(OrcConf.STRIPE_SIZE.getLong(configuration)).isEqualTo(stripeSizeBytes); + Assertions.assertThat(OrcConf.COMPRESS.getString(configuration)).isEqualTo(codecAsString); + Assertions.assertThat(OrcConf.COMPRESSION_STRATEGY.getString(configuration)) + .isEqualTo(strategyAsString); + Assertions.assertThat(configuration.get(TableProperties.DEFAULT_FILE_FORMAT)) + .isEqualTo(FileFormat.ORC.name()); } @Test @@ -120,17 +120,16 @@ public void testOrcTableDeleteProperties() throws Exception { TableProperties.DELETE_ORC_COMPRESSION_STRATEGY, strategyAsString, TableProperties.DEFAULT_FILE_FORMAT, FileFormat.ORC.name()); - File folder = TEMPORARY_FOLDER.newFolder(); - String warehouse = folder.getAbsolutePath(); String tablePath = warehouse.concat("/test"); - Assert.assertTrue("Should create the table path correctly.", new File(tablePath).mkdir()); + Assertions.assertThat(new File(tablePath).mkdir()) + .as("Should create the table path correctly.") + .isTrue(); PartitionSpec spec = PartitionSpec.unpartitioned(); Table table = new HadoopTables().create(SCHEMA, spec, properties, tablePath); - File testFile = TEMPORARY_FOLDER.newFile(); - Assert.assertTrue(testFile.delete()); + Assertions.assertThat(testFile.delete()).isTrue(); EqualityDeleteWriter deleteWriter = ORC.writeDeletes(Files.localOutput(testFile)) @@ -147,11 +146,12 @@ public void testOrcTableDeleteProperties() throws Exception { DynFields.builder().hiddenImpl(orcFileAppender.getClass(), "conf").build(orcFileAppender); Configuration configuration = confField.get(); - Assert.assertEquals(blockSizeBytes, OrcConf.BLOCK_SIZE.getLong(configuration)); - Assert.assertEquals(stripeSizeBytes, OrcConf.STRIPE_SIZE.getLong(configuration)); - Assert.assertEquals(codecAsString, OrcConf.COMPRESS.getString(configuration)); - Assert.assertEquals(strategyAsString, OrcConf.COMPRESSION_STRATEGY.getString(configuration)); - Assert.assertEquals( - FileFormat.ORC.name(), configuration.get(TableProperties.DEFAULT_FILE_FORMAT)); + Assertions.assertThat(OrcConf.BLOCK_SIZE.getLong(configuration)).isEqualTo(blockSizeBytes); + Assertions.assertThat(OrcConf.STRIPE_SIZE.getLong(configuration)).isEqualTo(stripeSizeBytes); + Assertions.assertThat(OrcConf.COMPRESS.getString(configuration)).isEqualTo(codecAsString); + Assertions.assertThat(OrcConf.COMPRESSION_STRATEGY.getString(configuration)) + .isEqualTo(strategyAsString); + Assertions.assertThat(configuration.get(TableProperties.DEFAULT_FILE_FORMAT)) + .isEqualTo(FileFormat.ORC.name()); } } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java index 078c4c4b167a..d240c84b9e4d 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java @@ -1101,12 +1101,8 @@ public CloseableIterable build() { conf.unset(property); } optionsBuilder = HadoopReadOptions.builder(conf); - // page size not used by decompressors - optionsBuilder.withCodecFactory(new ParquetCodecFactory(conf, 0)); } else { optionsBuilder = ParquetReadOptions.builder(); - // page size not used by decompressors - optionsBuilder.withCodecFactory(new ParquetCodecFactory(new Configuration(), 0)); } for (Map.Entry entry : properties.entrySet()) { diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvro.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvro.java index f985d979d7bc..974c00076b7d 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvro.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvro.java @@ -31,7 +31,6 @@ import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericFixed; import org.apache.avro.specific.SpecificData; -import org.apache.commons.math3.util.Pair; import org.apache.iceberg.avro.AvroSchemaVisitor; import org.apache.iceberg.avro.UUIDConversion; import org.apache.iceberg.relocated.com.google.common.base.Objects; @@ -368,4 +367,26 @@ private static Schema.Field copyField(Schema.Field field, Schema newSchema) { return copy; } } + + private static class Pair { + private final K first; + private final V second; + + Pair(final K first, final V second) { + this.first = first; + this.second = second; + } + + public static Pair of(K first, V second) { + return new Pair<>(first, second); + } + + public K getFirst() { + return first; + } + + public V getSecond() { + return second; + } + } } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetCodecFactory.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetCodecFactory.java index b61a31f04cd8..bfcece6259a6 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetCodecFactory.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetCodecFactory.java @@ -18,21 +18,17 @@ */ package org.apache.iceberg.parquet; -import java.io.IOException; -import java.io.InputStream; -import java.nio.ByteBuffer; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.io.compress.CodecPool; import org.apache.hadoop.io.compress.CompressionCodec; -import org.apache.hadoop.io.compress.Decompressor; -import org.apache.parquet.bytes.BytesInput; +import org.apache.hadoop.util.ReflectionUtils; +import org.apache.parquet.hadoop.BadConfigurationException; import org.apache.parquet.hadoop.CodecFactory; -import org.apache.parquet.hadoop.codec.ZstandardCodec; import org.apache.parquet.hadoop.metadata.CompressionCodecName; /** * This class implements a codec factory that is used when reading from Parquet. It adds a - * workaround for memory issues encountered when reading from zstd-compressed files. + * workaround to cache codecs by name and level, not just by name. This can be removed when this + * change is made to Parquet. */ public class ParquetCodecFactory extends CodecFactory { @@ -40,62 +36,57 @@ public ParquetCodecFactory(Configuration configuration, int pageSize) { super(configuration, pageSize); } - /** Copied and modified from CodecFactory.HeapBytesDecompressor */ - class HeapBytesDecompressor extends BytesDecompressor { - - private final CompressionCodec codec; - private final Decompressor decompressor; - - HeapBytesDecompressor(CompressionCodecName codecName) { - this.codec = getCodec(codecName); - if (codec != null) { - decompressor = CodecPool.getDecompressor(codec); - } else { - decompressor = null; - } - } - - @Override - public BytesInput decompress(BytesInput bytes, int uncompressedSize) throws IOException { - final BytesInput decompressed; - if (codec != null) { - if (decompressor != null) { - decompressor.reset(); - } - if (codec instanceof ZstandardCodec) { - // we need to close the zstd input stream ASAP to free up native resources, so - // read everything into a buffer and then close it - try (InputStream is = codec.createInputStream(bytes.toInputStream(), decompressor)) { - decompressed = BytesInput.copy(BytesInput.from(is, uncompressedSize)); - } - } else { - InputStream is = codec.createInputStream(bytes.toInputStream(), decompressor); - decompressed = BytesInput.from(is, uncompressedSize); - } - } else { - decompressed = bytes; - } - return decompressed; + /** + * This is copied from {@link CodecFactory} and modified to include the level in the cache key. + */ + @Override + protected CompressionCodec getCodec(CompressionCodecName codecName) { + String codecClassName = codecName.getHadoopCompressionCodecClassName(); + if (codecClassName == null) { + return null; } - - @Override - public void decompress( - ByteBuffer input, int compressedSize, ByteBuffer output, int uncompressedSize) - throws IOException { - ByteBuffer decompressed = decompress(BytesInput.from(input), uncompressedSize).toByteBuffer(); - output.put(decompressed); + String cacheKey = cacheKey(codecName); + CompressionCodec codec = CODEC_BY_NAME.get(cacheKey); + if (codec != null) { + return codec; } - @Override - public void release() { - if (decompressor != null) { - CodecPool.returnDecompressor(decompressor); + try { + Class codecClass; + try { + codecClass = Class.forName(codecClassName); + } catch (ClassNotFoundException e) { + // Try to load the class using the job classloader + codecClass = configuration.getClassLoader().loadClass(codecClassName); } + codec = (CompressionCodec) ReflectionUtils.newInstance(codecClass, configuration); + CODEC_BY_NAME.put(cacheKey, codec); + return codec; + } catch (ClassNotFoundException e) { + throw new BadConfigurationException("Class " + codecClassName + " was not found", e); } } - @Override - protected BytesDecompressor createDecompressor(CompressionCodecName codecName) { - return new HeapBytesDecompressor(codecName); + private String cacheKey(CompressionCodecName codecName) { + String level = null; + switch (codecName) { + case GZIP: + level = configuration.get("zlib.compress.level"); + break; + case BROTLI: + level = configuration.get("compression.brotli.quality"); + break; + case ZSTD: + level = configuration.get("parquet.compression.codec.zstd.level"); + if (level == null) { + // keep "io.compression.codec.zstd.level" for backwards compatibility + level = configuration.get("io.compression.codec.zstd.level"); + } + break; + default: + // compression level is not supported; ignore it + } + String codecClass = codecName.getHadoopCompressionCodecClassName(); + return level == null ? codecClass : codecClass + ":" + level; } } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriter.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriter.java index 577004993711..099cffc33bb8 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriter.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriter.java @@ -86,7 +86,8 @@ class ParquetWriter implements FileAppender, Closeable { this.targetRowGroupSize = rowGroupSize; this.props = properties; this.metadata = ImmutableMap.copyOf(metadata); - this.compressor = new CodecFactory(conf, props.getPageSizeThreshold()).getCompressor(codec); + this.compressor = + new ParquetCodecFactory(conf, props.getPageSizeThreshold()).getCompressor(codec); this.parquetSchema = ParquetSchemaUtil.convert(schema, "table"); this.model = (ParquetValueWriter) createWriterFunc.apply(parquetSchema); this.metricsConfig = metricsConfig; diff --git a/parquet/src/test/java/org/apache/iceberg/TestHelpers.java b/parquet/src/test/java/org/apache/iceberg/TestHelpers.java index be6ebe93d59e..0e7627cab1f5 100644 --- a/parquet/src/test/java/org/apache/iceberg/TestHelpers.java +++ b/parquet/src/test/java/org/apache/iceberg/TestHelpers.java @@ -18,11 +18,12 @@ */ package org.apache.iceberg; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.concurrent.Callable; import org.apache.avro.AvroRuntimeException; import org.apache.avro.generic.GenericRecord; import org.assertj.core.api.AbstractThrowableAssert; -import org.assertj.core.api.Assertions; public class TestHelpers { @@ -42,7 +43,7 @@ public static void assertThrows( String containedInMessage, Callable callable) { AbstractThrowableAssert check = - Assertions.assertThatThrownBy(callable::call).as(message).isInstanceOf(expected); + assertThatThrownBy(callable::call).as(message).isInstanceOf(expected); if (null != containedInMessage) { check.hasMessageContaining(containedInMessage); } @@ -62,7 +63,7 @@ public static void assertThrows( String containedInMessage, Runnable runnable) { AbstractThrowableAssert check = - Assertions.assertThatThrownBy(runnable::run).as(message).isInstanceOf(expected); + assertThatThrownBy(runnable::run).as(message).isInstanceOf(expected); if (null != containedInMessage) { check.hasMessageContaining(containedInMessage); } diff --git a/parquet/src/test/java/org/apache/iceberg/avro/TestParquetReadProjection.java b/parquet/src/test/java/org/apache/iceberg/avro/TestParquetReadProjection.java index 13bf1a37a119..2df806b1fb9a 100644 --- a/parquet/src/test/java/org/apache/iceberg/avro/TestParquetReadProjection.java +++ b/parquet/src/test/java/org/apache/iceberg/avro/TestParquetReadProjection.java @@ -32,7 +32,7 @@ public class TestParquetReadProjection extends TestReadProjection { protected GenericData.Record writeAndRead( String desc, Schema writeSchema, Schema readSchema, GenericData.Record record) throws IOException { - File file = temp.newFile(desc + ".parquet"); + File file = temp.resolve(desc + ".parquet").toFile(); file.delete(); try (FileAppender appender = diff --git a/parquet/src/test/java/org/apache/iceberg/avro/TestReadProjection.java b/parquet/src/test/java/org/apache/iceberg/avro/TestReadProjection.java index 851cf5f7eef9..93c6ad05379e 100644 --- a/parquet/src/test/java/org/apache/iceberg/avro/TestReadProjection.java +++ b/parquet/src/test/java/org/apache/iceberg/avro/TestReadProjection.java @@ -18,7 +18,12 @@ */ package org.apache.iceberg.avro; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assertions.within; + import java.io.IOException; +import java.nio.file.Path; import java.util.List; import java.util.Map; import org.apache.avro.generic.GenericData.Record; @@ -27,20 +32,15 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.types.Comparators; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public abstract class TestReadProjection { protected abstract Record writeAndRead( String desc, Schema writeSchema, Schema readSchema, Record record) throws IOException; - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir protected Path temp; @Test public void testFullProjection() throws Exception { @@ -55,10 +55,11 @@ public void testFullProjection() throws Exception { Record projected = writeAndRead("full_projection", schema, schema, record); - Assert.assertEquals("Should contain the correct id value", 34L, (long) projected.get("id")); + assertThat((long) projected.get("id")).as("Should contain the correct id value").isEqualTo(34L); - int cmp = Comparators.charSequences().compare("test", (CharSequence) projected.get("data")); - Assert.assertTrue("Should contain the correct data value", cmp == 0); + assertThat(projected.get("data").toString()) + .as("Should contain the correct data value") + .isEqualTo("test"); } @Test @@ -79,8 +80,10 @@ public void testReorderedFullProjection() throws Exception { Record projected = writeAndRead("full_projection", schema, reordered, record); - Assert.assertEquals("Should contain the correct 0 value", "test", projected.get(0).toString()); - Assert.assertEquals("Should contain the correct 1 value", 34L, projected.get(1)); + assertThat(projected.get(0).toString()) + .as("Should contain the correct 0 value") + .isEqualTo("test"); + assertThat(projected.get(1)).as("Should contain the correct 1 value").isEqualTo(34L); } @Test @@ -102,9 +105,11 @@ public void testReorderedProjection() throws Exception { Record projected = writeAndRead("full_projection", schema, reordered, record); - Assert.assertNull("Should contain the correct 0 value", projected.get(0)); - Assert.assertEquals("Should contain the correct 1 value", "test", projected.get(1).toString()); - Assert.assertNull("Should contain the correct 2 value", projected.get(2)); + assertThat(projected.get(0)).as("Should contain the correct 0 value").isNull(); + assertThat(projected.get(1).toString()) + .as("Should contain the correct 1 value") + .isEqualTo("test"); + assertThat(projected.get(2)).as("Should contain the correct 2 value").isNull(); } @Test @@ -120,10 +125,9 @@ public void testEmptyProjection() throws Exception { Record projected = writeAndRead("empty_projection", schema, schema.select(), record); - Assert.assertNotNull("Should read a non-null record", projected); + assertThat(projected).as("Should read a non-null record").isNotNull(); // this is expected because there are no values - Assertions.assertThatThrownBy(() -> projected.get(0)) - .isInstanceOf(ArrayIndexOutOfBoundsException.class); + assertThatThrownBy(() -> projected.get(0)).isInstanceOf(ArrayIndexOutOfBoundsException.class); } @Test @@ -141,15 +145,16 @@ public void testBasicProjection() throws Exception { Record projected = writeAndRead("basic_projection_id", writeSchema, idOnly, record); TestHelpers.assertEmptyAvroField(projected, "data"); - Assert.assertEquals("Should contain the correct id value", 34L, (long) projected.get("id")); + assertThat((long) projected.get("id")).as("Should contain the correct id value").isEqualTo(34L); Schema dataOnly = new Schema(Types.NestedField.optional(1, "data", Types.StringType.get())); projected = writeAndRead("basic_projection_data", writeSchema, dataOnly, record); TestHelpers.assertEmptyAvroField(projected, "id"); - int cmp = Comparators.charSequences().compare("test", (CharSequence) projected.get("data")); - Assert.assertEquals("Should contain the correct data value", 0, cmp); + assertThat(projected.get("data").toString()) + .as("Should contain the correct data value") + .isEqualTo("test"); } @Test @@ -170,9 +175,11 @@ public void testRename() throws Exception { Record projected = writeAndRead("project_and_rename", writeSchema, readSchema, record); - Assert.assertEquals("Should contain the correct id value", 34L, (long) projected.get("id")); - int cmp = Comparators.charSequences().compare("test", (CharSequence) projected.get("renamed")); - Assert.assertTrue("Should contain the correct data/renamed value", cmp == 0); + assertThat((long) projected.get("id")).as("Should contain the correct id value").isEqualTo(34L); + + assertThat(projected.get("renamed").toString()) + .as("Should contain the correct data/renamed value") + .isEqualTo("test"); } @Test @@ -199,7 +206,7 @@ public void testNestedStructProjection() throws Exception { Record projected = writeAndRead("id_only", writeSchema, idOnly, record); TestHelpers.assertEmptyAvroField(projected, "location"); - Assert.assertEquals("Should contain the correct id value", 34L, (long) projected.get("id")); + assertThat((long) projected.get("id")).as("Should contain the correct id value").isEqualTo(34L); Schema latOnly = new Schema( @@ -211,10 +218,11 @@ public void testNestedStructProjection() throws Exception { projected = writeAndRead("latitude_only", writeSchema, latOnly, record); Record projectedLocation = (Record) projected.get("location"); TestHelpers.assertEmptyAvroField(projected, "id"); - Assert.assertNotNull("Should project location", projected.get("location")); + assertThat(projected.get("location")).as("Should project location").isNotNull(); TestHelpers.assertEmptyAvroField(projectedLocation, "long"); - Assert.assertEquals( - "Should project latitude", 52.995143f, (float) projectedLocation.get("lat"), 0.000001f); + assertThat((float) projectedLocation.get("lat")) + .as("Should project latitude") + .isCloseTo(52.995143f, within(0.000001f)); Schema longOnly = new Schema( @@ -226,20 +234,24 @@ public void testNestedStructProjection() throws Exception { projected = writeAndRead("longitude_only", writeSchema, longOnly, record); projectedLocation = (Record) projected.get("location"); TestHelpers.assertEmptyAvroField(projected, "id"); - Assert.assertNotNull("Should project location", projected.get("location")); + assertThat(projected.get("location")).as("Should project location").isNotNull(); TestHelpers.assertEmptyAvroField(projectedLocation, "lat"); - Assert.assertEquals( - "Should project longitude", -1.539054f, (float) projectedLocation.get("long"), 0.000001f); + assertThat((float) projectedLocation.get("long")) + .as("Should project longitude") + .isCloseTo(-1.539054f, within(0.000001f)); Schema locationOnly = writeSchema.select("location"); projected = writeAndRead("location_only", writeSchema, locationOnly, record); projectedLocation = (Record) projected.get("location"); TestHelpers.assertEmptyAvroField(projected, "id"); - Assert.assertNotNull("Should project location", projected.get("location")); - Assert.assertEquals( - "Should project latitude", 52.995143f, (float) projectedLocation.get("lat"), 0.000001f); - Assert.assertEquals( - "Should project longitude", -1.539054f, (float) projectedLocation.get("long"), 0.000001f); + assertThat(projected.get("location")).as("Should project location").isNotNull(); + assertThat((float) projectedLocation.get("lat")) + .as("Should project latitude") + .isCloseTo(52.995143f, within(0.000001f)); + + assertThat((float) projectedLocation.get("long")) + .as("Should project longitude") + .isCloseTo(-1.539054f, within(0.000001f)); } @Test @@ -261,26 +273,29 @@ public void testMapProjection() throws IOException { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); Record projected = writeAndRead("id_only", writeSchema, idOnly, record); - Assert.assertEquals("Should contain the correct id value", 34L, (long) projected.get("id")); + assertThat((long) projected.get("id")).as("Should contain the correct id value").isEqualTo(34L); TestHelpers.assertEmptyAvroField(projected, "properties"); Schema keyOnly = writeSchema.select("properties.key"); projected = writeAndRead("key_only", writeSchema, keyOnly, record); TestHelpers.assertEmptyAvroField(projected, "id"); - Assert.assertEquals( - "Should project entire map", properties, toStringMap((Map) projected.get("properties"))); + assertThat(toStringMap((Map) projected.get("properties"))) + .as("Should project entire map") + .isEqualTo(properties); Schema valueOnly = writeSchema.select("properties.value"); projected = writeAndRead("value_only", writeSchema, valueOnly, record); TestHelpers.assertEmptyAvroField(projected, "id"); - Assert.assertEquals( - "Should project entire map", properties, toStringMap((Map) projected.get("properties"))); + assertThat(toStringMap((Map) projected.get("properties"))) + .as("Should project entire map") + .isEqualTo(properties); Schema mapOnly = writeSchema.select("properties"); projected = writeAndRead("map_only", writeSchema, mapOnly, record); TestHelpers.assertEmptyAvroField(projected, "id"); - Assert.assertEquals( - "Should project entire map", properties, toStringMap((Map) projected.get("properties"))); + assertThat(toStringMap((Map) projected.get("properties"))) + .as("Should project entire map") + .isEqualTo(properties); } private Map toStringMap(Map map) { @@ -328,50 +343,54 @@ public void testMapOfStructsProjection() throws IOException { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); Record projected = writeAndRead("id_only", writeSchema, idOnly, record); - Assert.assertEquals("Should contain the correct id value", 34L, (long) projected.get("id")); + assertThat((long) projected.get("id")).as("Should contain the correct id value").isEqualTo(34L); TestHelpers.assertEmptyAvroField(projected, "locations"); projected = writeAndRead("all_locations", writeSchema, writeSchema.select("locations"), record); TestHelpers.assertEmptyAvroField(projected, "id"); - Assert.assertEquals( - "Should project locations map", - record.get("locations"), - toStringMap((Map) projected.get("locations"))); + assertThat(toStringMap((Map) projected.get("locations"))) + .as("Should project locations map") + .isEqualTo(record.get("locations")); projected = writeAndRead("lat_only", writeSchema, writeSchema.select("locations.lat"), record); TestHelpers.assertEmptyAvroField(projected, "id"); Map locations = toStringMap((Map) projected.get("locations")); - Assert.assertNotNull("Should project locations map", locations); - Assert.assertEquals( - "Should contain L1 and L2", Sets.newHashSet("L1", "L2"), locations.keySet()); + assertThat(locations).as("Should project locations map").isNotNull(); + assertThat(locations.keySet()).as("Should contain L1 and L2").containsExactly("L1", "L2"); Record projectedL1 = (Record) locations.get("L1"); - Assert.assertNotNull("L1 should not be null", projectedL1); - Assert.assertEquals( - "L1 should contain lat", 53.992811f, (float) projectedL1.get("lat"), 0.000001); + assertThat(projectedL1).as("L1 should not be null").isNotNull(); + assertThat((float) projectedL1.get("lat")) + .as("L1 should contain lat") + .isCloseTo(53.992811f, within(0.000001f)); + TestHelpers.assertEmptyAvroField(projectedL1, "long"); Record projectedL2 = (Record) locations.get("L2"); - Assert.assertNotNull("L2 should not be null", projectedL2); - Assert.assertEquals( - "L2 should contain lat", 52.995143f, (float) projectedL2.get("lat"), 0.000001); + assertThat(projectedL2).as("L2 should not be null").isNotNull(); + assertThat((float) projectedL2.get("lat")) + .as("L2 should contain lat") + .isCloseTo(52.995143f, within(0.000001f)); + TestHelpers.assertEmptyAvroField(projectedL2, "long"); projected = writeAndRead("long_only", writeSchema, writeSchema.select("locations.long"), record); TestHelpers.assertEmptyAvroField(projected, "id"); locations = toStringMap((Map) projected.get("locations")); - Assert.assertNotNull("Should project locations map", locations); - Assert.assertEquals( - "Should contain L1 and L2", Sets.newHashSet("L1", "L2"), locations.keySet()); + assertThat(locations).as("Should project locations map").isNotNull(); + assertThat(locations.keySet()).as("Should contain L1 and L2").containsExactly("L1", "L2"); projectedL1 = (Record) locations.get("L1"); - Assert.assertNotNull("L1 should not be null", projectedL1); + assertThat(projectedL1).as("L1 should not be null").isNotNull(); TestHelpers.assertEmptyAvroField(projectedL1, "lat"); - Assert.assertEquals( - "L1 should contain long", -1.542616f, (float) projectedL1.get("long"), 0.000001); + assertThat((float) projectedL1.get("long")) + .as("L1 should contain long") + .isCloseTo(-1.542616f, within(0.000001f)); + projectedL2 = (Record) locations.get("L2"); - Assert.assertNotNull("L2 should not be null", projectedL2); + assertThat(projectedL2).as("L2 should not be null").isNotNull(); TestHelpers.assertEmptyAvroField(projectedL2, "lat"); - Assert.assertEquals( - "L2 should contain long", -1.539054f, (float) projectedL2.get("long"), 0.000001); + assertThat((float) projectedL2.get("long")) + .as("L2 should contain long") + .isCloseTo(-1.539054f, within(0.000001f)); Schema latitiudeRenamed = new Schema( @@ -388,19 +407,21 @@ public void testMapOfStructsProjection() throws IOException { projected = writeAndRead("latitude_renamed", writeSchema, latitiudeRenamed, record); TestHelpers.assertEmptyAvroField(projected, "id"); locations = toStringMap((Map) projected.get("locations")); - Assert.assertNotNull("Should project locations map", locations); - Assert.assertEquals( - "Should contain L1 and L2", Sets.newHashSet("L1", "L2"), locations.keySet()); + assertThat(locations).as("Should project locations map").isNotNull(); + assertThat(locations.keySet()).as("Should contain L1 and L2").containsExactly("L1", "L2"); projectedL1 = (Record) locations.get("L1"); - Assert.assertNotNull("L1 should not be null", projectedL1); - Assert.assertEquals( - "L1 should contain latitude", 53.992811f, (float) projectedL1.get("latitude"), 0.000001); + assertThat(projectedL1).as("L1 should not be null").isNotNull(); + assertThat((float) projectedL1.get("latitude")) + .as("L1 should contain latitude") + .isCloseTo(53.992811f, within(0.000001f)); TestHelpers.assertEmptyAvroField(projectedL1, "lat"); TestHelpers.assertEmptyAvroField(projectedL1, "long"); projectedL2 = (Record) locations.get("L2"); - Assert.assertNotNull("L2 should not be null", projectedL2); - Assert.assertEquals( - "L2 should contain latitude", 52.995143f, (float) projectedL2.get("latitude"), 0.000001); + assertThat(projectedL2).as("L2 should not be null").isNotNull(); + assertThat((float) projectedL2.get("latitude")) + .as("L2 should contain latitude") + .isCloseTo(52.995143f, within(0.000001f)); + TestHelpers.assertEmptyAvroField(projectedL2, "lat"); TestHelpers.assertEmptyAvroField(projectedL2, "long"); } @@ -422,18 +443,18 @@ public void testListProjection() throws IOException { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); Record projected = writeAndRead("id_only", writeSchema, idOnly, record); - Assert.assertEquals("Should contain the correct id value", 34L, (long) projected.get("id")); + assertThat((long) projected.get("id")).as("Should contain the correct id value").isEqualTo(34L); TestHelpers.assertEmptyAvroField(projected, "values"); Schema elementOnly = writeSchema.select("values.element"); projected = writeAndRead("element_only", writeSchema, elementOnly, record); TestHelpers.assertEmptyAvroField(projected, "id"); - Assert.assertEquals("Should project entire list", values, projected.get("values")); + assertThat(projected.get("values")).as("Should project entire list").isEqualTo(values); Schema listOnly = writeSchema.select("values"); projected = writeAndRead("list_only", writeSchema, listOnly, record); TestHelpers.assertEmptyAvroField(projected, "id"); - Assert.assertEquals("Should project entire list", values, projected.get("values")); + assertThat(projected.get("values")).as("Should project entire list").isEqualTo(values); } @Test @@ -468,37 +489,38 @@ public void testListOfStructsProjection() throws IOException { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); Record projected = writeAndRead("id_only", writeSchema, idOnly, record); - Assert.assertEquals("Should contain the correct id value", 34L, (long) projected.get("id")); + assertThat((long) projected.get("id")).as("Should contain the correct id value").isEqualTo(34L); TestHelpers.assertEmptyAvroField(projected, "points"); projected = writeAndRead("all_points", writeSchema, writeSchema.select("points"), record); TestHelpers.assertEmptyAvroField(projected, "id"); - Assert.assertEquals( - "Should project points list", record.get("points"), projected.get("points")); + assertThat(projected.get("points")) + .as("Should project points list") + .isEqualTo(record.get("points")); projected = writeAndRead("x_only", writeSchema, writeSchema.select("points.x"), record); TestHelpers.assertEmptyAvroField(projected, "id"); - Assert.assertNotNull("Should project points list", projected.get("points")); + assertThat(projected.get("points")).as("Should project points list").isNotNull(); List points = (List) projected.get("points"); - Assert.assertEquals("Should read 2 points", 2, points.size()); + assertThat(points).as("Should read 2 points").hasSize(2); Record projectedP1 = points.get(0); - Assert.assertEquals("Should project x", 1, (int) projectedP1.get("x")); + assertThat((int) projectedP1.get("x")).as("Should project x").isEqualTo(1); TestHelpers.assertEmptyAvroField(projectedP1, "y"); Record projectedP2 = points.get(1); - Assert.assertEquals("Should project x", 3, (int) projectedP2.get("x")); + assertThat((int) projectedP2.get("x")).as("Should project x").isEqualTo(3); TestHelpers.assertEmptyAvroField(projectedP2, "y"); projected = writeAndRead("y_only", writeSchema, writeSchema.select("points.y"), record); TestHelpers.assertEmptyAvroField(projected, "id"); - Assert.assertNotNull("Should project points list", projected.get("points")); + assertThat(projected.get("points")).as("Should project points list").isNotNull(); points = (List) projected.get("points"); - Assert.assertEquals("Should read 2 points", 2, points.size()); + assertThat(points).as("Should read 2 points").hasSize(2); projectedP1 = points.get(0); TestHelpers.assertEmptyAvroField(projectedP1, "x"); - Assert.assertEquals("Should project y", 2, (int) projectedP1.get("y")); + assertThat((int) projectedP1.get("y")).as("Should project y").isEqualTo(2); projectedP2 = points.get(1); TestHelpers.assertEmptyAvroField(projectedP2, "x"); - Assert.assertNull("Should project null y", projectedP2.get("y")); + assertThat(projectedP2.get("y")).as("Should project null y").isNull(); Schema yRenamed = new Schema( @@ -512,16 +534,16 @@ public void testListOfStructsProjection() throws IOException { projected = writeAndRead("y_renamed", writeSchema, yRenamed, record); TestHelpers.assertEmptyAvroField(projected, "id"); - Assert.assertNotNull("Should project points list", projected.get("points")); + assertThat(projected.get("points")).as("Should project points list").isNotNull(); points = (List) projected.get("points"); - Assert.assertEquals("Should read 2 points", 2, points.size()); + assertThat(points).as("Should read 2 points").hasSize(2); projectedP1 = points.get(0); TestHelpers.assertEmptyAvroField(projectedP1, "x"); TestHelpers.assertEmptyAvroField(projectedP1, "y"); - Assert.assertEquals("Should project z", 2, (int) projectedP1.get("z")); + assertThat((int) projectedP1.get("z")).as("Should project z").isEqualTo(2); projectedP2 = points.get(1); TestHelpers.assertEmptyAvroField(projectedP2, "x"); TestHelpers.assertEmptyAvroField(projectedP2, "y"); - Assert.assertNull("Should project null z", projectedP2.get("z")); + assertThat(projectedP2.get("z")).as("Should project null z").isNull(); } } diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/ParquetWritingTestUtils.java b/parquet/src/test/java/org/apache/iceberg/parquet/ParquetWritingTestUtils.java index 58463bbb1edc..09673e603338 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/ParquetWritingTestUtils.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/ParquetWritingTestUtils.java @@ -23,6 +23,7 @@ import java.io.Closeable; import java.io.File; import java.io.IOException; +import java.nio.file.Path; import java.util.Collections; import java.util.Map; import java.util.UUID; @@ -33,29 +34,25 @@ import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.parquet.schema.MessageType; -import org.junit.rules.TemporaryFolder; /** Utilities for tests that need to write Parquet files. */ class ParquetWritingTestUtils { private ParquetWritingTestUtils() {} - static File writeRecords(TemporaryFolder temp, Schema schema, GenericData.Record... records) + static File writeRecords(Path temp, Schema schema, GenericData.Record... records) throws IOException { return writeRecords(temp, schema, Collections.emptyMap(), null, records); } static File writeRecords( - TemporaryFolder temp, - Schema schema, - Map properties, - GenericData.Record... records) + Path temp, Schema schema, Map properties, GenericData.Record... records) throws IOException { return writeRecords(temp, schema, properties, null, records); } static File writeRecords( - TemporaryFolder temp, + Path temp, Schema schema, Map properties, Function> createWriterFunc, @@ -97,8 +94,8 @@ static long write( return len; } - static File createTempFile(TemporaryFolder temp) throws IOException { - File tmpFolder = temp.newFolder("parquet"); + static File createTempFile(Path temp) throws IOException { + File tmpFolder = temp.resolve("parquet").toFile(); String filename = UUID.randomUUID().toString(); return new File(tmpFolder, FileFormat.PARQUET.addExtension(filename)); } diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java index 756639ab128c..8af8a512b62b 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java @@ -38,6 +38,7 @@ import static org.apache.iceberg.expressions.Expressions.startsWith; import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; @@ -74,11 +75,9 @@ import org.apache.parquet.hadoop.metadata.BlockMetaData; import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; import org.apache.parquet.schema.MessageType; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestBloomRowGroupFilter { @@ -184,17 +183,17 @@ public class TestBloomRowGroupFilter { private BlockMetaData rowGroupMetadata = null; private BloomFilterReader bloomStore = null; - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private File temp; - @Before + @BeforeEach public void createInputFile() throws IOException { - File parquetFile = temp.newFile(); - Assert.assertTrue(parquetFile.delete()); + + assertThat(temp.delete()).isTrue(); // build struct field schema org.apache.avro.Schema structSchema = AvroSchemaUtil.convert(_structFieldType); - OutputFile outFile = Files.localOutput(parquetFile); + OutputFile outFile = Files.localOutput(temp); try (FileAppender appender = Parquet.write(outFile) .schema(FILE_SCHEMA) @@ -264,11 +263,11 @@ public void createInputFile() throws IOException { } } - InputFile inFile = Files.localInput(parquetFile); + InputFile inFile = Files.localInput(temp); ParquetFileReader reader = ParquetFileReader.open(ParquetIO.file(inFile)); - Assert.assertEquals("Should create only one row group", 1, reader.getRowGroups().size()); + assertThat(reader.getRowGroups()).as("Should create only one row group").hasSize(1); rowGroupMetadata = reader.getRowGroups().get(0); parquetSchema = reader.getFileMetaData().getSchema(); bloomStore = reader.getBloomFilterDataReader(rowGroupMetadata); @@ -279,22 +278,24 @@ public void testNotNull() { boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, notNull("all_nulls")) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, notNull("some_nulls")) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, notNull("no_nulls")) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, notNull("struct_not_null.int_field")) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: this field is required and are always not-null", shouldRead); + assertThat(shouldRead) + .as("Should read: this field is required and are always not-null") + .isTrue(); } @Test @@ -302,22 +303,24 @@ public void testIsNull() { boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, isNull("all_nulls")) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, isNull("some_nulls")) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, isNull("no_nulls")) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, isNull("struct_not_null.int_field")) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertFalse("Should skip: this field is required and are always not-null", shouldRead); + assertThat(shouldRead) + .as("Should skip: this field is required and are always not-null") + .isFalse(); } @Test @@ -325,12 +328,12 @@ public void testRequiredColumn() { boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, notNull("required")) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: required columns are always non-null", shouldRead); + assertThat(shouldRead).as("Should read: required columns are always non-null").isTrue(); shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, isNull("required")) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertFalse("Should skip: required columns are always non-null", shouldRead); + assertThat(shouldRead).as("Should skip: required columns are always non-null").isFalse(); } @Test @@ -338,17 +341,17 @@ public void testIsNaNs() { boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, isNaN("all_nans")) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, isNaN("some_nans")) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, isNaN("no_nans")) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); } @Test @@ -356,17 +359,17 @@ public void testNotNaNs() { boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, notNaN("all_nans")) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, notNaN("some_nans")) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, notNaN("no_nans")) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); } @Test @@ -374,37 +377,37 @@ public void testStartsWith() { boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, startsWith("non_bloom", "re")) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: no bloom", shouldRead); + assertThat(shouldRead).as("Should read: no bloom").isTrue(); shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, startsWith("required", "re")) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, startsWith("required", "req")) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, startsWith("some_nulls", "so")) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, startsWith("required", "reqs")) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, startsWith("some_nulls", "somex")) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, startsWith("no_nulls", "xxx")) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); } @Test @@ -437,7 +440,7 @@ public void testColumnNotInFile() { boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, expr) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter cannot be found: " + expr, shouldRead); + assertThat(shouldRead).as("Should read: bloom filter cannot be found: " + expr).isTrue(); } } @@ -454,7 +457,7 @@ public void testColumnNotBloomFilterEnabled() { boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, expr) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter cannot be found: " + expr, shouldRead); + assertThat(shouldRead).as("Should read: bloom filter cannot be found: " + expr).isTrue(); } } @@ -463,7 +466,9 @@ public void testMissingStats() { boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, equal("no_stats", "a")) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertFalse("Should skip: stats are missing but bloom filter is present", shouldRead); + assertThat(shouldRead) + .as("Should skip: stats are missing but bloom filter is present") + .isFalse(); } @Test @@ -474,7 +479,7 @@ public void testNot() { boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, not(equal("id", i))) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); } } @@ -486,19 +491,19 @@ public void testAnd() { new ParquetBloomRowGroupFilter( SCHEMA, and(equal("id", INT_MIN_VALUE - 25), equal("id", INT_MIN_VALUE + 30))) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertFalse("Should skip: and(false, true)", shouldRead); + assertThat(shouldRead).as("Should skip: and(false, true)").isFalse(); shouldRead = new ParquetBloomRowGroupFilter( SCHEMA, and(equal("id", INT_MIN_VALUE - 25), equal("id", INT_MAX_VALUE + 1))) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertFalse("Should skip: and(false, false)", shouldRead); + assertThat(shouldRead).as("Should skip: and(false, false)").isFalse(); shouldRead = new ParquetBloomRowGroupFilter( SCHEMA, and(equal("id", INT_MIN_VALUE + 25), equal("id", INT_MIN_VALUE))) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: and(true, true)", shouldRead); + assertThat(shouldRead).as("Should read: and(true, true)").isTrue(); // AND filters that refer different columns ("id", "long", "binary") shouldRead = @@ -509,7 +514,7 @@ SCHEMA, and(equal("id", INT_MIN_VALUE + 25), equal("id", INT_MIN_VALUE))) equal("long", LONG_BASE + 30), equal("binary", RANDOM_BYTES.get(30)))) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: and(true, true, true)", shouldRead); + assertThat(shouldRead).as("Should read: and(true, true, true)").isTrue(); // AND filters that refer different columns ("id", "long", "binary") shouldRead = @@ -520,7 +525,7 @@ SCHEMA, and(equal("id", INT_MIN_VALUE + 25), equal("id", INT_MIN_VALUE))) equal("long", LONG_BASE + 30), equal("binary", RANDOM_BYTES.get(30)))) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertFalse("Should skip: and(false, true, true)", shouldRead); + assertThat(shouldRead).as("Should skip: and(false, true, true)").isFalse(); // In And, one of the filter's column doesn't have bloom filter shouldRead = @@ -532,7 +537,7 @@ SCHEMA, and(equal("id", INT_MIN_VALUE + 25), equal("id", INT_MIN_VALUE))) equal("binary", RANDOM_BYTES.get(30)), equal("non_bloom", "a"))) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: and(true, true, true, true)", shouldRead); + assertThat(shouldRead).as("Should read: and(true, true, true, true)").isTrue(); // In And, one of the filter's column doesn't have bloom filter shouldRead = @@ -544,7 +549,7 @@ SCHEMA, and(equal("id", INT_MIN_VALUE + 25), equal("id", INT_MIN_VALUE))) equal("binary", RANDOM_BYTES.get(30)), equal("non_bloom", "a"))) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertFalse("Should skip: and(false, true, true, true)", shouldRead); + assertThat(shouldRead).as("Should skip: and(false, true, true, true)").isFalse(); // In And, one of the filter's column is not in the file shouldRead = @@ -556,7 +561,7 @@ SCHEMA, and(equal("id", INT_MIN_VALUE + 25), equal("id", INT_MIN_VALUE))) equal("binary", RANDOM_BYTES.get(30)), equal("not_in_file", 1.0f))) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: and(true, true, true, true)", shouldRead); + assertThat(shouldRead).as("Should read: and(true, true, true, true)").isTrue(); // In And, one of the filter's column is not in the file shouldRead = @@ -568,7 +573,7 @@ SCHEMA, and(equal("id", INT_MIN_VALUE + 25), equal("id", INT_MIN_VALUE))) equal("binary", RANDOM_BYTES.get(30)), equal("not_in_file", 1.0f))) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertFalse("Should skip: and(false, true, true, true)", shouldRead); + assertThat(shouldRead).as("Should skip: and(false, true, true, true)").isFalse(); } @Test @@ -579,13 +584,13 @@ public void testOr() { new ParquetBloomRowGroupFilter( SCHEMA, or(equal("id", INT_MIN_VALUE - 25), equal("id", INT_MAX_VALUE + 1))) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertFalse("Should skip: or(false, false)", shouldRead); + assertThat(shouldRead).as("Should skip: or(false, false)").isFalse(); shouldRead = new ParquetBloomRowGroupFilter( SCHEMA, or(equal("id", INT_MIN_VALUE - 25), equal("id", INT_MAX_VALUE - 19))) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: or(false, true)", shouldRead); + assertThat(shouldRead).as("Should read: or(false, true)").isTrue(); } @Test @@ -594,7 +599,7 @@ public void testIntegerLt() { boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, lessThan("id", i)) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); } } @@ -604,7 +609,7 @@ public void testIntegerLtEq() { boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, lessThanOrEqual("id", i)) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); } } @@ -614,7 +619,7 @@ public void testIntegerGt() { boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, greaterThan("id", i)) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); } } @@ -624,7 +629,7 @@ public void testIntegerGtEq() { boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, greaterThanOrEqual("id", i)) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); } } @@ -635,9 +640,9 @@ public void testIntegerEq() { new ParquetBloomRowGroupFilter(SCHEMA, equal("id", i)) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); if (i >= INT_MIN_VALUE && i <= INT_MAX_VALUE) { - Assert.assertTrue("Should read: integer within range", shouldRead); + assertThat(shouldRead).as("Should read: integer within range").isTrue(); } else { - Assert.assertFalse("Should not read: integer outside range", shouldRead); + assertThat(shouldRead).as("Should not read: integer outside range").isFalse(); } } } @@ -649,9 +654,9 @@ public void testLongEq() { new ParquetBloomRowGroupFilter(SCHEMA, equal("long", LONG_BASE + i)) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); if (i >= INT_MIN_VALUE && i <= INT_MAX_VALUE) { - Assert.assertTrue("Should read: long within range", shouldRead); + assertThat(shouldRead).as("Should read: long within range").isTrue(); } else { - Assert.assertFalse("Should not read: long outside range", shouldRead); + assertThat(shouldRead).as("Should not read: long outside range").isFalse(); } } } @@ -662,7 +667,7 @@ public void testBytesEq() { boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, equal("binary", RANDOM_BYTES.get(i))) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: binary within range", shouldRead); + assertThat(shouldRead).as("Should read: binary within range").isTrue(); } Random rd = new Random(54321); @@ -672,7 +677,7 @@ public void testBytesEq() { boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, equal("binary", byteArray)) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertFalse("Should not read: cannot match a new generated binary", shouldRead); + assertThat(shouldRead).as("Should not read: cannot match a new generated binary").isFalse(); } } @@ -683,13 +688,13 @@ public void testIntDeciamlEq() { new ParquetBloomRowGroupFilter( SCHEMA, equal("int_decimal", new BigDecimal(String.valueOf(77.77 + i)))) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: decimal within range", shouldRead); + assertThat(shouldRead).as("Should read: decimal within range").isTrue(); } boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, equal("int_decimal", new BigDecimal("1234.56"))) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertFalse("Should not read: decimal outside range", shouldRead); + assertThat(shouldRead).as("Should not read: decimal outside range").isFalse(); } @Test @@ -699,13 +704,13 @@ public void testLongDeciamlEq() { new ParquetBloomRowGroupFilter( SCHEMA, equal("long_decimal", new BigDecimal(String.valueOf(88.88 + i)))) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: decimal within range", shouldRead); + assertThat(shouldRead).as("Should read: decimal within range").isTrue(); } boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, equal("long_decimal", new BigDecimal("1234.56"))) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertFalse("Should not read: decimal outside range", shouldRead); + assertThat(shouldRead).as("Should not read: decimal outside range").isFalse(); } @Test @@ -715,13 +720,13 @@ public void testFixedDeciamlEq() { new ParquetBloomRowGroupFilter( SCHEMA, equal("fixed_decimal", new BigDecimal(String.valueOf(99.99 + i)))) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: decimal within range", shouldRead); + assertThat(shouldRead).as("Should read: decimal within range").isTrue(); } boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, equal("fixed_decimal", new BigDecimal("1234.56"))) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertFalse("Should not read: decimal outside range", shouldRead); + assertThat(shouldRead).as("Should not read: decimal outside range").isFalse(); } @Test @@ -731,9 +736,9 @@ public void testDoubleEq() { new ParquetBloomRowGroupFilter(SCHEMA, equal("double", DOUBLE_BASE + i)) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); if (i >= INT_MIN_VALUE && i <= INT_MAX_VALUE) { - Assert.assertTrue("Should read: double within range", shouldRead); + assertThat(shouldRead).as("Should read: double within range").isTrue(); } else { - Assert.assertFalse("Should not read: double outside range", shouldRead); + assertThat(shouldRead).as("Should not read: double outside range").isFalse(); } } } @@ -745,9 +750,9 @@ public void testFloatEq() { new ParquetBloomRowGroupFilter(SCHEMA, equal("float", FLOAT_BASE + i)) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); if (i >= INT_MIN_VALUE && i <= INT_MAX_VALUE) { - Assert.assertTrue("Should read: float within range", shouldRead); + assertThat(shouldRead).as("Should read: float within range").isTrue(); } else { - Assert.assertFalse("Should not read: float outside range", shouldRead); + assertThat(shouldRead).as("Should not read: float outside range").isFalse(); } } } @@ -759,9 +764,9 @@ public void testStringEq() { new ParquetBloomRowGroupFilter(SCHEMA, equal("string", BINARY_PREFIX + i)) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); if (i >= INT_MIN_VALUE && i <= INT_MAX_VALUE) { - Assert.assertTrue("Should read: string within range", shouldRead); + assertThat(shouldRead).as("Should read: string within range").isTrue(); } else { - Assert.assertFalse("Should not read: string outside range", shouldRead); + assertThat(shouldRead).as("Should not read: string outside range").isFalse(); } } } @@ -772,7 +777,7 @@ public void testRandomBinaryEq() { boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, equal("uuid", RANDOM_UUIDS.get(i))) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: uuid within range", shouldRead); + assertThat(shouldRead).as("Should read: uuid within range").isTrue(); } Random rd = new Random(1357); @@ -780,7 +785,9 @@ public void testRandomBinaryEq() { new ParquetBloomRowGroupFilter( SCHEMA, equal("uuid", new UUID(rd.nextLong(), rd.nextLong()).toString())) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertFalse("Should not read: cannot match a new generated random uuid", shouldRead); + assertThat(shouldRead) + .as("Should not read: cannot match a new generated random uuid") + .isFalse(); } @Test @@ -788,12 +795,12 @@ public void testBooleanEq() { boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, equal("boolean", true)) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter is not supported for Boolean", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter is not supported for Boolean").isTrue(); shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, equal("boolean", false)) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter is not supported for Boolean", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter is not supported for Boolean").isTrue(); } @Test @@ -804,9 +811,9 @@ public void testTimeEq() { new ParquetBloomRowGroupFilter(SCHEMA, equal("time", ins.toEpochMilli())) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); if (i >= 0 && i < INT_VALUE_COUNT) { - Assert.assertTrue("Should read: time within range", shouldRead); + assertThat(shouldRead).as("Should read: time within range").isTrue(); } else { - Assert.assertFalse("Should not read: time outside range", shouldRead); + assertThat(shouldRead).as("Should not read: time outside range").isFalse(); } } } @@ -819,9 +826,9 @@ public void testDateEq() { new ParquetBloomRowGroupFilter(SCHEMA, equal("date", ins.getEpochSecond())) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); if (i >= 0 && i < INT_VALUE_COUNT) { - Assert.assertTrue("Should read: date within range", shouldRead); + assertThat(shouldRead).as("Should read: date within range").isTrue(); } else { - Assert.assertFalse("Should not read: date outside range", shouldRead); + assertThat(shouldRead).as("Should not read: date outside range").isFalse(); } } } @@ -834,9 +841,9 @@ public void testTimestampEq() { new ParquetBloomRowGroupFilter(SCHEMA, equal("timestamp", ins.toEpochMilli())) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); if (i >= 0 && i < INT_VALUE_COUNT) { - Assert.assertTrue("Should read: timestamp within range", shouldRead); + assertThat(shouldRead).as("Should read: timestamp within range").isTrue(); } else { - Assert.assertFalse("Should not read: timestamp outside range", shouldRead); + assertThat(shouldRead).as("Should not read: timestamp outside range").isFalse(); } } } @@ -849,9 +856,9 @@ public void testTimestamptzEq() { new ParquetBloomRowGroupFilter(SCHEMA, equal("timestamptz", ins.toEpochMilli())) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); if (i >= 0 && i < INT_VALUE_COUNT) { - Assert.assertTrue("Should read: timestamptz within range", shouldRead); + assertThat(shouldRead).as("Should read: timestamptz within range").isTrue(); } else { - Assert.assertFalse("Should not read: timestamptz outside range", shouldRead); + assertThat(shouldRead).as("Should not read: timestamptz outside range").isFalse(); } } } @@ -862,7 +869,7 @@ public void testIntegerNotEq() { boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, notEqual("id", i)) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); } } @@ -872,7 +879,7 @@ public void testIntegerNotEqRewritten() { boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, not(equal("id", i))) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); } } @@ -881,12 +888,12 @@ public void testStringNotEq() { boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, notEqual("some_nulls", "some")) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, notEqual("no_nulls", "")) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); } @Test @@ -895,7 +902,7 @@ public void testStructFieldLt() { boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, lessThan("struct_not_null.int_field", i)) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); } } @@ -905,7 +912,7 @@ public void testStructFieldLtEq() { boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, lessThanOrEqual("struct_not_null.int_field", i)) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); } } @@ -915,7 +922,7 @@ public void testStructFieldGt() { boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, greaterThan("struct_not_null.int_field", i)) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); } } @@ -925,7 +932,7 @@ public void testStructFieldGtEq() { boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, greaterThanOrEqual("struct_not_null.int_field", i)) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); } } @@ -936,9 +943,9 @@ public void testStructFieldEq() { new ParquetBloomRowGroupFilter(SCHEMA, equal("struct_not_null.int_field", i)) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); if (i >= INT_MIN_VALUE && i <= INT_MAX_VALUE) { - Assert.assertTrue("Should read: value within range", shouldRead); + assertThat(shouldRead).as("Should read: value within range").isTrue(); } else { - Assert.assertFalse("Should not read: value outside range", shouldRead); + assertThat(shouldRead).as("Should not read: value outside range").isFalse(); } } } @@ -949,7 +956,7 @@ public void testStructFieldNotEq() { boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, notEqual("struct_not_null.int_field", i)) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); } } @@ -960,7 +967,7 @@ public void testCaseInsensitive() { boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, equal("Required", "Req"), false) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertFalse("Should skip: contains only 'req'", shouldRead); + assertThat(shouldRead).as("Should skip: contains only 'req'").isFalse(); } @Test @@ -996,7 +1003,7 @@ public void testIntegerIn() { new ParquetBloomRowGroupFilter( SCHEMA, in("id", INT_MIN_VALUE - 3 * i, INT_MIN_VALUE + i, INT_MAX_VALUE + 3 * i)) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: integer within range", shouldRead); + assertThat(shouldRead).as("Should read: integer within range").isTrue(); } // all values are present @@ -1009,7 +1016,7 @@ SCHEMA, in("id", INT_MIN_VALUE - 3 * i, INT_MIN_VALUE + i, INT_MAX_VALUE + 3 * i .boxed() .collect(Collectors.toList()))) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: the bloom is a subset of the in set", shouldRead); + assertThat(shouldRead).as("Should read: the bloom is a subset of the in set").isTrue(); // all values are present shouldRead = @@ -1021,7 +1028,7 @@ SCHEMA, in("id", INT_MIN_VALUE - 3 * i, INT_MIN_VALUE + i, INT_MAX_VALUE + 3 * i .boxed() .collect(Collectors.toList()))) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: the bloom is equal to the in set", shouldRead); + assertThat(shouldRead).as("Should read: the bloom is equal to the in set").isTrue(); // no values are present shouldRead = @@ -1033,7 +1040,7 @@ SCHEMA, in("id", INT_MIN_VALUE - 3 * i, INT_MIN_VALUE + i, INT_MAX_VALUE + 3 * i .boxed() .collect(Collectors.toList()))) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertFalse("Should not read: value outside range", shouldRead); + assertThat(shouldRead).as("Should not read: value outside range").isFalse(); } @Test @@ -1041,29 +1048,35 @@ public void testOtherTypesIn() { boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, in("all_nulls", 1, 2)) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertFalse("Should not read: in on all nulls column (bloom is empty) ", shouldRead); + assertThat(shouldRead) + .as("Should not read: in on all nulls column (bloom is empty) ") + .isFalse(); shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, in("some_nulls", "aaa", "some")) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: in on some nulls column", shouldRead); + assertThat(shouldRead).as("Should read: in on some nulls column").isTrue(); shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, in("some_nulls", "aaa", "bbb")) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertFalse("Should not read: some_nulls values are not within the set", shouldRead); + assertThat(shouldRead) + .as("Should not read: some_nulls values are not within the set") + .isFalse(); shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, in("no_nulls", "aaa", "bbb")) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertFalse( - "Should not read: in on no nulls column (empty string is not within the set)", shouldRead); + assertThat(shouldRead) + .as("Should not read: in on no nulls column (empty string is not within the set)") + .isFalse(); shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, in("no_nulls", "aaa", "")) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue( - "Should read: in on no nulls column (empty string is within the set)", shouldRead); + assertThat(shouldRead) + .as("Should read: in on no nulls column (empty string is within the set)") + .isTrue(); } @Test @@ -1075,7 +1088,7 @@ public void testIntegerNotIn() { SCHEMA, notIn("id", INT_MIN_VALUE - 3 * i, INT_MIN_VALUE + i, INT_MAX_VALUE + 3 * i)) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); } // all values are present @@ -1088,7 +1101,7 @@ public void testIntegerNotIn() { .boxed() .collect(Collectors.toList()))) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); // all values are present shouldRead = @@ -1100,7 +1113,7 @@ public void testIntegerNotIn() { .boxed() .collect(Collectors.toList()))) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); // no values are present shouldRead = @@ -1112,7 +1125,7 @@ public void testIntegerNotIn() { .boxed() .collect(Collectors.toList()))) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); } @Test @@ -1120,22 +1133,22 @@ public void testOtherTypesNotIn() { boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, notIn("all_nulls", 1, 2)) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, notIn("some_nulls", "aaa", "bbb")) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, notIn("no_nulls", "aaa", "bbb")) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, notIn("no_nulls", "aaa", "")) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: bloom filter doesn't help").isTrue(); } @Test @@ -1143,26 +1156,26 @@ public void testTypeConversions() { boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, equal("long", LONG_BASE + INT_MIN_VALUE + 1), true) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: Integer value promoted", shouldRead); + assertThat(shouldRead).as("Should read: Integer value promoted").isTrue(); shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, equal("long", LONG_BASE + INT_MIN_VALUE - 1), true) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertFalse("Should not read: Integer value promoted", shouldRead); + assertThat(shouldRead).as("Should not read: Integer value promoted").isFalse(); shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, equal("id", (long) (INT_MIN_VALUE + 1)), true) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertTrue("Should read: Long value truncated", shouldRead); + assertThat(shouldRead).as("Should read: Long value truncated").isTrue(); shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, equal("id", (long) (INT_MIN_VALUE - 1)), true) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertFalse("Should not read: Long value truncated", shouldRead); + assertThat(shouldRead).as("Should not read: Long value truncated").isFalse(); shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, equal("id", ((long) Integer.MAX_VALUE) + 1), true) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); - Assert.assertFalse("Should not read: Long value outside Integer range", shouldRead); + assertThat(shouldRead).as("Should not read: Long value outside Integer range").isFalse(); } } diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestCDHParquetStatistics.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestCDHParquetStatistics.java index ed400d28770d..51bea3220259 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/TestCDHParquetStatistics.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestCDHParquetStatistics.java @@ -18,12 +18,12 @@ */ package org.apache.iceberg.parquet; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import org.apache.parquet.column.statistics.Statistics; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; /** * Tests for Parquet 1.5.0-Stats which cannot be evaluated like later versions of Parquet stats. @@ -39,6 +39,6 @@ public void testCDHParquetStatistcs() { when(cdhBinaryColumnStats.getMaxBytes()).thenReturn(null); when(cdhBinaryColumnStats.getMinBytes()).thenReturn(null); when(cdhBinaryColumnStats.getNumNulls()).thenReturn(0L); - Assert.assertTrue(ParquetMetricsRowGroupFilter.minMaxUndefined(cdhBinaryColumnStats)); + assertThat(ParquetMetricsRowGroupFilter.minMaxUndefined(cdhBinaryColumnStats)).isTrue(); } } diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestDictionaryRowGroupFilter.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestDictionaryRowGroupFilter.java index eafa93af1c84..54e34cb6c339 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/TestDictionaryRowGroupFilter.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestDictionaryRowGroupFilter.java @@ -40,6 +40,7 @@ import static org.apache.iceberg.types.Types.NestedField.required; import static org.apache.parquet.column.ParquetProperties.WriterVersion.PARQUET_1_0; import static org.apache.parquet.column.ParquetProperties.WriterVersion.PARQUET_2_0; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; @@ -77,7 +78,6 @@ import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; import org.apache.parquet.hadoop.metadata.ColumnPath; import org.apache.parquet.schema.MessageType; -import org.junit.Assert; import org.junit.Assume; import org.junit.Before; import org.junit.Rule; @@ -171,7 +171,7 @@ public TestDictionaryRowGroupFilter(WriterVersion writerVersion) { @Before public void createInputFile() throws IOException { File parquetFile = temp.newFile(); - Assert.assertTrue(parquetFile.delete()); + assertThat(parquetFile.delete()).isTrue(); // build struct field schema org.apache.avro.Schema structSchema = AvroSchemaUtil.convert(_structFieldType); @@ -216,7 +216,7 @@ public void createInputFile() throws IOException { ParquetFileReader reader = ParquetFileReader.open(ParquetIO.file(inFile)); - Assert.assertEquals("Should create only one row group", 1, reader.getRowGroups().size()); + assertThat(reader.getRowGroups()).as("Should create only one row group").hasSize(1); rowGroupMetadata = reader.getRowGroups().get(0); parquetSchema = reader.getFileMetaData().getSchema(); dictionaryStore = reader.getNextDictionaryReader(); @@ -272,22 +272,22 @@ public void testAllNulls() { boolean shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, notNull("all_nulls")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: dictionary filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: dictionary filter doesn't help").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, notNull("some_nulls")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: dictionary filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: dictionary filter doesn't help").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, notNull("no_nulls")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: dictionary filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: dictionary filter doesn't help").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, notNull("struct_not_null")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: dictionary filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: dictionary filter doesn't help").isTrue(); } @Test @@ -295,22 +295,22 @@ public void testNoNulls() { boolean shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, isNull("all_nulls")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: dictionary filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: dictionary filter doesn't help").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, isNull("some_nulls")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: dictionary filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: dictionary filter doesn't help").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, isNull("no_nulls")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: dictionary filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: dictionary filter doesn't help").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, isNull("struct_not_null")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: dictionary filter doesn't help", shouldRead); + assertThat(shouldRead).as("Should read: dictionary filter doesn't help").isTrue(); } @Test @@ -318,12 +318,12 @@ public void testRequiredColumn() { boolean shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, notNull("required")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: required columns are always non-null", shouldRead); + assertThat(shouldRead).as("Should read: required columns are always non-null").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, isNull("required")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should skip: required columns are always non-null", shouldRead); + assertThat(shouldRead).as("Should skip: required columns are always non-null").isFalse(); } @Test @@ -331,17 +331,17 @@ public void testIsNaNs() { boolean shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, isNaN("all_nans")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: all_nans column will contain NaN", shouldRead); + assertThat(shouldRead).as("Should read: all_nans column will contain NaN").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, isNaN("some_nans")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: some_nans column will contain NaN", shouldRead); + assertThat(shouldRead).as("Should read: some_nans column will contain NaN").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, isNaN("no_nans")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should skip: no_nans column will not contain NaN", shouldRead); + assertThat(shouldRead).as("Should skip: no_nans column will not contain NaN").isFalse(); } @Test @@ -349,17 +349,17 @@ public void testNotNaNs() { boolean shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, notNaN("all_nans")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should skip: all_nans column will not contain non-NaN", shouldRead); + assertThat(shouldRead).as("Should skip: all_nans column will not contain non-NaN").isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, notNaN("some_nans")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: some_nans column will contain non-NaN", shouldRead); + assertThat(shouldRead).as("Should read: some_nans column will contain non-NaN").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, notNaN("no_nans")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: no_nans column will contain non-NaN", shouldRead); + assertThat(shouldRead).as("Should read: no_nans column will contain non-NaN").isTrue(); } @Test @@ -367,26 +367,30 @@ public void testNotNaNOnNaNsAndNulls() { boolean shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, isNull("_nans_and_nulls")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: _nans_and_nulls column will contain null values", shouldRead); + assertThat(shouldRead) + .as("Should read: _nans_and_nulls column will contain null values") + .isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, notNull("_nans_and_nulls")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue( - "Should read: _nans_and_nulls column will contain NaN values which are not null", - shouldRead); + assertThat(shouldRead) + .as("Should read: _nans_and_nulls column will contain NaN values which are not null") + .isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, isNaN("_nans_and_nulls")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: _nans_and_nulls column will contain NaN values", shouldRead); + assertThat(shouldRead) + .as("Should read: _nans_and_nulls column will contain NaN values") + .isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, notNaN("_nans_and_nulls")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue( - "Should read: _nans_and_nulls column will contain null values which are not NaN", - shouldRead); + assertThat(shouldRead) + .as("Should read: _nans_and_nulls column will contain null values which are not NaN") + .isTrue(); } @Test @@ -394,43 +398,43 @@ public void testStartsWith() { boolean shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, startsWith("non_dict", "re")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: no dictionary", shouldRead); + assertThat(shouldRead).as("Should read: no dictionary").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, startsWith("required", "re")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: dictionary contains a matching entry", shouldRead); + assertThat(shouldRead).as("Should read: dictionary contains a matching entry").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, startsWith("required", "req")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: dictionary contains a matching entry", shouldRead); + assertThat(shouldRead).as("Should read: dictionary contains a matching entry").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, startsWith("some_nulls", "so")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: dictionary contains a matching entry", shouldRead); + assertThat(shouldRead).as("Should read: dictionary contains a matching entry").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter( SCHEMA, startsWith("no_stats", UUID.randomUUID().toString())) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should skip: no stats but dictionary is present", shouldRead); + assertThat(shouldRead).as("Should skip: no stats but dictionary is present").isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, startsWith("required", "reqs")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should skip: no match in dictionary", shouldRead); + assertThat(shouldRead).as("Should skip: no match in dictionary").isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, startsWith("some_nulls", "somex")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should skip: no match in dictionary", shouldRead); + assertThat(shouldRead).as("Should skip: no match in dictionary").isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, startsWith("no_nulls", "xxx")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should skip: no match in dictionary", shouldRead); + assertThat(shouldRead).as("Should skip: no match in dictionary").isFalse(); } @Test @@ -438,48 +442,48 @@ public void testNotStartsWith() { boolean shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, notStartsWith("non_dict", "re")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: no dictionary", shouldRead); + assertThat(shouldRead).as("Should read: no dictionary").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, notStartsWith("required", "re")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should skip: no match in dictionary", shouldRead); + assertThat(shouldRead).as("Should skip: no match in dictionary").isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, notStartsWith("required", "req")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should skip: no match in dictionary", shouldRead); + assertThat(shouldRead).as("Should skip: no match in dictionary").isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, notStartsWith("some_nulls", "s!")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: dictionary contains a matching entry", shouldRead); + assertThat(shouldRead).as("Should read: dictionary contains a matching entry").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter( SCHEMA, notStartsWith("no_stats", UUID.randomUUID().toString())) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: no stats but dictionary is present", shouldRead); + assertThat(shouldRead).as("Should read: no stats but dictionary is present").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, notStartsWith("required", "reqs")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: dictionary contains a matching entry", shouldRead); + assertThat(shouldRead).as("Should read: dictionary contains a matching entry").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, notStartsWith("some_nulls", "somex")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: dictionary contains a matching entry", shouldRead); + assertThat(shouldRead).as("Should read: dictionary contains a matching entry").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, notStartsWith("some_nulls", "some")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should skip: no match in dictionary", shouldRead); + assertThat(shouldRead).as("Should skip: no match in dictionary").isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, notStartsWith("no_nulls", "xxx")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: dictionary contains a matching entry", shouldRead); + assertThat(shouldRead).as("Should read: dictionary contains a matching entry").isTrue(); } @Test @@ -507,7 +511,7 @@ public void testColumnNotInFile() { boolean shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, expr) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: dictionary cannot be found: " + expr, shouldRead); + assertThat(shouldRead).as("Should read: dictionary cannot be found: " + expr).isTrue(); } } @@ -524,7 +528,7 @@ public void testColumnFallbackOrNotDictionaryEncoded() { boolean shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, expr) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: dictionary cannot be found: " + expr, shouldRead); + assertThat(shouldRead).as("Should read: dictionary cannot be found: " + expr).isTrue(); } } @@ -533,7 +537,7 @@ public void testMissingStats() { boolean shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, equal("no_stats", "a")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should skip: stats are missing but dictionary is present", shouldRead); + assertThat(shouldRead).as("Should skip: stats are missing but dictionary is present").isFalse(); } @Test @@ -542,12 +546,12 @@ public void testNot() { boolean shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, not(lessThan("id", INT_MIN_VALUE - 25))) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: not(false)", shouldRead); + assertThat(shouldRead).as("Should read: not(false)").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, not(greaterThan("id", INT_MIN_VALUE - 25))) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should skip: not(true)", shouldRead); + assertThat(shouldRead).as("Should skip: not(true)").isFalse(); } @Test @@ -560,7 +564,7 @@ public void testAnd() { lessThan("id", INT_MIN_VALUE - 25), greaterThanOrEqual("id", INT_MIN_VALUE - 30))) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should skip: and(false, true)", shouldRead); + assertThat(shouldRead).as("Should skip: and(false, true)").isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter( @@ -569,14 +573,14 @@ public void testAnd() { lessThan("id", INT_MIN_VALUE - 25), greaterThanOrEqual("id", INT_MAX_VALUE + 1))) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should skip: and(false, false)", shouldRead); + assertThat(shouldRead).as("Should skip: and(false, false)").isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter( SCHEMA, and(greaterThan("id", INT_MIN_VALUE - 25), lessThanOrEqual("id", INT_MIN_VALUE))) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: and(true, true)", shouldRead); + assertThat(shouldRead).as("Should read: and(true, true)").isTrue(); } @Test @@ -587,7 +591,7 @@ public void testOr() { SCHEMA, or(lessThan("id", INT_MIN_VALUE - 25), greaterThanOrEqual("id", INT_MAX_VALUE + 1))) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should skip: or(false, false)", shouldRead); + assertThat(shouldRead).as("Should skip: or(false, false)").isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter( @@ -596,7 +600,7 @@ public void testOr() { lessThan("id", INT_MIN_VALUE - 25), greaterThanOrEqual("id", INT_MAX_VALUE - 19))) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: or(false, true)", shouldRead); + assertThat(shouldRead).as("Should read: or(false, true)").isTrue(); } @Test @@ -604,22 +608,24 @@ public void testIntegerLt() { boolean shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, lessThan("id", INT_MIN_VALUE - 25)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should not read: id range below lower bound (5 < 30)", shouldRead); + assertThat(shouldRead).as("Should not read: id range below lower bound (5 < 30)").isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, lessThan("id", INT_MIN_VALUE)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should not read: id range below lower bound (30 is not < 30)", shouldRead); + assertThat(shouldRead) + .as("Should not read: id range below lower bound (30 is not < 30)") + .isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, lessThan("id", INT_MIN_VALUE + 1)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: one possible id", shouldRead); + assertThat(shouldRead).as("Should read: one possible id").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, lessThan("id", INT_MAX_VALUE)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: may possible ids", shouldRead); + assertThat(shouldRead).as("Should read: may possible ids").isTrue(); } @Test @@ -627,22 +633,22 @@ public void testIntegerLtEq() { boolean shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, lessThanOrEqual("id", INT_MIN_VALUE - 25)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should not read: id range below lower bound (5 < 30)", shouldRead); + assertThat(shouldRead).as("Should not read: id range below lower bound (5 < 30)").isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, lessThanOrEqual("id", INT_MIN_VALUE - 1)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should not read: id range below lower bound (29 < 30)", shouldRead); + assertThat(shouldRead).as("Should not read: id range below lower bound (29 < 30)").isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, lessThanOrEqual("id", INT_MIN_VALUE)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: one possible id", shouldRead); + assertThat(shouldRead).as("Should read: one possible id").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, lessThanOrEqual("id", INT_MAX_VALUE)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: many possible ids", shouldRead); + assertThat(shouldRead).as("Should read: many possible ids").isTrue(); } @Test @@ -650,22 +656,24 @@ public void testIntegerGt() { boolean shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, greaterThan("id", INT_MAX_VALUE + 6)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should not read: id range above upper bound (85 < 79)", shouldRead); + assertThat(shouldRead).as("Should not read: id range above upper bound (85 < 79)").isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, greaterThan("id", INT_MAX_VALUE)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should not read: id range above upper bound (79 is not > 79)", shouldRead); + assertThat(shouldRead) + .as("Should not read: id range above upper bound (79 is not > 79)") + .isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, greaterThan("id", INT_MAX_VALUE - 1)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: one possible id", shouldRead); + assertThat(shouldRead).as("Should read: one possible id").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, greaterThan("id", INT_MAX_VALUE - 4)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: may possible ids", shouldRead); + assertThat(shouldRead).as("Should read: may possible ids").isTrue(); } @Test @@ -673,22 +681,22 @@ public void testIntegerGtEq() { boolean shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, greaterThanOrEqual("id", INT_MAX_VALUE + 6)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should not read: id range above upper bound (85 < 79)", shouldRead); + assertThat(shouldRead).as("Should not read: id range above upper bound (85 < 79)").isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, greaterThanOrEqual("id", INT_MAX_VALUE + 1)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should not read: id range above upper bound (80 > 79)", shouldRead); + assertThat(shouldRead).as("Should not read: id range above upper bound (80 > 79)").isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, greaterThanOrEqual("id", INT_MAX_VALUE)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: one possible id", shouldRead); + assertThat(shouldRead).as("Should read: one possible id").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, greaterThanOrEqual("id", INT_MAX_VALUE - 4)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: may possible ids", shouldRead); + assertThat(shouldRead).as("Should read: may possible ids").isTrue(); } @Test @@ -696,37 +704,37 @@ public void testIntegerEq() { boolean shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, equal("id", INT_MIN_VALUE - 25)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should not read: id below lower bound", shouldRead); + assertThat(shouldRead).as("Should not read: id below lower bound").isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, equal("id", INT_MIN_VALUE - 1)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should not read: id below lower bound", shouldRead); + assertThat(shouldRead).as("Should not read: id below lower bound").isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, equal("id", INT_MIN_VALUE)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: id equal to lower bound", shouldRead); + assertThat(shouldRead).as("Should read: id equal to lower bound").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, equal("id", INT_MAX_VALUE - 4)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: id between lower and upper bounds", shouldRead); + assertThat(shouldRead).as("Should read: id between lower and upper bounds").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, equal("id", INT_MAX_VALUE)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: id equal to upper bound", shouldRead); + assertThat(shouldRead).as("Should read: id equal to upper bound").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, equal("id", INT_MAX_VALUE + 1)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should not read: id above upper bound", shouldRead); + assertThat(shouldRead).as("Should not read: id above upper bound").isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, equal("id", INT_MAX_VALUE + 6)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should not read: id above upper bound", shouldRead); + assertThat(shouldRead).as("Should not read: id above upper bound").isFalse(); } @Test @@ -734,37 +742,37 @@ public void testIntegerNotEq() { boolean shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, notEqual("id", INT_MIN_VALUE - 25)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: id below lower bound", shouldRead); + assertThat(shouldRead).as("Should read: id below lower bound").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, notEqual("id", INT_MIN_VALUE - 1)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: id below lower bound", shouldRead); + assertThat(shouldRead).as("Should read: id below lower bound").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, notEqual("id", INT_MIN_VALUE)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: id equal to lower bound", shouldRead); + assertThat(shouldRead).as("Should read: id equal to lower bound").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, notEqual("id", INT_MAX_VALUE - 4)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: id between lower and upper bounds", shouldRead); + assertThat(shouldRead).as("Should read: id between lower and upper bounds").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, notEqual("id", INT_MAX_VALUE)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: id equal to upper bound", shouldRead); + assertThat(shouldRead).as("Should read: id equal to upper bound").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, notEqual("id", INT_MAX_VALUE + 1)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: id above upper bound", shouldRead); + assertThat(shouldRead).as("Should read: id above upper bound").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, notEqual("id", INT_MAX_VALUE + 6)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: id above upper bound", shouldRead); + assertThat(shouldRead).as("Should read: id above upper bound").isTrue(); } @Test @@ -772,37 +780,37 @@ public void testIntegerNotEqRewritten() { boolean shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, not(equal("id", INT_MIN_VALUE - 25))) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: id below lower bound", shouldRead); + assertThat(shouldRead).as("Should read: id below lower bound").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, not(equal("id", INT_MIN_VALUE - 1))) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: id below lower bound", shouldRead); + assertThat(shouldRead).as("Should read: id below lower bound").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, not(equal("id", INT_MIN_VALUE))) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: id equal to lower bound", shouldRead); + assertThat(shouldRead).as("Should read: id equal to lower bound").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, not(equal("id", INT_MAX_VALUE - 4))) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: id between lower and upper bounds", shouldRead); + assertThat(shouldRead).as("Should read: id between lower and upper bounds").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, not(equal("id", INT_MAX_VALUE))) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: id equal to upper bound", shouldRead); + assertThat(shouldRead).as("Should read: id equal to upper bound").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, not(equal("id", INT_MAX_VALUE + 1))) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: id above upper bound", shouldRead); + assertThat(shouldRead).as("Should read: id above upper bound").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, not(equal("id", INT_MAX_VALUE + 6))) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: id above upper bound", shouldRead); + assertThat(shouldRead).as("Should read: id above upper bound").isTrue(); } @Test @@ -810,12 +818,12 @@ public void testStringNotEq() { boolean shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, notEqual("some_nulls", "some")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: contains null != 'some'", shouldRead); + assertThat(shouldRead).as("Should read: contains null != 'some'").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, notEqual("no_nulls", "")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should skip: contains only ''", shouldRead); + assertThat(shouldRead).as("Should skip: contains only ''").isFalse(); } @Test @@ -824,25 +832,27 @@ public void testStructFieldLt() { new ParquetDictionaryRowGroupFilter( SCHEMA, lessThan("struct_not_null.int_field", INT_MIN_VALUE - 25)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should not read: id range below lower bound (5 < 30)", shouldRead); + assertThat(shouldRead).as("Should not read: id range below lower bound (5 < 30)").isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter( SCHEMA, lessThan("struct_not_null.int_field", INT_MIN_VALUE)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should not read: id range below lower bound (30 is not < 30)", shouldRead); + assertThat(shouldRead) + .as("Should not read: id range below lower bound (30 is not < 30)") + .isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter( SCHEMA, lessThan("struct_not_null.int_field", INT_MIN_VALUE + 1)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: one possible id", shouldRead); + assertThat(shouldRead).as("Should read: one possible id").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter( SCHEMA, lessThan("struct_not_null.int_field", INT_MAX_VALUE)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: may possible ids", shouldRead); + assertThat(shouldRead).as("Should read: may possible ids").isTrue(); } @Test @@ -851,25 +861,25 @@ public void testStructFieldLtEq() { new ParquetDictionaryRowGroupFilter( SCHEMA, lessThanOrEqual("struct_not_null.int_field", INT_MIN_VALUE - 25)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should not read: id range below lower bound (5 < 30)", shouldRead); + assertThat(shouldRead).as("Should not read: id range below lower bound (5 < 30)").isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter( SCHEMA, lessThanOrEqual("struct_not_null.int_field", INT_MIN_VALUE - 1)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should not read: id range below lower bound (29 < 30)", shouldRead); + assertThat(shouldRead).as("Should not read: id range below lower bound (29 < 30)").isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter( SCHEMA, lessThanOrEqual("struct_not_null.int_field", INT_MIN_VALUE)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: one possible id", shouldRead); + assertThat(shouldRead).as("Should read: one possible id").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter( SCHEMA, lessThanOrEqual("struct_not_null.int_field", INT_MAX_VALUE)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: many possible ids", shouldRead); + assertThat(shouldRead).as("Should read: many possible ids").isTrue(); } @Test @@ -878,25 +888,27 @@ public void testStructFieldGt() { new ParquetDictionaryRowGroupFilter( SCHEMA, greaterThan("struct_not_null.int_field", INT_MAX_VALUE + 6)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should not read: id range above upper bound (85 < 79)", shouldRead); + assertThat(shouldRead).as("Should not read: id range above upper bound (85 < 79)").isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter( SCHEMA, greaterThan("struct_not_null.int_field", INT_MAX_VALUE)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should not read: id range above upper bound (79 is not > 79)", shouldRead); + assertThat(shouldRead) + .as("Should not read: id range above upper bound (79 is not > 79)") + .isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter( SCHEMA, greaterThan("struct_not_null.int_field", INT_MAX_VALUE - 1)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: one possible id", shouldRead); + assertThat(shouldRead).as("Should read: one possible id").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter( SCHEMA, greaterThan("struct_not_null.int_field", INT_MAX_VALUE - 4)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: may possible ids", shouldRead); + assertThat(shouldRead).as("Should read: may possible ids").isTrue(); } @Test @@ -905,25 +917,25 @@ public void testStructFieldGtEq() { new ParquetDictionaryRowGroupFilter( SCHEMA, greaterThanOrEqual("struct_not_null.int_field", INT_MAX_VALUE + 6)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should not read: id range above upper bound (85 < 79)", shouldRead); + assertThat(shouldRead).as("Should not read: id range above upper bound (85 < 79)").isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter( SCHEMA, greaterThanOrEqual("struct_not_null.int_field", INT_MAX_VALUE + 1)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should not read: id range above upper bound (80 > 79)", shouldRead); + assertThat(shouldRead).as("Should not read: id range above upper bound (80 > 79)").isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter( SCHEMA, greaterThanOrEqual("struct_not_null.int_field", INT_MAX_VALUE)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: one possible id", shouldRead); + assertThat(shouldRead).as("Should read: one possible id").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter( SCHEMA, greaterThanOrEqual("struct_not_null.int_field", INT_MAX_VALUE - 4)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: may possible ids", shouldRead); + assertThat(shouldRead).as("Should read: may possible ids").isTrue(); } @Test @@ -932,43 +944,43 @@ public void testStructFieldEq() { new ParquetDictionaryRowGroupFilter( SCHEMA, equal("struct_not_null.int_field", INT_MIN_VALUE - 25)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should not read: id below lower bound", shouldRead); + assertThat(shouldRead).as("Should not read: id below lower bound").isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter( SCHEMA, equal("struct_not_null.int_field", INT_MIN_VALUE - 1)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should not read: id below lower bound", shouldRead); + assertThat(shouldRead).as("Should not read: id below lower bound").isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter( SCHEMA, equal("struct_not_null.int_field", INT_MIN_VALUE)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: id equal to lower bound", shouldRead); + assertThat(shouldRead).as("Should read: id equal to lower bound").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter( SCHEMA, equal("struct_not_null.int_field", INT_MAX_VALUE - 4)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: id between lower and upper bounds", shouldRead); + assertThat(shouldRead).as("Should read: id between lower and upper bounds").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter( SCHEMA, equal("struct_not_null.int_field", INT_MAX_VALUE)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: id equal to upper bound", shouldRead); + assertThat(shouldRead).as("Should read: id equal to upper bound").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter( SCHEMA, equal("struct_not_null.int_field", INT_MAX_VALUE + 1)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should not read: id above upper bound", shouldRead); + assertThat(shouldRead).as("Should not read: id above upper bound").isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter( SCHEMA, equal("struct_not_null.int_field", INT_MAX_VALUE + 6)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should not read: id above upper bound", shouldRead); + assertThat(shouldRead).as("Should not read: id above upper bound").isFalse(); } @Test @@ -977,42 +989,42 @@ public void testStructFieldNotEq() { new ParquetDictionaryRowGroupFilter( SCHEMA, notEqual("struct_not_null.int_field", INT_MIN_VALUE - 25)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: id below lower bound", shouldRead); + assertThat(shouldRead).as("Should read: id below lower bound").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter( SCHEMA, notEqual("struct_not_null.int_field", INT_MIN_VALUE - 1)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: id below lower bound", shouldRead); + assertThat(shouldRead).as("Should read: id below lower bound").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter( SCHEMA, notEqual("struct_not_null.int_field", INT_MIN_VALUE)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: id equal to lower bound", shouldRead); + assertThat(shouldRead).as("Should read: id equal to lower bound").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter( SCHEMA, notEqual("struct_not_null.int_field", INT_MAX_VALUE - 4)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: id between lower and upper bounds", shouldRead); + assertThat(shouldRead).as("Should read: id between lower and upper bounds").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter( SCHEMA, notEqual("struct_not_null.int_field", INT_MAX_VALUE)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: id equal to upper bound", shouldRead); + assertThat(shouldRead).as("Should read: id equal to upper bound").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, notEqual("id", INT_MAX_VALUE + 1)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: id above upper bound", shouldRead); + assertThat(shouldRead).as("Should read: id above upper bound").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter( SCHEMA, notEqual("struct_not_null.int_field", INT_MAX_VALUE + 6)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: id above upper bound", shouldRead); + assertThat(shouldRead).as("Should read: id above upper bound").isTrue(); } @Test @@ -1020,7 +1032,7 @@ public void testCaseInsensitive() { boolean shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, notEqual("no_Nulls", ""), false) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should skip: contains only ''", shouldRead); + assertThat(shouldRead).as("Should skip: contains only ''").isFalse(); } @Test @@ -1040,43 +1052,43 @@ public void testIntegerIn() { new ParquetDictionaryRowGroupFilter( SCHEMA, in("id", INT_MIN_VALUE - 25, INT_MIN_VALUE - 24)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse( - "Should not read: id below lower bound (5 < 30, 6 < 30). The two sets are disjoint.", - shouldRead); + assertThat(shouldRead) + .as("Should not read: id below lower bound (5 < 30, 6 < 30). The two sets are disjoint.") + .isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, in("id", INT_MIN_VALUE - 2, INT_MIN_VALUE - 1)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse( - "Should not read: id below lower bound (28 < 30, 29 < 30). The two sets are disjoint.", - shouldRead); + assertThat(shouldRead) + .as("Should not read: id below lower bound (28 < 30, 29 < 30). The two sets are disjoint.") + .isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, in("id", INT_MIN_VALUE - 1, INT_MIN_VALUE)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: id equal to lower bound (30 == 30)", shouldRead); + assertThat(shouldRead).as("Should read: id equal to lower bound (30 == 30)").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, in("id", INT_MAX_VALUE - 4, INT_MAX_VALUE - 3)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: in set is a subset of the dictionary", shouldRead); + assertThat(shouldRead).as("Should read: in set is a subset of the dictionary").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, in("id", INT_MAX_VALUE, INT_MAX_VALUE + 1)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: id equal to upper bound (79 == 79)", shouldRead); + assertThat(shouldRead).as("Should read: id equal to upper bound (79 == 79)").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, in("id", INT_MAX_VALUE + 1, INT_MAX_VALUE + 2)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should not read: id above upper bound (80 > 79, 81 > 79)", shouldRead); + assertThat(shouldRead).as("Should not read: id above upper bound (80 > 79, 81 > 79)").isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, in("id", INT_MAX_VALUE + 6, INT_MAX_VALUE + 7)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse( - "Should not read: id above upper bound (85 > 79, 86 > 79). The two sets are disjoint.", - shouldRead); + assertThat(shouldRead) + .as("Should not read: id above upper bound (85 > 79, 86 > 79). The two sets are disjoint.") + .isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter( @@ -1087,7 +1099,7 @@ SCHEMA, in("id", INT_MIN_VALUE - 25, INT_MIN_VALUE - 24)) .boxed() .collect(Collectors.toList()))) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: the dictionary is a subset of the in set", shouldRead); + assertThat(shouldRead).as("Should read: the dictionary is a subset of the in set").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter( @@ -1098,34 +1110,40 @@ SCHEMA, in("id", INT_MIN_VALUE - 25, INT_MIN_VALUE - 24)) .boxed() .collect(Collectors.toList()))) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: the dictionary is equal to the in set", shouldRead); + assertThat(shouldRead).as("Should read: the dictionary is equal to the in set").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, in("all_nulls", 1, 2)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: in on all nulls column (isFallback to be true) ", shouldRead); + assertThat(shouldRead) + .as("Should read: in on all nulls column (isFallback to be true) ") + .isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, in("some_nulls", "aaa", "some")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: in on some nulls column", shouldRead); + assertThat(shouldRead).as("Should read: in on some nulls column").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, in("some_nulls", "aaa", "bbb")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should not read: some_nulls values are not within the set", shouldRead); + assertThat(shouldRead) + .as("Should not read: some_nulls values are not within the set") + .isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, in("no_nulls", "aaa", "bbb")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse( - "Should not read: in on no nulls column (empty string is not within the set)", shouldRead); + assertThat(shouldRead) + .as("Should not read: in on no nulls column (empty string is not within the set)") + .isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, in("no_nulls", "aaa", "")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue( - "Should read: in on no nulls column (empty string is within the set)", shouldRead); + assertThat(shouldRead) + .as("Should read: in on no nulls column (empty string is within the set)") + .isTrue(); } @Test @@ -1134,49 +1152,49 @@ public void testIntegerNotIn() { new ParquetDictionaryRowGroupFilter( SCHEMA, notIn("id", INT_MIN_VALUE - 25, INT_MIN_VALUE - 24)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue( - "Should read: id below lower bound (5 < 30, 6 < 30). The two sets are disjoint.", - shouldRead); + assertThat(shouldRead) + .as("Should read: id below lower bound (5 < 30, 6 < 30). The two sets are disjoint.") + .isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter( SCHEMA, notIn("id", INT_MIN_VALUE - 2, INT_MIN_VALUE - 1)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue( - "Should read: id below lower bound (28 < 30, 29 < 30). The two sets are disjoint.", - shouldRead); + assertThat(shouldRead) + .as("Should read: id below lower bound (28 < 30, 29 < 30). The two sets are disjoint.") + .isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, notIn("id", INT_MIN_VALUE - 1, INT_MIN_VALUE)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: id equal to lower bound (30 == 30)", shouldRead); + assertThat(shouldRead).as("Should read: id equal to lower bound (30 == 30)").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter( SCHEMA, notIn("id", INT_MAX_VALUE - 4, INT_MAX_VALUE - 3)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: the notIn set is a subset of the dictionary", shouldRead); + assertThat(shouldRead).as("Should read: the notIn set is a subset of the dictionary").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, notIn("id", INT_MAX_VALUE, INT_MAX_VALUE + 1)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: id equal to upper bound (79 == 79)", shouldRead); + assertThat(shouldRead).as("Should read: id equal to upper bound (79 == 79)").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter( SCHEMA, notIn("id", INT_MAX_VALUE + 1, INT_MAX_VALUE + 2)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue( - "Should read: id above upper bound (80 > 79, 81 > 79). The two sets are disjoint.", - shouldRead); + assertThat(shouldRead) + .as("Should read: id above upper bound (80 > 79, 81 > 79). The two sets are disjoint.") + .isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter( SCHEMA, notIn("id", INT_MAX_VALUE + 6, INT_MAX_VALUE + 7)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue( - "Should read: id above upper bound (85 > 79, 86 > 79). The two sets are disjoint.", - shouldRead); + assertThat(shouldRead) + .as("Should read: id above upper bound (85 > 79, 86 > 79). The two sets are disjoint.") + .isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter( @@ -1187,7 +1205,9 @@ SCHEMA, notIn("id", INT_MAX_VALUE + 6, INT_MAX_VALUE + 7)) .boxed() .collect(Collectors.toList()))) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should not read: the dictionary is a subset of the notIn set", shouldRead); + assertThat(shouldRead) + .as("Should not read: the dictionary is a subset of the notIn set") + .isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter( @@ -1198,30 +1218,35 @@ SCHEMA, notIn("id", INT_MAX_VALUE + 6, INT_MAX_VALUE + 7)) .boxed() .collect(Collectors.toList()))) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse("Should not read: the dictionary is equal to the notIn set", shouldRead); + assertThat(shouldRead) + .as("Should not read: the dictionary is equal to the notIn set") + .isFalse(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, notIn("all_nulls", 1, 2)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should read: notIn on all nulls column", shouldRead); + assertThat(shouldRead).as("Should read: notIn on all nulls column").isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, notIn("some_nulls", "aaa", "bbb")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue( - "Should read: notIn on some nulls column (any null matches the notIn)", shouldRead); + assertThat(shouldRead) + .as("Should read: notIn on some nulls column (any null matches the notIn)") + .isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, notIn("no_nulls", "aaa", "bbb")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue( - "Should read: notIn on no nulls column (empty string is not within the set)", shouldRead); + assertThat(shouldRead) + .as("Should read: notIn on no nulls column (empty string is not within the set)") + .isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, notIn("no_nulls", "aaa", "")) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse( - "Should not read: notIn on no nulls column (empty string is within the set)", shouldRead); + assertThat(shouldRead) + .as("Should not read: notIn on no nulls column (empty string is within the set)") + .isFalse(); } @Test @@ -1230,7 +1255,7 @@ public void testTypePromotion() { boolean shouldRead = new ParquetDictionaryRowGroupFilter(promotedSchema, equal("id", INT_MIN_VALUE + 1), true) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue("Should succeed with promoted schema", shouldRead); + assertThat(shouldRead).as("Should succeed with promoted schema").isTrue(); } @Test @@ -1249,14 +1274,16 @@ public void testFixedLenByteArray() { new ParquetDictionaryRowGroupFilter( SCHEMA, greaterThanOrEqual("decimal_fixed", BigDecimal.ZERO)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertTrue( - "Should read: Half of the decimal_fixed values are greater than 0", shouldRead); + assertThat(shouldRead) + .as("Should read: Half of the decimal_fixed values are greater than 0") + .isTrue(); shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, lessThan("decimal_fixed", DECIMAL_MIN_VALUE)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - Assert.assertFalse( - "Should not read: No decimal_fixed values less than -1234567890.0987654321", shouldRead); + assertThat(shouldRead) + .as("Should not read: No decimal_fixed values less than -1234567890.0987654321") + .isFalse(); } private ColumnChunkMetaData getColumnForName(BlockMetaData rowGroup, String columnName) { diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java index 1762802fd527..b21e234a5d3a 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java @@ -25,17 +25,18 @@ import static org.apache.iceberg.parquet.ParquetWritingTestUtils.createTempFile; import static org.apache.iceberg.parquet.ParquetWritingTestUtils.write; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; +import java.nio.file.Path; import java.util.Collections; import java.util.List; import java.util.function.Function; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecordBuilder; -import org.apache.hadoop.fs.Path; import org.apache.iceberg.Files; import org.apache.iceberg.Schema; import org.apache.iceberg.avro.AvroSchemaUtil; @@ -49,14 +50,12 @@ import org.apache.parquet.hadoop.ParquetFileReader; import org.apache.parquet.hadoop.ParquetWriter; import org.apache.parquet.schema.MessageType; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestParquet { - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; @Test public void testRowGroupSizeConfigurable() throws IOException { @@ -68,7 +67,7 @@ public void testRowGroupSizeConfigurable() throws IOException { try (ParquetFileReader reader = ParquetFileReader.open(ParquetIO.file(localInput(parquetFile)))) { - Assert.assertEquals(2, reader.getRowGroups().size()); + assertThat(reader.getRowGroups()).hasSize(2); } } @@ -83,7 +82,7 @@ public void testRowGroupSizeConfigurableWithWriter() throws IOException { try (ParquetFileReader reader = ParquetFileReader.open(ParquetIO.file(localInput(parquetFile)))) { - Assert.assertEquals(2, reader.getRowGroups().size()); + assertThat(reader.getRowGroups()).hasSize(2); } } @@ -116,7 +115,7 @@ public void testNumberOfBytesWritten() throws IOException { records.toArray(new GenericData.Record[] {})); long expectedSize = ParquetIO.file(localInput(file)).getLength(); - Assert.assertEquals(expectedSize, actualSize); + assertThat(actualSize).isEqualTo(expectedSize); } @Test @@ -127,11 +126,11 @@ public void testTwoLevelList() throws IOException { optional(2, "topbytes", Types.BinaryType.get())); org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(schema.asStruct()); - File testFile = temp.newFile(); - Assert.assertTrue(testFile.delete()); + File testFile = temp.toFile(); + assertThat(testFile.delete()).isTrue(); ParquetWriter writer = - AvroParquetWriter.builder(new Path(testFile.toURI())) + AvroParquetWriter.builder(new org.apache.hadoop.fs.Path(testFile.toURI())) .withDataModel(GenericData.get()) .withSchema(avroSchema) .config("parquet.avro.add-list-element-records", "true") @@ -154,8 +153,8 @@ public void testTwoLevelList() throws IOException { Iterables.getOnlyElement( Parquet.read(Files.localInput(testFile)).project(schema).callInit().build()); - Assert.assertEquals(expectedByteList, recordRead.get("arraybytes")); - Assert.assertEquals(expectedBinary, recordRead.get("topbytes")); + assertThat(recordRead.get("arraybytes")).isEqualTo(expectedByteList); + assertThat(recordRead.get("topbytes")).isEqualTo(expectedBinary); } private Pair generateFile( diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetDataWriter.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetDataWriter.java index 728488329585..d0da4589f2c9 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetDataWriter.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetDataWriter.java @@ -18,8 +18,12 @@ */ package org.apache.iceberg.parquet; +import static org.apache.iceberg.parquet.ParquetWritingTestUtils.createTempFile; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; import java.nio.ByteBuffer; +import java.nio.file.Path; import java.util.List; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileContent; @@ -43,11 +47,9 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestParquetDataWriter { private static final Schema SCHEMA = @@ -58,9 +60,9 @@ public class TestParquetDataWriter { private List records; - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; - @Before + @BeforeEach public void createRecords() { GenericRecord record = GenericRecord.create(SCHEMA); @@ -76,7 +78,7 @@ public void createRecords() { @Test public void testDataWriter() throws IOException { - OutputFile file = Files.localOutput(temp.newFile()); + OutputFile file = Files.localOutput(createTempFile(temp)); SortOrder sortOrder = SortOrder.builderFor(SCHEMA).withOrderId(10).asc("id").build(); @@ -99,13 +101,14 @@ public void testDataWriter() throws IOException { DataFile dataFile = dataWriter.toDataFile(); - Assert.assertEquals("Format should be Parquet", FileFormat.PARQUET, dataFile.format()); - Assert.assertEquals("Should be data file", FileContent.DATA, dataFile.content()); - Assert.assertEquals("Record count should match", records.size(), dataFile.recordCount()); - Assert.assertEquals("Partition should be empty", 0, dataFile.partition().size()); - Assert.assertEquals( - "Sort order should match", sortOrder.orderId(), (int) dataFile.sortOrderId()); - Assert.assertNull("Key metadata should be null", dataFile.keyMetadata()); + assertThat(dataFile.format()).as("Format should be Parquet").isEqualTo(FileFormat.PARQUET); + assertThat(dataFile.content()).as("Should be data file").isEqualTo(FileContent.DATA); + assertThat(dataFile.recordCount()).as("Record count should match").isEqualTo(records.size()); + assertThat(dataFile.partition().size()).as("Partition should be empty").isEqualTo(0); + assertThat((int) dataFile.sortOrderId()) + .as("Sort order should match") + .isEqualTo(sortOrder.orderId()); + assertThat(dataFile.keyMetadata()).as("Key metadata should be null").isNull(); List writtenRecords; try (CloseableIterable reader = @@ -116,17 +119,17 @@ public void testDataWriter() throws IOException { writtenRecords = Lists.newArrayList(reader); } - Assert.assertEquals("Written records should match", records, writtenRecords); + assertThat(writtenRecords).as("Written records should match").isEqualTo(records); } @SuppressWarnings("checkstyle:AvoidEscapedUnicodeCharacters") @Test public void testInvalidUpperBoundString() throws Exception { - OutputFile file = Files.localOutput(temp.newFile()); + OutputFile file = Files.localOutput(createTempFile(temp)); Table testTable = TestTables.create( - temp.newFile(), + createTempFile(temp), "test_invalid_string_bound", SCHEMA, PartitionSpec.unpartitioned(), @@ -167,12 +170,13 @@ public void testInvalidUpperBoundString() throws Exception { DataFile dataFile = dataWriter.toDataFile(); - Assert.assertEquals("Format should be Parquet", FileFormat.PARQUET, dataFile.format()); - Assert.assertEquals("Should be data file", FileContent.DATA, dataFile.content()); - Assert.assertEquals( - "Record count should match", overflowRecords.size(), dataFile.recordCount()); - Assert.assertEquals("Partition should be empty", 0, dataFile.partition().size()); - Assert.assertNull("Key metadata should be null", dataFile.keyMetadata()); + assertThat(dataFile.format()).as("Format should be Parquet").isEqualTo(FileFormat.PARQUET); + assertThat(dataFile.content()).as("Should be data file").isEqualTo(FileContent.DATA); + assertThat(dataFile.recordCount()) + .as("Record count should match") + .isEqualTo(overflowRecords.size()); + assertThat(dataFile.partition().size()).as("Partition should be empty").isEqualTo(0); + assertThat(dataFile.keyMetadata()).as("Key metadata should be null").isNull(); List writtenRecords; try (CloseableIterable reader = @@ -183,22 +187,22 @@ public void testInvalidUpperBoundString() throws Exception { writtenRecords = Lists.newArrayList(reader); } - Assert.assertEquals("Written records should match", overflowRecords, writtenRecords); + assertThat(writtenRecords).as("Written records should match").isEqualTo(overflowRecords); - Assert.assertTrue("Should have a valid lower bound", dataFile.lowerBounds().containsKey(1)); - Assert.assertTrue("Should have a valid upper bound", dataFile.upperBounds().containsKey(1)); - Assert.assertTrue("Should have a valid lower bound", dataFile.lowerBounds().containsKey(2)); - Assert.assertFalse("Should have a null upper bound", dataFile.upperBounds().containsKey(2)); + assertThat(dataFile.lowerBounds()).as("Should have a valid lower bound").containsKey(1); + assertThat(dataFile.upperBounds()).as("Should have a valid upper bound").containsKey(1); + assertThat(dataFile.lowerBounds()).as("Should have a valid lower bound").containsKey(2); + assertThat(dataFile.upperBounds()).as("Should have a null upper bound").doesNotContainKey(2); } @SuppressWarnings("checkstyle:AvoidEscapedUnicodeCharacters") @Test public void testInvalidUpperBoundBinary() throws Exception { - OutputFile file = Files.localOutput(temp.newFile()); + OutputFile file = Files.localOutput(createTempFile(temp)); Table testTable = TestTables.create( - temp.newFile(), + createTempFile(temp), "test_invalid_binary_bound", SCHEMA, PartitionSpec.unpartitioned(), @@ -238,12 +242,13 @@ public void testInvalidUpperBoundBinary() throws Exception { DataFile dataFile = dataWriter.toDataFile(); - Assert.assertEquals("Format should be Parquet", FileFormat.PARQUET, dataFile.format()); - Assert.assertEquals("Should be data file", FileContent.DATA, dataFile.content()); - Assert.assertEquals( - "Record count should match", overflowRecords.size(), dataFile.recordCount()); - Assert.assertEquals("Partition should be empty", 0, dataFile.partition().size()); - Assert.assertNull("Key metadata should be null", dataFile.keyMetadata()); + assertThat(dataFile.format()).as("Format should be Parquet").isEqualTo(FileFormat.PARQUET); + assertThat(dataFile.content()).as("Should be data file").isEqualTo(FileContent.DATA); + assertThat(dataFile.recordCount()) + .as("Record count should match") + .isEqualTo(overflowRecords.size()); + assertThat(dataFile.partition().size()).as("Partition should be empty").isEqualTo(0); + assertThat(dataFile.keyMetadata()).as("Key metadata should be null").isNull(); List writtenRecords; try (CloseableIterable reader = @@ -254,11 +259,11 @@ public void testInvalidUpperBoundBinary() throws Exception { writtenRecords = Lists.newArrayList(reader); } - Assert.assertEquals("Written records should match", overflowRecords, writtenRecords); + assertThat(writtenRecords).as("Written records should match").isEqualTo(overflowRecords); - Assert.assertTrue("Should have a valid lower bound", dataFile.lowerBounds().containsKey(1)); - Assert.assertTrue("Should have a valid upper bound", dataFile.upperBounds().containsKey(1)); - Assert.assertTrue("Should have a valid lower bound", dataFile.lowerBounds().containsKey(3)); - Assert.assertFalse("Should have a null upper bound", dataFile.upperBounds().containsKey(3)); + assertThat(dataFile.lowerBounds()).as("Should have a valid lower bound").containsKey(1); + assertThat(dataFile.upperBounds()).as("Should have a valid upper bound").containsKey(1); + assertThat(dataFile.lowerBounds()).as("Should have a valid lower bound").containsKey(3); + assertThat(dataFile.upperBounds()).as("Should have a null upper bound").doesNotContainKey(3); } } diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetDeleteWriters.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetDeleteWriters.java index d70ca377be33..3f886c6dc771 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetDeleteWriters.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetDeleteWriters.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.parquet; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; import java.util.List; @@ -42,11 +44,9 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; import org.apache.iceberg.types.Types.NestedField; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestParquetDeleteWriters { private static final Schema SCHEMA = @@ -56,9 +56,9 @@ public class TestParquetDeleteWriters { private List records; - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private File temp; - @Before + @BeforeEach public void createDeleteRecords() { GenericRecord record = GenericRecord.create(SCHEMA); @@ -74,9 +74,7 @@ public void createDeleteRecords() { @Test public void testEqualityDeleteWriter() throws IOException { - File deleteFile = temp.newFile(); - - OutputFile out = Files.localOutput(deleteFile); + OutputFile out = Files.localOutput(temp); EqualityDeleteWriter deleteWriter = Parquet.writeDeletes(out) .createWriterFunc(GenericParquetWriter::buildWriter) @@ -91,12 +89,15 @@ public void testEqualityDeleteWriter() throws IOException { } DeleteFile metadata = deleteWriter.toDeleteFile(); - Assert.assertEquals("Format should be Parquet", FileFormat.PARQUET, metadata.format()); - Assert.assertEquals( - "Should be equality deletes", FileContent.EQUALITY_DELETES, metadata.content()); - Assert.assertEquals("Record count should be correct", records.size(), metadata.recordCount()); - Assert.assertEquals("Partition should be empty", 0, metadata.partition().size()); - Assert.assertNull("Key metadata should be null", metadata.keyMetadata()); + assertThat(metadata.format()).as("Format should be Parquet").isEqualTo(FileFormat.PARQUET); + assertThat(metadata.content()) + .as("Should be equality deletes") + .isEqualTo(FileContent.EQUALITY_DELETES); + assertThat(metadata.recordCount()) + .as("Record count should be correct") + .isEqualTo(records.size()); + assertThat(metadata.partition().size()).as("Partition should be empty").isEqualTo(0); + assertThat(metadata.keyMetadata()).as("Key metadata should be null").isNull(); List deletedRecords; try (CloseableIterable reader = @@ -107,13 +108,11 @@ public void testEqualityDeleteWriter() throws IOException { deletedRecords = Lists.newArrayList(reader); } - Assert.assertEquals("Deleted records should match expected", records, deletedRecords); + assertThat(deletedRecords).as("Deleted records should match expected").isEqualTo(records); } @Test public void testPositionDeleteWriter() throws IOException { - File deleteFile = temp.newFile(); - Schema deleteSchema = new Schema( MetadataColumns.DELETE_FILE_PATH, @@ -125,7 +124,7 @@ public void testPositionDeleteWriter() throws IOException { GenericRecord posDelete = GenericRecord.create(deleteSchema); List expectedDeleteRecords = Lists.newArrayList(); - OutputFile out = Files.localOutput(deleteFile); + OutputFile out = Files.localOutput(temp); PositionDeleteWriter deleteWriter = Parquet.writeDeletes(out) .createWriterFunc(GenericParquetWriter::buildWriter) @@ -147,12 +146,15 @@ public void testPositionDeleteWriter() throws IOException { } DeleteFile metadata = deleteWriter.toDeleteFile(); - Assert.assertEquals("Format should be Parquet", FileFormat.PARQUET, metadata.format()); - Assert.assertEquals( - "Should be position deletes", FileContent.POSITION_DELETES, metadata.content()); - Assert.assertEquals("Record count should be correct", records.size(), metadata.recordCount()); - Assert.assertEquals("Partition should be empty", 0, metadata.partition().size()); - Assert.assertNull("Key metadata should be null", metadata.keyMetadata()); + assertThat(metadata.format()).as("Format should be Parquet").isEqualTo(FileFormat.PARQUET); + assertThat(metadata.content()) + .as("Should be position deletes") + .isEqualTo(FileContent.POSITION_DELETES); + assertThat(metadata.recordCount()) + .as("Record count should be correct") + .isEqualTo(records.size()); + assertThat(metadata.partition().size()).as("Partition should be empty").isEqualTo(0); + assertThat(metadata.keyMetadata()).as("Key metadata should be null").isNull(); List deletedRecords; try (CloseableIterable reader = @@ -164,14 +166,13 @@ public void testPositionDeleteWriter() throws IOException { deletedRecords = Lists.newArrayList(reader); } - Assert.assertEquals( - "Deleted records should match expected", expectedDeleteRecords, deletedRecords); + assertThat(deletedRecords) + .as("Deleted records should match expected") + .isEqualTo(expectedDeleteRecords); } @Test public void testPositionDeleteWriterWithEmptyRow() throws IOException { - File deleteFile = temp.newFile(); - Schema deleteSchema = new Schema(MetadataColumns.DELETE_FILE_PATH, MetadataColumns.DELETE_FILE_POS); @@ -179,7 +180,7 @@ public void testPositionDeleteWriterWithEmptyRow() throws IOException { GenericRecord posDelete = GenericRecord.create(deleteSchema); List expectedDeleteRecords = Lists.newArrayList(); - OutputFile out = Files.localOutput(deleteFile); + OutputFile out = Files.localOutput(temp); PositionDeleteWriter deleteWriter = Parquet.writeDeletes(out) .createWriterFunc(GenericParquetWriter::buildWriter) @@ -202,12 +203,15 @@ public void testPositionDeleteWriterWithEmptyRow() throws IOException { } DeleteFile metadata = deleteWriter.toDeleteFile(); - Assert.assertEquals("Format should be Parquet", FileFormat.PARQUET, metadata.format()); - Assert.assertEquals( - "Should be position deletes", FileContent.POSITION_DELETES, metadata.content()); - Assert.assertEquals("Record count should be correct", records.size(), metadata.recordCount()); - Assert.assertEquals("Partition should be empty", 0, metadata.partition().size()); - Assert.assertNull("Key metadata should be null", metadata.keyMetadata()); + assertThat(metadata.format()).as("Format should be Parquet").isEqualTo(FileFormat.PARQUET); + assertThat(metadata.content()) + .as("Should be position deletes") + .isEqualTo(FileContent.POSITION_DELETES); + assertThat(metadata.recordCount()) + .as("Record count should be correct") + .isEqualTo(records.size()); + assertThat(metadata.partition().size()).as("Partition should be empty").isEqualTo(0); + assertThat(metadata.keyMetadata()).as("Key metadata should be null").isNull(); List deletedRecords; try (CloseableIterable reader = @@ -219,7 +223,8 @@ public void testPositionDeleteWriterWithEmptyRow() throws IOException { deletedRecords = Lists.newArrayList(reader); } - Assert.assertEquals( - "Deleted records should match expected", expectedDeleteRecords, deletedRecords); + assertThat(deletedRecords) + .as("Deleted records should match expected") + .isEqualTo(expectedDeleteRecords); } } diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetEncryption.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetEncryption.java index 32923c8424b2..c68a79c68977 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetEncryption.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetEncryption.java @@ -22,11 +22,13 @@ import static org.apache.iceberg.Files.localOutput; import static org.apache.iceberg.parquet.ParquetWritingTestUtils.createTempFile; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; import java.io.Closeable; import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; +import java.nio.file.Path; import java.security.SecureRandom; import java.util.List; import org.apache.avro.generic.GenericData; @@ -38,11 +40,9 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types.IntegerType; import org.apache.parquet.crypto.ParquetCryptoRuntimeException; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestParquetEncryption { @@ -53,9 +53,9 @@ public class TestParquetEncryption { private static File file; private static final Schema schema = new Schema(optional(1, columnName, IntegerType.get())); - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; - @Before + @BeforeEach public void writeEncryptedFile() throws IOException { List records = Lists.newArrayListWithCapacity(recordCount); org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(schema.asStruct()); @@ -120,7 +120,7 @@ public void testReadEncryptedFile() throws IOException { .iterator()) { for (int i = 1; i <= recordCount; i++) { GenericData.Record readRecord = (GenericData.Record) readRecords.next(); - Assert.assertEquals(i, readRecord.get(columnName)); + assertThat(readRecord.get(columnName)).isEqualTo(i); } } } diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetSchemaUtil.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetSchemaUtil.java index c669cf02a2be..84fbf2a7d989 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetSchemaUtil.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetSchemaUtil.java @@ -20,6 +20,7 @@ import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; import java.util.concurrent.atomic.AtomicInteger; import org.apache.iceberg.Schema; @@ -38,8 +39,7 @@ import org.apache.parquet.schema.Types.ListBuilder; import org.apache.parquet.schema.Types.MapBuilder; import org.apache.parquet.schema.Types.PrimitiveBuilder; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestParquetSchemaUtil { private static final Types.StructType SUPPORTED_PRIMITIVES = @@ -124,7 +124,7 @@ public void testAssignIdsByNameMapping() { MessageType messageTypeWithIdsFromNameMapping = ParquetSchemaUtil.applyNameMapping(RemoveIds.removeIds(messageTypeWithIds), nameMapping); - Assert.assertEquals(messageTypeWithIds, messageTypeWithIdsFromNameMapping); + assertThat(messageTypeWithIdsFromNameMapping).isEqualTo(messageTypeWithIds); } @Test @@ -259,7 +259,9 @@ public void testSchemaConversionWithoutAssigningIds() { 28, 29, Types.IntegerType.get(), Types.IntegerType.get()))); Schema actualSchema = ParquetSchemaUtil.convertAndPrune(messageType); - Assert.assertEquals("Schema must match", expectedSchema.asStruct(), actualSchema.asStruct()); + assertThat(actualSchema.asStruct()) + .as("Schema must match") + .isEqualTo(expectedSchema.asStruct()); } @Test @@ -286,7 +288,9 @@ public void testSchemaConversionForHiveStyleLists() { NameMapping nameMapping = MappingUtil.create(expectedSchema); MessageType messageTypeWithIds = ParquetSchemaUtil.applyNameMapping(messageType, nameMapping); Schema actualSchema = ParquetSchemaUtil.convertAndPrune(messageTypeWithIds); - Assert.assertEquals("Schema must match", expectedSchema.asStruct(), actualSchema.asStruct()); + assertThat(actualSchema.asStruct()) + .as("Schema must match") + .isEqualTo(expectedSchema.asStruct()); } @Test @@ -304,7 +308,9 @@ public void testLegacyTwoLevelListTypeWithPrimitiveElement() { optional(1, "arraybytes", Types.ListType.ofRequired(1000, Types.BinaryType.get()))); Schema actualSchema = ParquetSchemaUtil.convert(messageType); - Assert.assertEquals("Schema must match", expectedSchema.asStruct(), actualSchema.asStruct()); + assertThat(actualSchema.asStruct()) + .as("Schema must match") + .isEqualTo(expectedSchema.asStruct()); } @Test @@ -338,7 +344,9 @@ public void testLegacyTwoLevelListTypeWithGroupTypeElementWithTwoFields() { optional(1001, "f001", Types.LongType.get()))))))); Schema actualSchema = ParquetSchemaUtil.convert(parquetScehma); - Assert.assertEquals("Schema must match", expectedSchema.asStruct(), actualSchema.asStruct()); + assertThat(actualSchema.asStruct()) + .as("Schema must match") + .isEqualTo(expectedSchema.asStruct()); } @Test @@ -362,7 +370,9 @@ public void testLegacyTwoLevelListGenByParquetAvro() { 1001, Types.StructType.of(required(1000, "str", Types.StringType.get()))))); Schema actualSchema = ParquetSchemaUtil.convert(parquetScehma); - Assert.assertEquals("Schema must match", expectedSchema.asStruct(), actualSchema.asStruct()); + assertThat(actualSchema.asStruct()) + .as("Schema must match") + .isEqualTo(expectedSchema.asStruct()); } @Test @@ -386,7 +396,9 @@ public void testLegacyTwoLevelListGenByParquetThrift() { 1001, Types.StructType.of(required(1000, "str", Types.StringType.get()))))); Schema actualSchema = ParquetSchemaUtil.convert(parquetScehma); - Assert.assertEquals("Schema must match", expectedSchema.asStruct(), actualSchema.asStruct()); + assertThat(actualSchema.asStruct()) + .as("Schema must match") + .isEqualTo(expectedSchema.asStruct()); } @Test @@ -410,7 +422,9 @@ public void testLegacyTwoLevelListGenByParquetThrift1() { 1001, Types.ListType.ofRequired(1000, Types.IntegerType.get())))); Schema actualSchema = ParquetSchemaUtil.convert(parquetScehma); - Assert.assertEquals("Schema must match", expectedSchema.asStruct(), actualSchema.asStruct()); + assertThat(actualSchema.asStruct()) + .as("Schema must match") + .isEqualTo(expectedSchema.asStruct()); } private Type primitive( diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestPruneColumns.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestPruneColumns.java index 9e33ffa314ed..70345adf1b8b 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/TestPruneColumns.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestPruneColumns.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.parquet; +import static org.assertj.core.api.Assertions.assertThat; + import org.apache.iceberg.Schema; import org.apache.iceberg.types.Types.DoubleType; import org.apache.iceberg.types.Types.ListType; @@ -30,8 +32,7 @@ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; import org.apache.parquet.schema.Type; import org.apache.parquet.schema.Types; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestPruneColumns { @Test @@ -118,7 +119,7 @@ public void testMapKeyValueName() { .named("table"); MessageType actual = ParquetSchemaUtil.pruneColumns(fileSchema, projection); - Assert.assertEquals("Pruned schema should not rename repeated struct", expected, actual); + assertThat(actual).as("Pruned schema should not rename repeated struct").isEqualTo(expected); } @Test @@ -193,7 +194,7 @@ public void testListElementName() { .named("table"); MessageType actual = ParquetSchemaUtil.pruneColumns(fileSchema, projection); - Assert.assertEquals("Pruned schema should not rename repeated struct", expected, actual); + assertThat(actual).as("Pruned schema should not rename repeated struct").isEqualTo(expected); } @Test @@ -267,6 +268,6 @@ public void testStructElementName() { .named("table"); MessageType actual = ParquetSchemaUtil.pruneColumns(fileSchema, projection); - Assert.assertEquals("Pruned schema should be matched", expected, actual); + assertThat(actual).as("Pruned schema should be matched").isEqualTo(expected); } } diff --git a/python/.pre-commit-config.yaml b/python/.pre-commit-config.yaml index 02f947d5c208..3a58520011a4 100644 --- a/python/.pre-commit-config.yaml +++ b/python/.pre-commit-config.yaml @@ -38,7 +38,7 @@ repos: - id: isort args: [--settings-path=python/pyproject.toml] - repo: https://github.com/pre-commit/mirrors-mypy - rev: v1.2.0 + rev: v1.3.0 hooks: - id: mypy args: @@ -52,12 +52,12 @@ repos: - prettier@2.7.1 - prettier-plugin-toml@0.3.1 - repo: https://github.com/hadialqattan/pycln - rev: v2.1.3 + rev: v2.1.5 hooks: - id: pycln args: [--config=python/pyproject.toml] - repo: https://github.com/asottile/pyupgrade - rev: v3.3.2 + rev: v3.4.0 hooks: - id: pyupgrade args: [--py38-plus, --keep-runtime-typing] @@ -79,19 +79,19 @@ repos: hooks: - id: mdformat additional_dependencies: - - mdformat-black - - mdformat-config - - mdformat-beautysh - - mdformat-admon - - mdformat-mkdocs - - mdformat-frontmatter + - mdformat-black==0.1.1 + - mdformat-config==0.1.3 + - mdformat-beautysh==0.1.1 + - mdformat-admon==1.0.1 + - mdformat-mkdocs==1.0.1 + - mdformat-frontmatter==2.0.1 - repo: https://github.com/pycqa/pydocstyle rev: 6.3.0 hooks: - id: pydocstyle args: [ - "--ignore=D100,D102,D101,D103,D104,D105,D106,D107,D200,D202,D203,D205,D209,D212,D213,D400,D401,D403,D404,D405,D406,D407,D411,D413,D414,D415,D417", + "--ignore=D100,D102,D101,D103,D104,D106,D107,D203,D212,D213,D401,D404,D405,D406,D407,D411,D413,D415,D417", ] additional_dependencies: - tomli==2.0.1 diff --git a/python/Makefile b/python/Makefile index 444a3785bcc5..932c90dd775b 100644 --- a/python/Makefile +++ b/python/Makefile @@ -17,7 +17,7 @@ install: pip install poetry - poetry install -E pyarrow -E hive -E s3fs -E glue -E adlfs -E duckdb -E ray + poetry install -E pyarrow -E hive -E s3fs -E glue -E adlfs -E duckdb -E ray -E sql-postgres check-license: ./dev/check-license diff --git a/python/mkdocs/docs/api.md b/python/mkdocs/docs/api.md index ddd5ca180f73..d3b8fceee5c5 100644 --- a/python/mkdocs/docs/api.md +++ b/python/mkdocs/docs/api.md @@ -78,110 +78,15 @@ catalog.load_table(("nyc", "taxis")) # The tuple syntax can be used if the namespace or table contains a dot. ``` -This returns a `Table` that represents an Iceberg table: - -```python -Table( - identifier=('nyc', 'taxis'), - metadata_location='s3a://warehouse/wh/nyc.db/taxis/metadata/00002-6ea51ce3-62aa-4197-9cf8-43d07c3440ca.metadata.json', - metadata=TableMetadataV2( - location='s3a://warehouse/wh/nyc.db/taxis', - table_uuid=UUID('ebd5d172-2162-453d-b586-1cdce52c1116'), - last_updated_ms=1662633437826, - last_column_id=19, - schemas=[Schema( - NestedField(field_id=1, name='VendorID', field_type=LongType(), required=False), - NestedField(field_id=2, name='tpep_pickup_datetime', field_type=TimestamptzType(), required=False), - NestedField(field_id=3, name='tpep_dropoff_datetime', field_type=TimestamptzType(), required=False), - NestedField(field_id=4, name='passenger_count', field_type=DoubleType(), required=False), - NestedField(field_id=5, name='trip_distance', field_type=DoubleType(), required=False), - NestedField(field_id=6, name='RatecodeID', field_type=DoubleType(), required=False), - NestedField(field_id=7, name='store_and_fwd_flag', field_type=StringType(), required=False), - NestedField(field_id=8, name='PULocationID', field_type=LongType(), required=False), - NestedField(field_id=9, name='DOLocationID', field_type=LongType(), required=False), - NestedField(field_id=10, name='payment_type', field_type=LongType(), required=False), - NestedField(field_id=11, name='fare_amount', field_type=DoubleType(), required=False), - NestedField(field_id=12, name='extra', field_type=DoubleType(), required=False), - NestedField(field_id=13, name='mta_tax', field_type=DoubleType(), required=False), - NestedField(field_id=14, name='tip_amount', field_type=DoubleType(), required=False), - NestedField(field_id=15, name='tolls_amount', field_type=DoubleType(), required=False), - NestedField(field_id=16, name='improvement_surcharge', field_type=DoubleType(), required=False), - NestedField(field_id=17, name='total_amount', field_type=DoubleType(), required=False), - NestedField(field_id=18, name='congestion_surcharge', field_type=DoubleType(), required=False), - NestedField(field_id=19, name='airport_fee', field_type=DoubleType(), required=False) - ), - schema_id=0, - identifier_field_ids=[] - )], - current_schema_id=0, - partition_specs=[PartitionSpec(spec_id=0)], - default_spec_id=0, - last_partition_id=999, - properties={ - 'owner': 'root', - 'write.format.default': 'parquet' - }, - current_snapshot_id=8334458494559715805, - snapshots=[ - Snapshot( - snapshot_id=7910949481055846233, - parent_snapshot_id=None, - sequence_number=None, - timestamp_ms=1662489306555, - manifest_list='s3a://warehouse/wh/nyc.db/taxis/metadata/snap-7910949481055846233-1-3eb7a2e1-5b7a-4e76-a29a-3e29c176eea4.avro', - summary=Summary( - Operation.APPEND, - **{ - 'spark.app.id': 'local-1662489289173', - 'added-data-files': '1', - 'added-records': '2979431', - 'added-files-size': '46600777', - 'changed-partition-count': '1', - 'total-records': '2979431', - 'total-files-size': '46600777', - 'total-data-files': '1', - 'total-delete-files': '0', - 'total-position-deletes': '0', - 'total-equality-deletes': '0' - } - ), - schema_id=0 - ), - ], - snapshot_log=[ - SnapshotLogEntry( - snapshot_id='7910949481055846233', - timestamp_ms=1662489306555 - ) - ], - metadata_log=[ - MetadataLogEntry( - metadata_file='s3a://warehouse/wh/nyc.db/taxis/metadata/00000-b58341ba-6a63-4eea-9b2f-e85e47c7d09f.metadata.json', - timestamp_ms=1662489306555 - ) - ], - sort_orders=[SortOrder(order_id=0)], - default_sort_order_id=0, - refs={ - 'main': SnapshotRef( - snapshot_id=8334458494559715805, - snapshot_ref_type=SnapshotRefType.BRANCH, - min_snapshots_to_keep=None, - max_snapshot_age_ms=None, - max_ref_age_ms=None - ) - }, - format_version=2, - last_sequence_number=1 - ) -) -``` +This returns a `Table` that represents an Iceberg table that can be queried and altered. ### Directly from a metadata file To load a table directly from a metadata file (i.e., **without** using a catalog), you can use a `StaticTable` as follows: ```python +from pyiceberg.table import StaticTable + table = StaticTable.from_metadata( "s3a://warehouse/wh/nyc.db/taxis/metadata/00002-6ea51ce3-62aa-4197-9cf8-43d07c3440ca.metadata.json" ) @@ -241,52 +146,37 @@ catalog.create_table( ) ``` -Which returns a newly created table: +### Update table properties + +Set and remove properties through the `Transaction` API: ```python -Table( - identifier=('default', 'bids'), - metadata_location='/Users/fokkodriesprong/Desktop/docker-spark-iceberg/wh/bids//metadata/00000-c8cd93ab-f784-474d-a167-b1a86b05195f.metadata.json', - metadata=TableMetadataV2( - location='/Users/fokkodriesprong/Desktop/docker-spark-iceberg/wh/bids/', - table_uuid=UUID('38d4cb39-4945-4bf2-b374-984b5c4984d2'), - last_updated_ms=1661847562069, - last_column_id=4, - schemas=[ - Schema( - NestedField(field_id=1, name='datetime', field_type=TimestampType(), required=False), - NestedField(field_id=2, name='bid', field_type=DoubleType(), required=False), - NestedField(field_id=3, name='ask', field_type=DoubleType(), required=False), - NestedField(field_id=4, name='symbol', field_type=StringType(), required=False)), - schema_id=1, - identifier_field_ids=[]) - ], - current_schema_id=1, - partition_specs=[ - PartitionSpec( - PartitionField(source_id=1, field_id=1000, transform=DayTransform(), name='datetime_day'),)) - ], - default_spec_id=0, - last_partition_id=1000, - properties={}, - current_snapshot_id=None, - snapshots=[], - snapshot_log=[], - metadata_log=[], - sort_orders=[ - SortOrder(order_id=1, fields=[SortField(source_id=4, transform=IdentityTransform(), direction=SortDirection.ASC, null_order=NullOrder.NULLS_FIRST)]) - ], - default_sort_order_id=1, - refs={}, - format_version=2, - last_sequence_number=0 - ) -) +with table.transaction() as transaction: + transaction.set_properties(abc="def") + +assert table.properties == {"abc": "def"} + +with table.transaction() as transaction: + transaction.remove_properties("abc") + +assert table.properties == {} +``` + +Or, without a context manager: + +```python +table = table.transaction().set_properties(abc="def").commit_transaction() + +assert table.properties == {"abc": "def"} + +table = table.transaction().remove_properties("abc").commit_transaction() + +assert table.properties == {} ``` -## Query a table +## Query the data -To query a table, a table scan is needed. A table scan accepts a filter, columns and optionally a limit and a snapshot ID: +To query a table, a table scan is needed. A table scan accepts a filter, columns, optionally a limit and a snapshot ID: ```python from pyiceberg.catalog import load_catalog diff --git a/python/mkdocs/docs/configuration.md b/python/mkdocs/docs/configuration.md index 947b701e1b62..0510e4ff66ca 100644 --- a/python/mkdocs/docs/configuration.md +++ b/python/mkdocs/docs/configuration.md @@ -24,7 +24,7 @@ hide: # Catalogs -PyIceberg currently has native support for REST, Hive and Glue. +PyIceberg currently has native support for REST, SQL, Hive, Glue and DynamoDB. There are three ways to pass in configuration: @@ -60,13 +60,23 @@ For the FileIO there are several configuration options available: ### S3 -| Key | Example | Description | -| -------------------- | ------------------- | --------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | -| s3.endpoint | https://10.0.19.25/ | Configure an alternative endpoint of the S3 service for the FileIO to access. This could be used to use S3FileIO with any s3-compatible object storage service that has a different endpoint, or access a private S3 endpoint in a virtual private cloud. | -| s3.access-key-id | admin | Configure the static secret access key used to access the FileIO. | -| s3.secret-access-key | password | Configure the static session token used to access the FileIO. | -| s3.signer | bearer | Configure the signature version of the FileIO. | -| s3.region | us-west-2 | Sets the region of the bucket | +| Key | Example | Description | +| -------------------- | ------------------------ | --------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | +| s3.endpoint | https://10.0.19.25/ | Configure an alternative endpoint of the S3 service for the FileIO to access. This could be used to use S3FileIO with any s3-compatible object storage service that has a different endpoint, or access a private S3 endpoint in a virtual private cloud. | +| s3.access-key-id | admin | Configure the static secret access key used to access the FileIO. | +| s3.secret-access-key | password | Configure the static session token used to access the FileIO. | +| s3.signer | bearer | Configure the signature version of the FileIO. | +| s3.region | us-west-2 | Sets the region of the bucket | +| s3.proxy-uri | http://my.proxy.com:8080 | Configure the proxy server to be used by the FileIO. | + +### HDFS + +| Key | Example | Description | +| -------------------- | ------------------- | ------------------------------------------------ | +| hdfs.host | https://10.0.19.25/ | Configure the HDFS host to connect to | +| hdfs.port | 9000 | Configure the HDFS port to connect to. | +| hdfs.user | user | Configure the HDFS username used for connection. | +| hdfs.kerberos_ticket | kerberos_ticket | Configure the path to the Kerberos ticket cache. | ### Azure Data lake @@ -106,6 +116,18 @@ catalog: | rest.signing-region | us-east-1 | The region to use when SigV4 signing a request | | rest.signing-name | execute-api | The service signing name to use when SigV4 signing a request | +## SQL Catalog + +The SQL catalog requires a database for its backend. As of now, pyiceberg only supports PostgreSQL through psycopg2. +The database connection has to be configured using the `uri` property (see SQLAlchemy's [documentation for URL format](https://docs.sqlalchemy.org/en/20/core/engines.html#backend-specific-urls)): + +```yaml +catalog: + default: + type: sql + uri: postgresql+psycopg2://username:password@localhost/mydatabase +``` + ## Hive Catalog ```yaml @@ -119,13 +141,27 @@ catalog: ## Glue Catalog -If you want to use AWS Glue as the catalog, you can use the last two ways to configure the pyiceberg and refer +Your AWS credentials can be passed directly through the Python API. +Otherwise, please refer to [How to configure AWS credentials](https://docs.aws.amazon.com/cli/latest/userguide/cli-chap-configure.html) to set your AWS account credentials locally. +If you did not set up a default AWS profile, you can configure the `profile_name`. + +```yaml +catalog: + default: + type: glue + aws_access_key_id: + aws_secret_access_key: + aws_session_token: + region_name: +``` ```yaml catalog: default: type: glue + profile_name: + region_name: ``` ## DynamoDB Catalog diff --git a/python/mkdocs/docs/how-to-release.md b/python/mkdocs/docs/how-to-release.md index 4b5c6345a996..647799e51e9f 100644 --- a/python/mkdocs/docs/how-to-release.md +++ b/python/mkdocs/docs/how-to-release.md @@ -44,9 +44,10 @@ export LAST_COMMIT_ID=$(git rev-list ${GIT_TAG} 2> /dev/null | head -n 1) The `-s` option will sign the commit. If you don't have a key yet, you can find the instructions [here](http://www.apache.org/dev/openpgp.html#key-gen-generate-key). To install gpg on a M1 based Mac, a couple of additional steps are required: https://gist.github.com/phortuin/cf24b1cca3258720c71ad42977e1ba57 -Next step is to create a source distribution (`sdist`) which will generate a `.tar.gz` with all the source files. These files need to be uploaded to the Apache SVN. +Next step is to remove the `dist/` directory to make sure that we have a clean start. Create a source distribution (`sdist`) which will generate a `.tar.gz` with all the source files using `poetry build`. These files need to be uploaded to the Apache SVN. -``` +```sh +rm -rf dist/ poetry build ``` @@ -94,7 +95,7 @@ poetry build twine upload -s dist/* ``` -Finally step is to generate the email what send to the dev mail list: +Final step is to generate the email to the dev mail list: ```bash cat << EOF > release-announcement-email.txt diff --git a/python/mkdocs/docs/verify-release.md b/python/mkdocs/docs/verify-release.md index d21a2e42ccf3..5750d28c2d11 100644 --- a/python/mkdocs/docs/verify-release.md +++ b/python/mkdocs/docs/verify-release.md @@ -47,20 +47,20 @@ gpg --import KEYS Next, verify the `.asc` file. ```sh -gpg --verify pyiceberg-0.2.0rc0-py3-none-any.whl.asc pyiceberg-0.2.0rc0-py3-none-any.whl +gpg --verify pyiceberg-0.4.0-py3-none-any.whl.asc pyiceberg-0.4.0-py3-none-any.whl ``` ## Verifying checksums ```sh -shasum -a 512 pyiceberg-0.2.0rc0.tar.gz +shasum -a 512 --check pyiceberg-0.4.0-py3-none-any.whl.sha512 ``` ## Verifying License Documentation ```sh -tar xzf pyiceberg-0.2.0rc0.tar.gztar -cd pyiceberg-0.2.0 +tar xzf pyiceberg-0.4.0.tar.gz +cd pyiceberg-0.4.0 ``` Run RAT checks to validate license header: diff --git a/python/mkdocs/requirements.txt b/python/mkdocs/requirements.txt index 3d6fa4241ea5..aa80656decbb 100644 --- a/python/mkdocs/requirements.txt +++ b/python/mkdocs/requirements.txt @@ -15,14 +15,14 @@ # specific language governing permissions and limitations # under the License. -mkdocs==1.4.3 -griffe==0.29.0 +mkdocs==1.5.1 +griffe==0.32.3 jinja2==3.1.2 mkdocstrings==0.22.0 -mkdocstrings-python==1.1.0 +mkdocstrings-python==1.2.1 mkdocs-literate-nav==0.6.0 mkdocs-autorefs==0.4.1 mkdocs-gen-files==0.5.0 -mkdocs-material==9.1.15 +mkdocs-material==9.1.21 mkdocs-material-extensions==1.1.1 mkdocs-section-index==0.3.5 diff --git a/python/poetry.lock b/python/poetry.lock index 8da29fa20a9e..7c15c41c75ef 100644 --- a/python/poetry.lock +++ b/python/poetry.lock @@ -1,10 +1,9 @@ -# This file is automatically @generated by Poetry and should not be changed by hand. +# This file is automatically @generated by Poetry 1.5.1 and should not be changed by hand. [[package]] name = "adlfs" version = "2023.4.0" description = "Access Azure Datalake Gen1 with fsspec and dask" -category = "main" optional = true python-versions = ">=3.8" files = [ @@ -25,121 +24,119 @@ docs = ["furo", "myst-parser", "numpydoc", "sphinx"] [[package]] name = "aiobotocore" -version = "2.5.0" +version = "2.5.2" description = "Async client for aws services using botocore and aiohttp" -category = "main" optional = true python-versions = ">=3.7" files = [ - {file = "aiobotocore-2.5.0-py3-none-any.whl", hash = "sha256:9a2a022d7b78ec9a2af0de589916d2721cddbf96264401b78d7a73c1a1435f3b"}, - {file = "aiobotocore-2.5.0.tar.gz", hash = "sha256:6a5b397cddd4f81026aa91a14c7dd2650727425740a5af8ba75127ff663faf67"}, + {file = "aiobotocore-2.5.2-py3-none-any.whl", hash = "sha256:337429ffd3cc367532572d40be809a84c7b5335f3f8eca2f23e09dfaa9a9ef90"}, + {file = "aiobotocore-2.5.2.tar.gz", hash = "sha256:e7399f21570db1c287f1c0c814dd3475dfe1c8166722e2c77ce67f172cbcfa89"}, ] [package.dependencies] -aiohttp = ">=3.3.1" -aioitertools = ">=0.5.1" -botocore = ">=1.29.76,<1.29.77" -wrapt = ">=1.10.10" +aiohttp = ">=3.3.1,<4.0.0" +aioitertools = ">=0.5.1,<1.0.0" +botocore = ">=1.29.161,<1.29.162" +wrapt = ">=1.10.10,<2.0.0" [package.extras] -awscli = ["awscli (>=1.27.76,<1.27.77)"] -boto3 = ["boto3 (>=1.26.76,<1.26.77)"] +awscli = ["awscli (>=1.27.161,<1.27.162)"] +boto3 = ["boto3 (>=1.26.161,<1.26.162)"] [[package]] name = "aiohttp" -version = "3.8.4" +version = "3.8.5" description = "Async http client/server framework (asyncio)" -category = "main" optional = true python-versions = ">=3.6" files = [ - {file = "aiohttp-3.8.4-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:5ce45967538fb747370308d3145aa68a074bdecb4f3a300869590f725ced69c1"}, - {file = "aiohttp-3.8.4-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:b744c33b6f14ca26b7544e8d8aadff6b765a80ad6164fb1a430bbadd593dfb1a"}, - {file = "aiohttp-3.8.4-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:1a45865451439eb320784918617ba54b7a377e3501fb70402ab84d38c2cd891b"}, - {file = "aiohttp-3.8.4-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a86d42d7cba1cec432d47ab13b6637bee393a10f664c425ea7b305d1301ca1a3"}, - {file = "aiohttp-3.8.4-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:ee3c36df21b5714d49fc4580247947aa64bcbe2939d1b77b4c8dcb8f6c9faecc"}, - {file = "aiohttp-3.8.4-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:176a64b24c0935869d5bbc4c96e82f89f643bcdf08ec947701b9dbb3c956b7dd"}, - {file = "aiohttp-3.8.4-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c844fd628851c0bc309f3c801b3a3d58ce430b2ce5b359cd918a5a76d0b20cb5"}, - {file = "aiohttp-3.8.4-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:5393fb786a9e23e4799fec788e7e735de18052f83682ce2dfcabaf1c00c2c08e"}, - {file = "aiohttp-3.8.4-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:e4b09863aae0dc965c3ef36500d891a3ff495a2ea9ae9171e4519963c12ceefd"}, - {file = "aiohttp-3.8.4-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:adfbc22e87365a6e564c804c58fc44ff7727deea782d175c33602737b7feadb6"}, - {file = "aiohttp-3.8.4-cp310-cp310-musllinux_1_1_ppc64le.whl", hash = "sha256:147ae376f14b55f4f3c2b118b95be50a369b89b38a971e80a17c3fd623f280c9"}, - {file = "aiohttp-3.8.4-cp310-cp310-musllinux_1_1_s390x.whl", hash = "sha256:eafb3e874816ebe2a92f5e155f17260034c8c341dad1df25672fb710627c6949"}, - {file = "aiohttp-3.8.4-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:c6cc15d58053c76eacac5fa9152d7d84b8d67b3fde92709195cb984cfb3475ea"}, - {file = "aiohttp-3.8.4-cp310-cp310-win32.whl", hash = "sha256:59f029a5f6e2d679296db7bee982bb3d20c088e52a2977e3175faf31d6fb75d1"}, - {file = "aiohttp-3.8.4-cp310-cp310-win_amd64.whl", hash = "sha256:fe7ba4a51f33ab275515f66b0a236bcde4fb5561498fe8f898d4e549b2e4509f"}, - {file = "aiohttp-3.8.4-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:3d8ef1a630519a26d6760bc695842579cb09e373c5f227a21b67dc3eb16cfea4"}, - {file = "aiohttp-3.8.4-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:5b3f2e06a512e94722886c0827bee9807c86a9f698fac6b3aee841fab49bbfb4"}, - {file = "aiohttp-3.8.4-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:3a80464982d41b1fbfe3154e440ba4904b71c1a53e9cd584098cd41efdb188ef"}, - {file = "aiohttp-3.8.4-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8b631e26df63e52f7cce0cce6507b7a7f1bc9b0c501fcde69742130b32e8782f"}, - {file = "aiohttp-3.8.4-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:3f43255086fe25e36fd5ed8f2ee47477408a73ef00e804cb2b5cba4bf2ac7f5e"}, - {file = "aiohttp-3.8.4-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:4d347a172f866cd1d93126d9b239fcbe682acb39b48ee0873c73c933dd23bd0f"}, - {file = "aiohttp-3.8.4-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a3fec6a4cb5551721cdd70473eb009d90935b4063acc5f40905d40ecfea23e05"}, - {file = "aiohttp-3.8.4-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:80a37fe8f7c1e6ce8f2d9c411676e4bc633a8462844e38f46156d07a7d401654"}, - {file = "aiohttp-3.8.4-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:d1e6a862b76f34395a985b3cd39a0d949ca80a70b6ebdea37d3ab39ceea6698a"}, - {file = "aiohttp-3.8.4-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:cd468460eefef601ece4428d3cf4562459157c0f6523db89365202c31b6daebb"}, - {file = "aiohttp-3.8.4-cp311-cp311-musllinux_1_1_ppc64le.whl", hash = "sha256:618c901dd3aad4ace71dfa0f5e82e88b46ef57e3239fc7027773cb6d4ed53531"}, - {file = "aiohttp-3.8.4-cp311-cp311-musllinux_1_1_s390x.whl", hash = "sha256:652b1bff4f15f6287550b4670546a2947f2a4575b6c6dff7760eafb22eacbf0b"}, - {file = "aiohttp-3.8.4-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:80575ba9377c5171407a06d0196b2310b679dc752d02a1fcaa2bc20b235dbf24"}, - {file = "aiohttp-3.8.4-cp311-cp311-win32.whl", hash = "sha256:bbcf1a76cf6f6dacf2c7f4d2ebd411438c275faa1dc0c68e46eb84eebd05dd7d"}, - {file = "aiohttp-3.8.4-cp311-cp311-win_amd64.whl", hash = "sha256:6e74dd54f7239fcffe07913ff8b964e28b712f09846e20de78676ce2a3dc0bfc"}, - {file = "aiohttp-3.8.4-cp36-cp36m-macosx_10_9_x86_64.whl", hash = "sha256:880e15bb6dad90549b43f796b391cfffd7af373f4646784795e20d92606b7a51"}, - {file = "aiohttp-3.8.4-cp36-cp36m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:bb96fa6b56bb536c42d6a4a87dfca570ff8e52de2d63cabebfd6fb67049c34b6"}, - {file = "aiohttp-3.8.4-cp36-cp36m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:4a6cadebe132e90cefa77e45f2d2f1a4b2ce5c6b1bfc1656c1ddafcfe4ba8131"}, - {file = "aiohttp-3.8.4-cp36-cp36m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:f352b62b45dff37b55ddd7b9c0c8672c4dd2eb9c0f9c11d395075a84e2c40f75"}, - {file = "aiohttp-3.8.4-cp36-cp36m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7ab43061a0c81198d88f39aaf90dae9a7744620978f7ef3e3708339b8ed2ef01"}, - {file = "aiohttp-3.8.4-cp36-cp36m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c9cb1565a7ad52e096a6988e2ee0397f72fe056dadf75d17fa6b5aebaea05622"}, - {file = "aiohttp-3.8.4-cp36-cp36m-musllinux_1_1_aarch64.whl", hash = "sha256:1b3ea7edd2d24538959c1c1abf97c744d879d4e541d38305f9bd7d9b10c9ec41"}, - {file = "aiohttp-3.8.4-cp36-cp36m-musllinux_1_1_i686.whl", hash = "sha256:7c7837fe8037e96b6dd5cfcf47263c1620a9d332a87ec06a6ca4564e56bd0f36"}, - {file = "aiohttp-3.8.4-cp36-cp36m-musllinux_1_1_ppc64le.whl", hash = "sha256:3b90467ebc3d9fa5b0f9b6489dfb2c304a1db7b9946fa92aa76a831b9d587e99"}, - {file = "aiohttp-3.8.4-cp36-cp36m-musllinux_1_1_s390x.whl", hash = "sha256:cab9401de3ea52b4b4c6971db5fb5c999bd4260898af972bf23de1c6b5dd9d71"}, - {file = "aiohttp-3.8.4-cp36-cp36m-musllinux_1_1_x86_64.whl", hash = "sha256:d1f9282c5f2b5e241034a009779e7b2a1aa045f667ff521e7948ea9b56e0c5ff"}, - {file = "aiohttp-3.8.4-cp36-cp36m-win32.whl", hash = "sha256:5e14f25765a578a0a634d5f0cd1e2c3f53964553a00347998dfdf96b8137f777"}, - {file = "aiohttp-3.8.4-cp36-cp36m-win_amd64.whl", hash = "sha256:4c745b109057e7e5f1848c689ee4fb3a016c8d4d92da52b312f8a509f83aa05e"}, - {file = "aiohttp-3.8.4-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:aede4df4eeb926c8fa70de46c340a1bc2c6079e1c40ccf7b0eae1313ffd33519"}, - {file = "aiohttp-3.8.4-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4ddaae3f3d32fc2cb4c53fab020b69a05c8ab1f02e0e59665c6f7a0d3a5be54f"}, - {file = "aiohttp-3.8.4-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:c4eb3b82ca349cf6fadcdc7abcc8b3a50ab74a62e9113ab7a8ebc268aad35bb9"}, - {file = "aiohttp-3.8.4-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:9bcb89336efa095ea21b30f9e686763f2be4478f1b0a616969551982c4ee4c3b"}, - {file = "aiohttp-3.8.4-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6c08e8ed6fa3d477e501ec9db169bfac8140e830aa372d77e4a43084d8dd91ab"}, - {file = "aiohttp-3.8.4-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c6cd05ea06daca6ad6a4ca3ba7fe7dc5b5de063ff4daec6170ec0f9979f6c332"}, - {file = "aiohttp-3.8.4-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:b7a00a9ed8d6e725b55ef98b1b35c88013245f35f68b1b12c5cd4100dddac333"}, - {file = "aiohttp-3.8.4-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:de04b491d0e5007ee1b63a309956eaed959a49f5bb4e84b26c8f5d49de140fa9"}, - {file = "aiohttp-3.8.4-cp37-cp37m-musllinux_1_1_ppc64le.whl", hash = "sha256:40653609b3bf50611356e6b6554e3a331f6879fa7116f3959b20e3528783e699"}, - {file = "aiohttp-3.8.4-cp37-cp37m-musllinux_1_1_s390x.whl", hash = "sha256:dbf3a08a06b3f433013c143ebd72c15cac33d2914b8ea4bea7ac2c23578815d6"}, - {file = "aiohttp-3.8.4-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:854f422ac44af92bfe172d8e73229c270dc09b96535e8a548f99c84f82dde241"}, - {file = "aiohttp-3.8.4-cp37-cp37m-win32.whl", hash = "sha256:aeb29c84bb53a84b1a81c6c09d24cf33bb8432cc5c39979021cc0f98c1292a1a"}, - {file = "aiohttp-3.8.4-cp37-cp37m-win_amd64.whl", hash = "sha256:db3fc6120bce9f446d13b1b834ea5b15341ca9ff3f335e4a951a6ead31105480"}, - {file = "aiohttp-3.8.4-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:fabb87dd8850ef0f7fe2b366d44b77d7e6fa2ea87861ab3844da99291e81e60f"}, - {file = "aiohttp-3.8.4-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:91f6d540163f90bbaef9387e65f18f73ffd7c79f5225ac3d3f61df7b0d01ad15"}, - {file = "aiohttp-3.8.4-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:d265f09a75a79a788237d7f9054f929ced2e69eb0bb79de3798c468d8a90f945"}, - {file = "aiohttp-3.8.4-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3d89efa095ca7d442a6d0cbc755f9e08190ba40069b235c9886a8763b03785da"}, - {file = "aiohttp-3.8.4-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:4dac314662f4e2aa5009977b652d9b8db7121b46c38f2073bfeed9f4049732cd"}, - {file = "aiohttp-3.8.4-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:fe11310ae1e4cd560035598c3f29d86cef39a83d244c7466f95c27ae04850f10"}, - {file = "aiohttp-3.8.4-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6ddb2a2026c3f6a68c3998a6c47ab6795e4127315d2e35a09997da21865757f8"}, - {file = "aiohttp-3.8.4-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e75b89ac3bd27d2d043b234aa7b734c38ba1b0e43f07787130a0ecac1e12228a"}, - {file = "aiohttp-3.8.4-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:6e601588f2b502c93c30cd5a45bfc665faaf37bbe835b7cfd461753068232074"}, - {file = "aiohttp-3.8.4-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:a5d794d1ae64e7753e405ba58e08fcfa73e3fad93ef9b7e31112ef3c9a0efb52"}, - {file = "aiohttp-3.8.4-cp38-cp38-musllinux_1_1_ppc64le.whl", hash = "sha256:a1f4689c9a1462f3df0a1f7e797791cd6b124ddbee2b570d34e7f38ade0e2c71"}, - {file = "aiohttp-3.8.4-cp38-cp38-musllinux_1_1_s390x.whl", hash = "sha256:3032dcb1c35bc330134a5b8a5d4f68c1a87252dfc6e1262c65a7e30e62298275"}, - {file = "aiohttp-3.8.4-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:8189c56eb0ddbb95bfadb8f60ea1b22fcfa659396ea36f6adcc521213cd7b44d"}, - {file = "aiohttp-3.8.4-cp38-cp38-win32.whl", hash = "sha256:33587f26dcee66efb2fff3c177547bd0449ab7edf1b73a7f5dea1e38609a0c54"}, - {file = "aiohttp-3.8.4-cp38-cp38-win_amd64.whl", hash = "sha256:e595432ac259af2d4630008bf638873d69346372d38255774c0e286951e8b79f"}, - {file = "aiohttp-3.8.4-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:5a7bdf9e57126dc345b683c3632e8ba317c31d2a41acd5800c10640387d193ed"}, - {file = "aiohttp-3.8.4-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:22f6eab15b6db242499a16de87939a342f5a950ad0abaf1532038e2ce7d31567"}, - {file = "aiohttp-3.8.4-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:7235604476a76ef249bd64cb8274ed24ccf6995c4a8b51a237005ee7a57e8643"}, - {file = "aiohttp-3.8.4-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ea9eb976ffdd79d0e893869cfe179a8f60f152d42cb64622fca418cd9b18dc2a"}, - {file = "aiohttp-3.8.4-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:92c0cea74a2a81c4c76b62ea1cac163ecb20fb3ba3a75c909b9fa71b4ad493cf"}, - {file = "aiohttp-3.8.4-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:493f5bc2f8307286b7799c6d899d388bbaa7dfa6c4caf4f97ef7521b9cb13719"}, - {file = "aiohttp-3.8.4-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0a63f03189a6fa7c900226e3ef5ba4d3bd047e18f445e69adbd65af433add5a2"}, - {file = "aiohttp-3.8.4-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:10c8cefcff98fd9168cdd86c4da8b84baaa90bf2da2269c6161984e6737bf23e"}, - {file = "aiohttp-3.8.4-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:bca5f24726e2919de94f047739d0a4fc01372801a3672708260546aa2601bf57"}, - {file = "aiohttp-3.8.4-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:03baa76b730e4e15a45f81dfe29a8d910314143414e528737f8589ec60cf7391"}, - {file = "aiohttp-3.8.4-cp39-cp39-musllinux_1_1_ppc64le.whl", hash = "sha256:8c29c77cc57e40f84acef9bfb904373a4e89a4e8b74e71aa8075c021ec9078c2"}, - {file = "aiohttp-3.8.4-cp39-cp39-musllinux_1_1_s390x.whl", hash = "sha256:03543dcf98a6619254b409be2d22b51f21ec66272be4ebda7b04e6412e4b2e14"}, - {file = "aiohttp-3.8.4-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:17b79c2963db82086229012cff93ea55196ed31f6493bb1ccd2c62f1724324e4"}, - {file = "aiohttp-3.8.4-cp39-cp39-win32.whl", hash = "sha256:34ce9f93a4a68d1272d26030655dd1b58ff727b3ed2a33d80ec433561b03d67a"}, - {file = "aiohttp-3.8.4-cp39-cp39-win_amd64.whl", hash = "sha256:41a86a69bb63bb2fc3dc9ad5ea9f10f1c9c8e282b471931be0268ddd09430b04"}, - {file = "aiohttp-3.8.4.tar.gz", hash = "sha256:bf2e1a9162c1e441bf805a1fd166e249d574ca04e03b34f97e2928769e91ab5c"}, + {file = "aiohttp-3.8.5-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:a94159871304770da4dd371f4291b20cac04e8c94f11bdea1c3478e557fbe0d8"}, + {file = "aiohttp-3.8.5-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:13bf85afc99ce6f9ee3567b04501f18f9f8dbbb2ea11ed1a2e079670403a7c84"}, + {file = "aiohttp-3.8.5-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:2ce2ac5708501afc4847221a521f7e4b245abf5178cf5ddae9d5b3856ddb2f3a"}, + {file = "aiohttp-3.8.5-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:96943e5dcc37a6529d18766597c491798b7eb7a61d48878611298afc1fca946c"}, + {file = "aiohttp-3.8.5-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:2ad5c3c4590bb3cc28b4382f031f3783f25ec223557124c68754a2231d989e2b"}, + {file = "aiohttp-3.8.5-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:0c413c633d0512df4dc7fd2373ec06cc6a815b7b6d6c2f208ada7e9e93a5061d"}, + {file = "aiohttp-3.8.5-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:df72ac063b97837a80d80dec8d54c241af059cc9bb42c4de68bd5b61ceb37caa"}, + {file = "aiohttp-3.8.5-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c48c5c0271149cfe467c0ff8eb941279fd6e3f65c9a388c984e0e6cf57538e14"}, + {file = "aiohttp-3.8.5-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:368a42363c4d70ab52c2c6420a57f190ed3dfaca6a1b19afda8165ee16416a82"}, + {file = "aiohttp-3.8.5-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:7607ec3ce4993464368505888af5beb446845a014bc676d349efec0e05085905"}, + {file = "aiohttp-3.8.5-cp310-cp310-musllinux_1_1_ppc64le.whl", hash = "sha256:0d21c684808288a98914e5aaf2a7c6a3179d4df11d249799c32d1808e79503b5"}, + {file = "aiohttp-3.8.5-cp310-cp310-musllinux_1_1_s390x.whl", hash = "sha256:312fcfbacc7880a8da0ae8b6abc6cc7d752e9caa0051a53d217a650b25e9a691"}, + {file = "aiohttp-3.8.5-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:ad093e823df03bb3fd37e7dec9d4670c34f9e24aeace76808fc20a507cace825"}, + {file = "aiohttp-3.8.5-cp310-cp310-win32.whl", hash = "sha256:33279701c04351a2914e1100b62b2a7fdb9a25995c4a104259f9a5ead7ed4802"}, + {file = "aiohttp-3.8.5-cp310-cp310-win_amd64.whl", hash = "sha256:6e4a280e4b975a2e7745573e3fc9c9ba0d1194a3738ce1cbaa80626cc9b4f4df"}, + {file = "aiohttp-3.8.5-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:ae871a964e1987a943d83d6709d20ec6103ca1eaf52f7e0d36ee1b5bebb8b9b9"}, + {file = "aiohttp-3.8.5-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:461908b2578955045efde733719d62f2b649c404189a09a632d245b445c9c975"}, + {file = "aiohttp-3.8.5-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:72a860c215e26192379f57cae5ab12b168b75db8271f111019509a1196dfc780"}, + {file = "aiohttp-3.8.5-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:cc14be025665dba6202b6a71cfcdb53210cc498e50068bc088076624471f8bb9"}, + {file = "aiohttp-3.8.5-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:8af740fc2711ad85f1a5c034a435782fbd5b5f8314c9a3ef071424a8158d7f6b"}, + {file = "aiohttp-3.8.5-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:841cd8233cbd2111a0ef0a522ce016357c5e3aff8a8ce92bcfa14cef890d698f"}, + {file = "aiohttp-3.8.5-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:5ed1c46fb119f1b59304b5ec89f834f07124cd23ae5b74288e364477641060ff"}, + {file = "aiohttp-3.8.5-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:84f8ae3e09a34f35c18fa57f015cc394bd1389bce02503fb30c394d04ee6b938"}, + {file = "aiohttp-3.8.5-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:62360cb771707cb70a6fd114b9871d20d7dd2163a0feafe43fd115cfe4fe845e"}, + {file = "aiohttp-3.8.5-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:23fb25a9f0a1ca1f24c0a371523546366bb642397c94ab45ad3aedf2941cec6a"}, + {file = "aiohttp-3.8.5-cp311-cp311-musllinux_1_1_ppc64le.whl", hash = "sha256:b0ba0d15164eae3d878260d4c4df859bbdc6466e9e6689c344a13334f988bb53"}, + {file = "aiohttp-3.8.5-cp311-cp311-musllinux_1_1_s390x.whl", hash = "sha256:5d20003b635fc6ae3f96d7260281dfaf1894fc3aa24d1888a9b2628e97c241e5"}, + {file = "aiohttp-3.8.5-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:0175d745d9e85c40dcc51c8f88c74bfbaef9e7afeeeb9d03c37977270303064c"}, + {file = "aiohttp-3.8.5-cp311-cp311-win32.whl", hash = "sha256:2e1b1e51b0774408f091d268648e3d57f7260c1682e7d3a63cb00d22d71bb945"}, + {file = "aiohttp-3.8.5-cp311-cp311-win_amd64.whl", hash = "sha256:043d2299f6dfdc92f0ac5e995dfc56668e1587cea7f9aa9d8a78a1b6554e5755"}, + {file = "aiohttp-3.8.5-cp36-cp36m-macosx_10_9_x86_64.whl", hash = "sha256:cae533195e8122584ec87531d6df000ad07737eaa3c81209e85c928854d2195c"}, + {file = "aiohttp-3.8.5-cp36-cp36m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4f21e83f355643c345177a5d1d8079f9f28b5133bcd154193b799d380331d5d3"}, + {file = "aiohttp-3.8.5-cp36-cp36m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:a7a75ef35f2df54ad55dbf4b73fe1da96f370e51b10c91f08b19603c64004acc"}, + {file = "aiohttp-3.8.5-cp36-cp36m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:2e2e9839e14dd5308ee773c97115f1e0a1cb1d75cbeeee9f33824fa5144c7634"}, + {file = "aiohttp-3.8.5-cp36-cp36m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c44e65da1de4403d0576473e2344828ef9c4c6244d65cf4b75549bb46d40b8dd"}, + {file = "aiohttp-3.8.5-cp36-cp36m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:78d847e4cde6ecc19125ccbc9bfac4a7ab37c234dd88fbb3c5c524e8e14da543"}, + {file = "aiohttp-3.8.5-cp36-cp36m-musllinux_1_1_aarch64.whl", hash = "sha256:c7a815258e5895d8900aec4454f38dca9aed71085f227537208057853f9d13f2"}, + {file = "aiohttp-3.8.5-cp36-cp36m-musllinux_1_1_i686.whl", hash = "sha256:8b929b9bd7cd7c3939f8bcfffa92fae7480bd1aa425279d51a89327d600c704d"}, + {file = "aiohttp-3.8.5-cp36-cp36m-musllinux_1_1_ppc64le.whl", hash = "sha256:5db3a5b833764280ed7618393832e0853e40f3d3e9aa128ac0ba0f8278d08649"}, + {file = "aiohttp-3.8.5-cp36-cp36m-musllinux_1_1_s390x.whl", hash = "sha256:a0215ce6041d501f3155dc219712bc41252d0ab76474615b9700d63d4d9292af"}, + {file = "aiohttp-3.8.5-cp36-cp36m-musllinux_1_1_x86_64.whl", hash = "sha256:fd1ed388ea7fbed22c4968dd64bab0198de60750a25fe8c0c9d4bef5abe13824"}, + {file = "aiohttp-3.8.5-cp36-cp36m-win32.whl", hash = "sha256:6e6783bcc45f397fdebc118d772103d751b54cddf5b60fbcc958382d7dd64f3e"}, + {file = "aiohttp-3.8.5-cp36-cp36m-win_amd64.whl", hash = "sha256:b5411d82cddd212644cf9360879eb5080f0d5f7d809d03262c50dad02f01421a"}, + {file = "aiohttp-3.8.5-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:01d4c0c874aa4ddfb8098e85d10b5e875a70adc63db91f1ae65a4b04d3344cda"}, + {file = "aiohttp-3.8.5-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e5980a746d547a6ba173fd5ee85ce9077e72d118758db05d229044b469d9029a"}, + {file = "aiohttp-3.8.5-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:2a482e6da906d5e6e653be079b29bc173a48e381600161c9932d89dfae5942ef"}, + {file = "aiohttp-3.8.5-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:80bd372b8d0715c66c974cf57fe363621a02f359f1ec81cba97366948c7fc873"}, + {file = "aiohttp-3.8.5-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c1161b345c0a444ebcf46bf0a740ba5dcf50612fd3d0528883fdc0eff578006a"}, + {file = "aiohttp-3.8.5-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:cd56db019015b6acfaaf92e1ac40eb8434847d9bf88b4be4efe5bfd260aee692"}, + {file = "aiohttp-3.8.5-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:153c2549f6c004d2754cc60603d4668899c9895b8a89397444a9c4efa282aaf4"}, + {file = "aiohttp-3.8.5-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:4a01951fabc4ce26ab791da5f3f24dca6d9a6f24121746eb19756416ff2d881b"}, + {file = "aiohttp-3.8.5-cp37-cp37m-musllinux_1_1_ppc64le.whl", hash = "sha256:bfb9162dcf01f615462b995a516ba03e769de0789de1cadc0f916265c257e5d8"}, + {file = "aiohttp-3.8.5-cp37-cp37m-musllinux_1_1_s390x.whl", hash = "sha256:7dde0009408969a43b04c16cbbe252c4f5ef4574ac226bc8815cd7342d2028b6"}, + {file = "aiohttp-3.8.5-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:4149d34c32f9638f38f544b3977a4c24052042affa895352d3636fa8bffd030a"}, + {file = "aiohttp-3.8.5-cp37-cp37m-win32.whl", hash = "sha256:68c5a82c8779bdfc6367c967a4a1b2aa52cd3595388bf5961a62158ee8a59e22"}, + {file = "aiohttp-3.8.5-cp37-cp37m-win_amd64.whl", hash = "sha256:2cf57fb50be5f52bda004b8893e63b48530ed9f0d6c96c84620dc92fe3cd9b9d"}, + {file = "aiohttp-3.8.5-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:eca4bf3734c541dc4f374ad6010a68ff6c6748f00451707f39857f429ca36ced"}, + {file = "aiohttp-3.8.5-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:1274477e4c71ce8cfe6c1ec2f806d57c015ebf84d83373676036e256bc55d690"}, + {file = "aiohttp-3.8.5-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:28c543e54710d6158fc6f439296c7865b29e0b616629767e685a7185fab4a6b9"}, + {file = "aiohttp-3.8.5-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:910bec0c49637d213f5d9877105d26e0c4a4de2f8b1b29405ff37e9fc0ad52b8"}, + {file = "aiohttp-3.8.5-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:5443910d662db951b2e58eb70b0fbe6b6e2ae613477129a5805d0b66c54b6cb7"}, + {file = "aiohttp-3.8.5-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:2e460be6978fc24e3df83193dc0cc4de46c9909ed92dd47d349a452ef49325b7"}, + {file = "aiohttp-3.8.5-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fb1558def481d84f03b45888473fc5a1f35747b5f334ef4e7a571bc0dfcb11f8"}, + {file = "aiohttp-3.8.5-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:34dd0c107799dcbbf7d48b53be761a013c0adf5571bf50c4ecad5643fe9cfcd0"}, + {file = "aiohttp-3.8.5-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:aa1990247f02a54185dc0dff92a6904521172a22664c863a03ff64c42f9b5410"}, + {file = "aiohttp-3.8.5-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:0e584a10f204a617d71d359fe383406305a4b595b333721fa50b867b4a0a1548"}, + {file = "aiohttp-3.8.5-cp38-cp38-musllinux_1_1_ppc64le.whl", hash = "sha256:a3cf433f127efa43fee6b90ea4c6edf6c4a17109d1d037d1a52abec84d8f2e42"}, + {file = "aiohttp-3.8.5-cp38-cp38-musllinux_1_1_s390x.whl", hash = "sha256:c11f5b099adafb18e65c2c997d57108b5bbeaa9eeee64a84302c0978b1ec948b"}, + {file = "aiohttp-3.8.5-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:84de26ddf621d7ac4c975dbea4c945860e08cccde492269db4e1538a6a6f3c35"}, + {file = "aiohttp-3.8.5-cp38-cp38-win32.whl", hash = "sha256:ab88bafedc57dd0aab55fa728ea10c1911f7e4d8b43e1d838a1739f33712921c"}, + {file = "aiohttp-3.8.5-cp38-cp38-win_amd64.whl", hash = "sha256:5798a9aad1879f626589f3df0f8b79b3608a92e9beab10e5fda02c8a2c60db2e"}, + {file = "aiohttp-3.8.5-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:a6ce61195c6a19c785df04e71a4537e29eaa2c50fe745b732aa937c0c77169f3"}, + {file = "aiohttp-3.8.5-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:773dd01706d4db536335fcfae6ea2440a70ceb03dd3e7378f3e815b03c97ab51"}, + {file = "aiohttp-3.8.5-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:f83a552443a526ea38d064588613aca983d0ee0038801bc93c0c916428310c28"}, + {file = "aiohttp-3.8.5-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1f7372f7341fcc16f57b2caded43e81ddd18df53320b6f9f042acad41f8e049a"}, + {file = "aiohttp-3.8.5-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:ea353162f249c8097ea63c2169dd1aa55de1e8fecbe63412a9bc50816e87b761"}, + {file = "aiohttp-3.8.5-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:e5d47ae48db0b2dcf70bc8a3bc72b3de86e2a590fc299fdbbb15af320d2659de"}, + {file = "aiohttp-3.8.5-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d827176898a2b0b09694fbd1088c7a31836d1a505c243811c87ae53a3f6273c1"}, + {file = "aiohttp-3.8.5-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3562b06567c06439d8b447037bb655ef69786c590b1de86c7ab81efe1c9c15d8"}, + {file = "aiohttp-3.8.5-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:4e874cbf8caf8959d2adf572a78bba17cb0e9d7e51bb83d86a3697b686a0ab4d"}, + {file = "aiohttp-3.8.5-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:6809a00deaf3810e38c628e9a33271892f815b853605a936e2e9e5129762356c"}, + {file = "aiohttp-3.8.5-cp39-cp39-musllinux_1_1_ppc64le.whl", hash = "sha256:33776e945d89b29251b33a7e7d006ce86447b2cfd66db5e5ded4e5cd0340585c"}, + {file = "aiohttp-3.8.5-cp39-cp39-musllinux_1_1_s390x.whl", hash = "sha256:eaeed7abfb5d64c539e2db173f63631455f1196c37d9d8d873fc316470dfbacd"}, + {file = "aiohttp-3.8.5-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:e91d635961bec2d8f19dfeb41a539eb94bd073f075ca6dae6c8dc0ee89ad6f91"}, + {file = "aiohttp-3.8.5-cp39-cp39-win32.whl", hash = "sha256:00ad4b6f185ec67f3e6562e8a1d2b69660be43070bd0ef6fcec5211154c7df67"}, + {file = "aiohttp-3.8.5-cp39-cp39-win_amd64.whl", hash = "sha256:c0a9034379a37ae42dea7ac1e048352d96286626251862e448933c0f59cbd79c"}, + {file = "aiohttp-3.8.5.tar.gz", hash = "sha256:b9552ec52cc147dbf1944ac7ac98af7602e51ea2dcd076ed194ca3c0d1c7d0bc"}, ] [package.dependencies] @@ -158,7 +155,6 @@ speedups = ["Brotli", "aiodns", "cchardet"] name = "aioitertools" version = "0.11.0" description = "itertools and builtins for AsyncIO and mixed iterables" -category = "main" optional = true python-versions = ">=3.6" files = [ @@ -173,7 +169,6 @@ typing_extensions = {version = ">=4.0", markers = "python_version < \"3.10\""} name = "aiosignal" version = "1.3.1" description = "aiosignal: a list of registered asynchronous callbacks" -category = "main" optional = true python-versions = ">=3.7" files = [ @@ -188,7 +183,6 @@ frozenlist = ">=1.1.0" name = "async-timeout" version = "4.0.2" description = "Timeout context manager for asyncio programs" -category = "main" optional = true python-versions = ">=3.6" files = [ @@ -200,7 +194,6 @@ files = [ name = "attrs" version = "23.1.0" description = "Classes Without Boilerplate" -category = "main" optional = true python-versions = ">=3.7" files = [ @@ -217,14 +210,13 @@ tests-no-zope = ["cloudpickle", "hypothesis", "mypy (>=1.1.1)", "pympler", "pyte [[package]] name = "azure-core" -version = "1.26.4" +version = "1.28.0" description = "Microsoft Azure Core Library for Python" -category = "main" optional = true python-versions = ">=3.7" files = [ - {file = "azure-core-1.26.4.zip", hash = "sha256:075fe06b74c3007950dd93d49440c2f3430fd9b4a5a2756ec8c79454afc989c6"}, - {file = "azure_core-1.26.4-py3-none-any.whl", hash = "sha256:d9664b4bc2675d72fba461a285ac43ae33abb2967014a955bf136d9703a2ab3c"}, + {file = "azure-core-1.28.0.zip", hash = "sha256:e9eefc66fc1fde56dab6f04d4e5d12c60754d5a9fa49bdcfd8534fc96ed936bd"}, + {file = "azure_core-1.28.0-py3-none-any.whl", hash = "sha256:dec36dfc8eb0b052a853f30c07437effec2f9e3e1fc8f703d9bdaa5cfc0043d9"}, ] [package.dependencies] @@ -239,7 +231,6 @@ aio = ["aiohttp (>=3.0)"] name = "azure-datalake-store" version = "0.0.53" description = "Azure Data Lake Store Filesystem Client Library for Python" -category = "main" optional = true python-versions = "*" files = [ @@ -256,7 +247,6 @@ requests = ">=2.20.0" name = "azure-identity" version = "1.13.0" description = "Microsoft Azure Identity Library for Python" -category = "main" optional = true python-versions = ">=3.7" files = [ @@ -273,39 +263,37 @@ six = ">=1.12.0" [[package]] name = "azure-storage-blob" -version = "12.16.0" +version = "12.17.0" description = "Microsoft Azure Blob Storage Client Library for Python" -category = "main" optional = true python-versions = ">=3.7" files = [ - {file = "azure-storage-blob-12.16.0.zip", hash = "sha256:43b45f19a518a5c6895632f263b3825ebc23574f25cc84b66e1630a6160e466f"}, - {file = "azure_storage_blob-12.16.0-py3-none-any.whl", hash = "sha256:91bb192b2a97939c4259c72373bac0f41e30810bbc853d5184f0f45904eacafd"}, + {file = "azure-storage-blob-12.17.0.zip", hash = "sha256:c14b785a17050b30fc326a315bdae6bc4a078855f4f94a4c303ad74a48dc8c63"}, + {file = "azure_storage_blob-12.17.0-py3-none-any.whl", hash = "sha256:0016e0c549a80282d7b4920c03f2f4ba35c53e6e3c7dbcd2a4a8c8eb3882c1e7"}, ] [package.dependencies] -azure-core = ">=1.26.0,<2.0.0" +azure-core = ">=1.28.0,<2.0.0" cryptography = ">=2.1.4" isodate = ">=0.6.1" -typing-extensions = ">=4.0.1" +typing-extensions = ">=4.3.0" [package.extras] -aio = ["azure-core[aio] (>=1.26.0,<2.0.0)"] +aio = ["azure-core[aio] (>=1.28.0,<2.0.0)"] [[package]] name = "boto3" -version = "1.26.76" +version = "1.26.161" description = "The AWS SDK for Python" -category = "main" optional = false python-versions = ">= 3.7" files = [ - {file = "boto3-1.26.76-py3-none-any.whl", hash = "sha256:b4c2969b7677762914394b8273cc1905dfe5b71f250741c1a575487ae357e729"}, - {file = "boto3-1.26.76.tar.gz", hash = "sha256:30c7d967ed1c6b5a05643e42cae9d4d36c3f1cb6782637ddc7007a104cfd9027"}, + {file = "boto3-1.26.161-py3-none-any.whl", hash = "sha256:f66e5c9dbe7f34383bcf64fa6070771355c11a44dd75c7f1279f2f37e1c89183"}, + {file = "boto3-1.26.161.tar.gz", hash = "sha256:662731e464d14af1035f44fc6a46b0e3112ee011ac0a5ed416d205daa3e15f25"}, ] [package.dependencies] -botocore = ">=1.29.76,<1.30.0" +botocore = ">=1.29.161,<1.30.0" jmespath = ">=0.7.1,<2.0.0" s3transfer = ">=0.6.0,<0.7.0" @@ -314,14 +302,13 @@ crt = ["botocore[crt] (>=1.21.0,<2.0a0)"] [[package]] name = "botocore" -version = "1.29.76" +version = "1.29.161" description = "Low-level, data-driven core of boto 3." -category = "main" optional = false python-versions = ">= 3.7" files = [ - {file = "botocore-1.29.76-py3-none-any.whl", hash = "sha256:70735b00cd529f152992231ca6757e458e5ec25db43767b3526e9a35b2f143b7"}, - {file = "botocore-1.29.76.tar.gz", hash = "sha256:c2f67b6b3f8acf2968eafca06526f07b9fb0d27bac4c68a635d51abb675134a7"}, + {file = "botocore-1.29.161-py3-none-any.whl", hash = "sha256:b906999dd53dda2ef0ef6f7f55fcc81a4b06b9f1c8a9f65c546e0b981f959f5f"}, + {file = "botocore-1.29.161.tar.gz", hash = "sha256:a50edd715eb510343e27849f36483804aae4b871590db4d4996aa53368dcac40"}, ] [package.dependencies] @@ -336,7 +323,6 @@ crt = ["awscrt (==0.16.9)"] name = "build" version = "0.10.0" description = "A simple, correct Python build frontend" -category = "dev" optional = false python-versions = ">= 3.7" files = [ @@ -358,21 +344,19 @@ virtualenv = ["virtualenv (>=20.0.35)"] [[package]] name = "certifi" -version = "2023.5.7" +version = "2023.7.22" description = "Python package for providing Mozilla's CA Bundle." -category = "main" optional = false python-versions = ">=3.6" files = [ - {file = "certifi-2023.5.7-py3-none-any.whl", hash = "sha256:c6c2e98f5c7869efca1f8916fed228dd91539f9f1b444c314c06eef02980c716"}, - {file = "certifi-2023.5.7.tar.gz", hash = "sha256:0f0d56dc5a6ad56fd4ba36484d6cc34451e1c6548c61daad8c320169f91eddc7"}, + {file = "certifi-2023.7.22-py3-none-any.whl", hash = "sha256:92d6037539857d8206b8f6ae472e8b77db8058fec5937a1ef3f54304089edbb9"}, + {file = "certifi-2023.7.22.tar.gz", hash = "sha256:539cc1d13202e33ca466e88b2807e29f4c13049d6d87031a3c110744495cb082"}, ] [[package]] name = "cffi" version = "1.15.1" description = "Foreign Function Interface for Python calling C code." -category = "main" optional = false python-versions = "*" files = [ @@ -449,7 +433,6 @@ pycparser = "*" name = "cfgv" version = "3.3.1" description = "Validate configuration and produce human readable error messages." -category = "dev" optional = false python-versions = ">=3.6.1" files = [ @@ -459,99 +442,97 @@ files = [ [[package]] name = "charset-normalizer" -version = "3.1.0" +version = "3.2.0" description = "The Real First Universal Charset Detector. Open, modern and actively maintained alternative to Chardet." -category = "main" optional = false python-versions = ">=3.7.0" files = [ - {file = "charset-normalizer-3.1.0.tar.gz", hash = "sha256:34e0a2f9c370eb95597aae63bf85eb5e96826d81e3dcf88b8886012906f509b5"}, - {file = "charset_normalizer-3.1.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:e0ac8959c929593fee38da1c2b64ee9778733cdf03c482c9ff1d508b6b593b2b"}, - {file = "charset_normalizer-3.1.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:d7fc3fca01da18fbabe4625d64bb612b533533ed10045a2ac3dd194bfa656b60"}, - {file = "charset_normalizer-3.1.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:04eefcee095f58eaabe6dc3cc2262f3bcd776d2c67005880894f447b3f2cb9c1"}, - {file = "charset_normalizer-3.1.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:20064ead0717cf9a73a6d1e779b23d149b53daf971169289ed2ed43a71e8d3b0"}, - {file = "charset_normalizer-3.1.0-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:1435ae15108b1cb6fffbcea2af3d468683b7afed0169ad718451f8db5d1aff6f"}, - {file = "charset_normalizer-3.1.0-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:c84132a54c750fda57729d1e2599bb598f5fa0344085dbde5003ba429a4798c0"}, - {file = "charset_normalizer-3.1.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:75f2568b4189dda1c567339b48cba4ac7384accb9c2a7ed655cd86b04055c795"}, - {file = "charset_normalizer-3.1.0-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:11d3bcb7be35e7b1bba2c23beedac81ee893ac9871d0ba79effc7fc01167db6c"}, - {file = "charset_normalizer-3.1.0-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:891cf9b48776b5c61c700b55a598621fdb7b1e301a550365571e9624f270c203"}, - {file = "charset_normalizer-3.1.0-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:5f008525e02908b20e04707a4f704cd286d94718f48bb33edddc7d7b584dddc1"}, - {file = "charset_normalizer-3.1.0-cp310-cp310-musllinux_1_1_ppc64le.whl", hash = "sha256:b06f0d3bf045158d2fb8837c5785fe9ff9b8c93358be64461a1089f5da983137"}, - {file = "charset_normalizer-3.1.0-cp310-cp310-musllinux_1_1_s390x.whl", hash = "sha256:49919f8400b5e49e961f320c735388ee686a62327e773fa5b3ce6721f7e785ce"}, - {file = "charset_normalizer-3.1.0-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:22908891a380d50738e1f978667536f6c6b526a2064156203d418f4856d6e86a"}, - {file = "charset_normalizer-3.1.0-cp310-cp310-win32.whl", hash = "sha256:12d1a39aa6b8c6f6248bb54550efcc1c38ce0d8096a146638fd4738e42284448"}, - {file = "charset_normalizer-3.1.0-cp310-cp310-win_amd64.whl", hash = "sha256:65ed923f84a6844de5fd29726b888e58c62820e0769b76565480e1fdc3d062f8"}, - {file = "charset_normalizer-3.1.0-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:9a3267620866c9d17b959a84dd0bd2d45719b817245e49371ead79ed4f710d19"}, - {file = "charset_normalizer-3.1.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:6734e606355834f13445b6adc38b53c0fd45f1a56a9ba06c2058f86893ae8017"}, - {file = "charset_normalizer-3.1.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:f8303414c7b03f794347ad062c0516cee0e15f7a612abd0ce1e25caf6ceb47df"}, - {file = "charset_normalizer-3.1.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:aaf53a6cebad0eae578f062c7d462155eada9c172bd8c4d250b8c1d8eb7f916a"}, - {file = "charset_normalizer-3.1.0-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:3dc5b6a8ecfdc5748a7e429782598e4f17ef378e3e272eeb1340ea57c9109f41"}, - {file = "charset_normalizer-3.1.0-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:e1b25e3ad6c909f398df8921780d6a3d120d8c09466720226fc621605b6f92b1"}, - {file = "charset_normalizer-3.1.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0ca564606d2caafb0abe6d1b5311c2649e8071eb241b2d64e75a0d0065107e62"}, - {file = "charset_normalizer-3.1.0-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:b82fab78e0b1329e183a65260581de4375f619167478dddab510c6c6fb04d9b6"}, - {file = "charset_normalizer-3.1.0-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:bd7163182133c0c7701b25e604cf1611c0d87712e56e88e7ee5d72deab3e76b5"}, - {file = "charset_normalizer-3.1.0-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:11d117e6c63e8f495412d37e7dc2e2fff09c34b2d09dbe2bee3c6229577818be"}, - {file = "charset_normalizer-3.1.0-cp311-cp311-musllinux_1_1_ppc64le.whl", hash = "sha256:cf6511efa4801b9b38dc5546d7547d5b5c6ef4b081c60b23e4d941d0eba9cbeb"}, - {file = "charset_normalizer-3.1.0-cp311-cp311-musllinux_1_1_s390x.whl", hash = "sha256:abc1185d79f47c0a7aaf7e2412a0eb2c03b724581139193d2d82b3ad8cbb00ac"}, - {file = "charset_normalizer-3.1.0-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:cb7b2ab0188829593b9de646545175547a70d9a6e2b63bf2cd87a0a391599324"}, - {file = "charset_normalizer-3.1.0-cp311-cp311-win32.whl", hash = "sha256:c36bcbc0d5174a80d6cccf43a0ecaca44e81d25be4b7f90f0ed7bcfbb5a00909"}, - {file = "charset_normalizer-3.1.0-cp311-cp311-win_amd64.whl", hash = "sha256:cca4def576f47a09a943666b8f829606bcb17e2bc2d5911a46c8f8da45f56755"}, - {file = "charset_normalizer-3.1.0-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:0c95f12b74681e9ae127728f7e5409cbbef9cd914d5896ef238cc779b8152373"}, - {file = "charset_normalizer-3.1.0-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:fca62a8301b605b954ad2e9c3666f9d97f63872aa4efcae5492baca2056b74ab"}, - {file = "charset_normalizer-3.1.0-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:ac0aa6cd53ab9a31d397f8303f92c42f534693528fafbdb997c82bae6e477ad9"}, - {file = "charset_normalizer-3.1.0-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:c3af8e0f07399d3176b179f2e2634c3ce9c1301379a6b8c9c9aeecd481da494f"}, - {file = "charset_normalizer-3.1.0-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3a5fc78f9e3f501a1614a98f7c54d3969f3ad9bba8ba3d9b438c3bc5d047dd28"}, - {file = "charset_normalizer-3.1.0-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:628c985afb2c7d27a4800bfb609e03985aaecb42f955049957814e0491d4006d"}, - {file = "charset_normalizer-3.1.0-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:74db0052d985cf37fa111828d0dd230776ac99c740e1a758ad99094be4f1803d"}, - {file = "charset_normalizer-3.1.0-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:1e8fcdd8f672a1c4fc8d0bd3a2b576b152d2a349782d1eb0f6b8e52e9954731d"}, - {file = "charset_normalizer-3.1.0-cp37-cp37m-musllinux_1_1_ppc64le.whl", hash = "sha256:04afa6387e2b282cf78ff3dbce20f0cc071c12dc8f685bd40960cc68644cfea6"}, - {file = "charset_normalizer-3.1.0-cp37-cp37m-musllinux_1_1_s390x.whl", hash = "sha256:dd5653e67b149503c68c4018bf07e42eeed6b4e956b24c00ccdf93ac79cdff84"}, - {file = "charset_normalizer-3.1.0-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:d2686f91611f9e17f4548dbf050e75b079bbc2a82be565832bc8ea9047b61c8c"}, - {file = "charset_normalizer-3.1.0-cp37-cp37m-win32.whl", hash = "sha256:4155b51ae05ed47199dc5b2a4e62abccb274cee6b01da5b895099b61b1982974"}, - {file = "charset_normalizer-3.1.0-cp37-cp37m-win_amd64.whl", hash = "sha256:322102cdf1ab682ecc7d9b1c5eed4ec59657a65e1c146a0da342b78f4112db23"}, - {file = "charset_normalizer-3.1.0-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:e633940f28c1e913615fd624fcdd72fdba807bf53ea6925d6a588e84e1151531"}, - {file = "charset_normalizer-3.1.0-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:3a06f32c9634a8705f4ca9946d667609f52cf130d5548881401f1eb2c39b1e2c"}, - {file = "charset_normalizer-3.1.0-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:7381c66e0561c5757ffe616af869b916c8b4e42b367ab29fedc98481d1e74e14"}, - {file = "charset_normalizer-3.1.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3573d376454d956553c356df45bb824262c397c6e26ce43e8203c4c540ee0acb"}, - {file = "charset_normalizer-3.1.0-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:e89df2958e5159b811af9ff0f92614dabf4ff617c03a4c1c6ff53bf1c399e0e1"}, - {file = "charset_normalizer-3.1.0-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:78cacd03e79d009d95635e7d6ff12c21eb89b894c354bd2b2ed0b4763373693b"}, - {file = "charset_normalizer-3.1.0-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:de5695a6f1d8340b12a5d6d4484290ee74d61e467c39ff03b39e30df62cf83a0"}, - {file = "charset_normalizer-3.1.0-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1c60b9c202d00052183c9be85e5eaf18a4ada0a47d188a83c8f5c5b23252f649"}, - {file = "charset_normalizer-3.1.0-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:f645caaf0008bacf349875a974220f1f1da349c5dbe7c4ec93048cdc785a3326"}, - {file = "charset_normalizer-3.1.0-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:ea9f9c6034ea2d93d9147818f17c2a0860d41b71c38b9ce4d55f21b6f9165a11"}, - {file = "charset_normalizer-3.1.0-cp38-cp38-musllinux_1_1_ppc64le.whl", hash = "sha256:80d1543d58bd3d6c271b66abf454d437a438dff01c3e62fdbcd68f2a11310d4b"}, - {file = "charset_normalizer-3.1.0-cp38-cp38-musllinux_1_1_s390x.whl", hash = "sha256:73dc03a6a7e30b7edc5b01b601e53e7fc924b04e1835e8e407c12c037e81adbd"}, - {file = "charset_normalizer-3.1.0-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:6f5c2e7bc8a4bf7c426599765b1bd33217ec84023033672c1e9a8b35eaeaaaf8"}, - {file = "charset_normalizer-3.1.0-cp38-cp38-win32.whl", hash = "sha256:12a2b561af122e3d94cdb97fe6fb2bb2b82cef0cdca131646fdb940a1eda04f0"}, - {file = "charset_normalizer-3.1.0-cp38-cp38-win_amd64.whl", hash = "sha256:3160a0fd9754aab7d47f95a6b63ab355388d890163eb03b2d2b87ab0a30cfa59"}, - {file = "charset_normalizer-3.1.0-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:38e812a197bf8e71a59fe55b757a84c1f946d0ac114acafaafaf21667a7e169e"}, - {file = "charset_normalizer-3.1.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:6baf0baf0d5d265fa7944feb9f7451cc316bfe30e8df1a61b1bb08577c554f31"}, - {file = "charset_normalizer-3.1.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:8f25e17ab3039b05f762b0a55ae0b3632b2e073d9c8fc88e89aca31a6198e88f"}, - {file = "charset_normalizer-3.1.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3747443b6a904001473370d7810aa19c3a180ccd52a7157aacc264a5ac79265e"}, - {file = "charset_normalizer-3.1.0-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:b116502087ce8a6b7a5f1814568ccbd0e9f6cfd99948aa59b0e241dc57cf739f"}, - {file = "charset_normalizer-3.1.0-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:d16fd5252f883eb074ca55cb622bc0bee49b979ae4e8639fff6ca3ff44f9f854"}, - {file = "charset_normalizer-3.1.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:21fa558996782fc226b529fdd2ed7866c2c6ec91cee82735c98a197fae39f706"}, - {file = "charset_normalizer-3.1.0-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:6f6c7a8a57e9405cad7485f4c9d3172ae486cfef1344b5ddd8e5239582d7355e"}, - {file = "charset_normalizer-3.1.0-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:ac3775e3311661d4adace3697a52ac0bab17edd166087d493b52d4f4f553f9f0"}, - {file = "charset_normalizer-3.1.0-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:10c93628d7497c81686e8e5e557aafa78f230cd9e77dd0c40032ef90c18f2230"}, - {file = "charset_normalizer-3.1.0-cp39-cp39-musllinux_1_1_ppc64le.whl", hash = "sha256:6f4f4668e1831850ebcc2fd0b1cd11721947b6dc7c00bf1c6bd3c929ae14f2c7"}, - {file = "charset_normalizer-3.1.0-cp39-cp39-musllinux_1_1_s390x.whl", hash = "sha256:0be65ccf618c1e7ac9b849c315cc2e8a8751d9cfdaa43027d4f6624bd587ab7e"}, - {file = "charset_normalizer-3.1.0-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:53d0a3fa5f8af98a1e261de6a3943ca631c526635eb5817a87a59d9a57ebf48f"}, - {file = "charset_normalizer-3.1.0-cp39-cp39-win32.whl", hash = "sha256:a04f86f41a8916fe45ac5024ec477f41f886b3c435da2d4e3d2709b22ab02af1"}, - {file = "charset_normalizer-3.1.0-cp39-cp39-win_amd64.whl", hash = "sha256:830d2948a5ec37c386d3170c483063798d7879037492540f10a475e3fd6f244b"}, - {file = "charset_normalizer-3.1.0-py3-none-any.whl", hash = "sha256:3d9098b479e78c85080c98e1e35ff40b4a31d8953102bb0fd7d1b6f8a2111a3d"}, + {file = "charset-normalizer-3.2.0.tar.gz", hash = "sha256:3bb3d25a8e6c0aedd251753a79ae98a093c7e7b471faa3aa9a93a81431987ace"}, + {file = "charset_normalizer-3.2.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:0b87549028f680ca955556e3bd57013ab47474c3124dc069faa0b6545b6c9710"}, + {file = "charset_normalizer-3.2.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:7c70087bfee18a42b4040bb9ec1ca15a08242cf5867c58726530bdf3945672ed"}, + {file = "charset_normalizer-3.2.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:a103b3a7069b62f5d4890ae1b8f0597618f628b286b03d4bc9195230b154bfa9"}, + {file = "charset_normalizer-3.2.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:94aea8eff76ee6d1cdacb07dd2123a68283cb5569e0250feab1240058f53b623"}, + {file = "charset_normalizer-3.2.0-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:db901e2ac34c931d73054d9797383d0f8009991e723dab15109740a63e7f902a"}, + {file = "charset_normalizer-3.2.0-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:b0dac0ff919ba34d4df1b6131f59ce95b08b9065233446be7e459f95554c0dc8"}, + {file = "charset_normalizer-3.2.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:193cbc708ea3aca45e7221ae58f0fd63f933753a9bfb498a3b474878f12caaad"}, + {file = "charset_normalizer-3.2.0-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:09393e1b2a9461950b1c9a45d5fd251dc7c6f228acab64da1c9c0165d9c7765c"}, + {file = "charset_normalizer-3.2.0-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:baacc6aee0b2ef6f3d308e197b5d7a81c0e70b06beae1f1fcacffdbd124fe0e3"}, + {file = "charset_normalizer-3.2.0-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:bf420121d4c8dce6b889f0e8e4ec0ca34b7f40186203f06a946fa0276ba54029"}, + {file = "charset_normalizer-3.2.0-cp310-cp310-musllinux_1_1_ppc64le.whl", hash = "sha256:c04a46716adde8d927adb9457bbe39cf473e1e2c2f5d0a16ceb837e5d841ad4f"}, + {file = "charset_normalizer-3.2.0-cp310-cp310-musllinux_1_1_s390x.whl", hash = "sha256:aaf63899c94de41fe3cf934601b0f7ccb6b428c6e4eeb80da72c58eab077b19a"}, + {file = "charset_normalizer-3.2.0-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:d62e51710986674142526ab9f78663ca2b0726066ae26b78b22e0f5e571238dd"}, + {file = "charset_normalizer-3.2.0-cp310-cp310-win32.whl", hash = "sha256:04e57ab9fbf9607b77f7d057974694b4f6b142da9ed4a199859d9d4d5c63fe96"}, + {file = "charset_normalizer-3.2.0-cp310-cp310-win_amd64.whl", hash = "sha256:48021783bdf96e3d6de03a6e39a1171ed5bd7e8bb93fc84cc649d11490f87cea"}, + {file = "charset_normalizer-3.2.0-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:4957669ef390f0e6719db3613ab3a7631e68424604a7b448f079bee145da6e09"}, + {file = "charset_normalizer-3.2.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:46fb8c61d794b78ec7134a715a3e564aafc8f6b5e338417cb19fe9f57a5a9bf2"}, + {file = "charset_normalizer-3.2.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:f779d3ad205f108d14e99bb3859aa7dd8e9c68874617c72354d7ecaec2a054ac"}, + {file = "charset_normalizer-3.2.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f25c229a6ba38a35ae6e25ca1264621cc25d4d38dca2942a7fce0b67a4efe918"}, + {file = "charset_normalizer-3.2.0-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:2efb1bd13885392adfda4614c33d3b68dee4921fd0ac1d3988f8cbb7d589e72a"}, + {file = "charset_normalizer-3.2.0-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:1f30b48dd7fa1474554b0b0f3fdfdd4c13b5c737a3c6284d3cdc424ec0ffff3a"}, + {file = "charset_normalizer-3.2.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:246de67b99b6851627d945db38147d1b209a899311b1305dd84916f2b88526c6"}, + {file = "charset_normalizer-3.2.0-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:9bd9b3b31adcb054116447ea22caa61a285d92e94d710aa5ec97992ff5eb7cf3"}, + {file = "charset_normalizer-3.2.0-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:8c2f5e83493748286002f9369f3e6607c565a6a90425a3a1fef5ae32a36d749d"}, + {file = "charset_normalizer-3.2.0-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:3170c9399da12c9dc66366e9d14da8bf7147e1e9d9ea566067bbce7bb74bd9c2"}, + {file = "charset_normalizer-3.2.0-cp311-cp311-musllinux_1_1_ppc64le.whl", hash = "sha256:7a4826ad2bd6b07ca615c74ab91f32f6c96d08f6fcc3902ceeedaec8cdc3bcd6"}, + {file = "charset_normalizer-3.2.0-cp311-cp311-musllinux_1_1_s390x.whl", hash = "sha256:3b1613dd5aee995ec6d4c69f00378bbd07614702a315a2cf6c1d21461fe17c23"}, + {file = "charset_normalizer-3.2.0-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:9e608aafdb55eb9f255034709e20d5a83b6d60c054df0802fa9c9883d0a937aa"}, + {file = "charset_normalizer-3.2.0-cp311-cp311-win32.whl", hash = "sha256:f2a1d0fd4242bd8643ce6f98927cf9c04540af6efa92323e9d3124f57727bfc1"}, + {file = "charset_normalizer-3.2.0-cp311-cp311-win_amd64.whl", hash = "sha256:681eb3d7e02e3c3655d1b16059fbfb605ac464c834a0c629048a30fad2b27489"}, + {file = "charset_normalizer-3.2.0-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:c57921cda3a80d0f2b8aec7e25c8aa14479ea92b5b51b6876d975d925a2ea346"}, + {file = "charset_normalizer-3.2.0-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:41b25eaa7d15909cf3ac4c96088c1f266a9a93ec44f87f1d13d4a0e86c81b982"}, + {file = "charset_normalizer-3.2.0-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:f058f6963fd82eb143c692cecdc89e075fa0828db2e5b291070485390b2f1c9c"}, + {file = "charset_normalizer-3.2.0-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:a7647ebdfb9682b7bb97e2a5e7cb6ae735b1c25008a70b906aecca294ee96cf4"}, + {file = "charset_normalizer-3.2.0-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:eef9df1eefada2c09a5e7a40991b9fc6ac6ef20b1372abd48d2794a316dc0449"}, + {file = "charset_normalizer-3.2.0-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e03b8895a6990c9ab2cdcd0f2fe44088ca1c65ae592b8f795c3294af00a461c3"}, + {file = "charset_normalizer-3.2.0-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:ee4006268ed33370957f55bf2e6f4d263eaf4dc3cfc473d1d90baff6ed36ce4a"}, + {file = "charset_normalizer-3.2.0-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:c4983bf937209c57240cff65906b18bb35e64ae872da6a0db937d7b4af845dd7"}, + {file = "charset_normalizer-3.2.0-cp37-cp37m-musllinux_1_1_ppc64le.whl", hash = "sha256:3bb7fda7260735efe66d5107fb7e6af6a7c04c7fce9b2514e04b7a74b06bf5dd"}, + {file = "charset_normalizer-3.2.0-cp37-cp37m-musllinux_1_1_s390x.whl", hash = "sha256:72814c01533f51d68702802d74f77ea026b5ec52793c791e2da806a3844a46c3"}, + {file = "charset_normalizer-3.2.0-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:70c610f6cbe4b9fce272c407dd9d07e33e6bf7b4aa1b7ffb6f6ded8e634e3592"}, + {file = "charset_normalizer-3.2.0-cp37-cp37m-win32.whl", hash = "sha256:a401b4598e5d3f4a9a811f3daf42ee2291790c7f9d74b18d75d6e21dda98a1a1"}, + {file = "charset_normalizer-3.2.0-cp37-cp37m-win_amd64.whl", hash = "sha256:c0b21078a4b56965e2b12f247467b234734491897e99c1d51cee628da9786959"}, + {file = "charset_normalizer-3.2.0-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:95eb302ff792e12aba9a8b8f8474ab229a83c103d74a750ec0bd1c1eea32e669"}, + {file = "charset_normalizer-3.2.0-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:1a100c6d595a7f316f1b6f01d20815d916e75ff98c27a01ae817439ea7726329"}, + {file = "charset_normalizer-3.2.0-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:6339d047dab2780cc6220f46306628e04d9750f02f983ddb37439ca47ced7149"}, + {file = "charset_normalizer-3.2.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e4b749b9cc6ee664a3300bb3a273c1ca8068c46be705b6c31cf5d276f8628a94"}, + {file = "charset_normalizer-3.2.0-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:a38856a971c602f98472050165cea2cdc97709240373041b69030be15047691f"}, + {file = "charset_normalizer-3.2.0-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:f87f746ee241d30d6ed93969de31e5ffd09a2961a051e60ae6bddde9ec3583aa"}, + {file = "charset_normalizer-3.2.0-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:89f1b185a01fe560bc8ae5f619e924407efca2191b56ce749ec84982fc59a32a"}, + {file = "charset_normalizer-3.2.0-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e1c8a2f4c69e08e89632defbfabec2feb8a8d99edc9f89ce33c4b9e36ab63037"}, + {file = "charset_normalizer-3.2.0-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:2f4ac36d8e2b4cc1aa71df3dd84ff8efbe3bfb97ac41242fbcfc053c67434f46"}, + {file = "charset_normalizer-3.2.0-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:a386ebe437176aab38c041de1260cd3ea459c6ce5263594399880bbc398225b2"}, + {file = "charset_normalizer-3.2.0-cp38-cp38-musllinux_1_1_ppc64le.whl", hash = "sha256:ccd16eb18a849fd8dcb23e23380e2f0a354e8daa0c984b8a732d9cfaba3a776d"}, + {file = "charset_normalizer-3.2.0-cp38-cp38-musllinux_1_1_s390x.whl", hash = "sha256:e6a5bf2cba5ae1bb80b154ed68a3cfa2fa00fde979a7f50d6598d3e17d9ac20c"}, + {file = "charset_normalizer-3.2.0-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:45de3f87179c1823e6d9e32156fb14c1927fcc9aba21433f088fdfb555b77c10"}, + {file = "charset_normalizer-3.2.0-cp38-cp38-win32.whl", hash = "sha256:1000fba1057b92a65daec275aec30586c3de2401ccdcd41f8a5c1e2c87078706"}, + {file = "charset_normalizer-3.2.0-cp38-cp38-win_amd64.whl", hash = "sha256:8b2c760cfc7042b27ebdb4a43a4453bd829a5742503599144d54a032c5dc7e9e"}, + {file = "charset_normalizer-3.2.0-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:855eafa5d5a2034b4621c74925d89c5efef61418570e5ef9b37717d9c796419c"}, + {file = "charset_normalizer-3.2.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:203f0c8871d5a7987be20c72442488a0b8cfd0f43b7973771640fc593f56321f"}, + {file = "charset_normalizer-3.2.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:e857a2232ba53ae940d3456f7533ce6ca98b81917d47adc3c7fd55dad8fab858"}, + {file = "charset_normalizer-3.2.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5e86d77b090dbddbe78867a0275cb4df08ea195e660f1f7f13435a4649e954e5"}, + {file = "charset_normalizer-3.2.0-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:c4fb39a81950ec280984b3a44f5bd12819953dc5fa3a7e6fa7a80db5ee853952"}, + {file = "charset_normalizer-3.2.0-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:2dee8e57f052ef5353cf608e0b4c871aee320dd1b87d351c28764fc0ca55f9f4"}, + {file = "charset_normalizer-3.2.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8700f06d0ce6f128de3ccdbc1acaea1ee264d2caa9ca05daaf492fde7c2a7200"}, + {file = "charset_normalizer-3.2.0-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1920d4ff15ce893210c1f0c0e9d19bfbecb7983c76b33f046c13a8ffbd570252"}, + {file = "charset_normalizer-3.2.0-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:c1c76a1743432b4b60ab3358c937a3fe1341c828ae6194108a94c69028247f22"}, + {file = "charset_normalizer-3.2.0-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:f7560358a6811e52e9c4d142d497f1a6e10103d3a6881f18d04dbce3729c0e2c"}, + {file = "charset_normalizer-3.2.0-cp39-cp39-musllinux_1_1_ppc64le.whl", hash = "sha256:c8063cf17b19661471ecbdb3df1c84f24ad2e389e326ccaf89e3fb2484d8dd7e"}, + {file = "charset_normalizer-3.2.0-cp39-cp39-musllinux_1_1_s390x.whl", hash = "sha256:cd6dbe0238f7743d0efe563ab46294f54f9bc8f4b9bcf57c3c666cc5bc9d1299"}, + {file = "charset_normalizer-3.2.0-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:1249cbbf3d3b04902ff081ffbb33ce3377fa6e4c7356f759f3cd076cc138d020"}, + {file = "charset_normalizer-3.2.0-cp39-cp39-win32.whl", hash = "sha256:6c409c0deba34f147f77efaa67b8e4bb83d2f11c8806405f76397ae5b8c0d1c9"}, + {file = "charset_normalizer-3.2.0-cp39-cp39-win_amd64.whl", hash = "sha256:7095f6fbfaa55defb6b733cfeb14efaae7a29f0b59d8cf213be4e7ca0b857b80"}, + {file = "charset_normalizer-3.2.0-py3-none-any.whl", hash = "sha256:8e098148dd37b4ce3baca71fb394c81dc5d9c7728c95df695d2dca218edf40e6"}, ] [[package]] name = "click" -version = "8.1.3" +version = "8.1.6" description = "Composable command line interface toolkit" -category = "main" optional = false python-versions = ">=3.7" files = [ - {file = "click-8.1.3-py3-none-any.whl", hash = "sha256:bb4d8133cb15a609f44e8213d9b391b0809795062913b383c62be0ee95b1db48"}, - {file = "click-8.1.3.tar.gz", hash = "sha256:7682dc8afb30297001674575ea00d1814d808d6a36af415a82bd481d37ba7b8e"}, + {file = "click-8.1.6-py3-none-any.whl", hash = "sha256:fa244bb30b3b5ee2cae3da8f55c9e5e0c0e86093306301fb418eb9dc40fbded5"}, + {file = "click-8.1.6.tar.gz", hash = "sha256:48ee849951919527a045bfe3bf7baa8a959c423134e1a5b98c05c20ba75a1cbd"}, ] [package.dependencies] @@ -561,7 +542,6 @@ colorama = {version = "*", markers = "platform_system == \"Windows\""} name = "colorama" version = "0.4.6" description = "Cross-platform colored terminal text." -category = "main" optional = false python-versions = "!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*,!=3.4.*,!=3.5.*,!=3.6.*,>=2.7" files = [ @@ -573,7 +553,6 @@ files = [ name = "coverage" version = "7.2.7" description = "Code coverage measurement for Python" -category = "dev" optional = false python-versions = ">=3.7" files = [ @@ -647,31 +626,34 @@ toml = ["tomli"] [[package]] name = "cryptography" -version = "41.0.0" +version = "41.0.2" description = "cryptography is a package which provides cryptographic recipes and primitives to Python developers." -category = "main" optional = false python-versions = ">=3.7" files = [ - {file = "cryptography-41.0.0-cp37-abi3-macosx_10_12_universal2.whl", hash = "sha256:3c5ef25d060c80d6d9f7f9892e1d41bb1c79b78ce74805b8cb4aa373cb7d5ec8"}, - {file = "cryptography-41.0.0-cp37-abi3-macosx_10_12_x86_64.whl", hash = "sha256:8362565b3835ceacf4dc8f3b56471a2289cf51ac80946f9087e66dc283a810e0"}, - {file = "cryptography-41.0.0-cp37-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3680248309d340fda9611498a5319b0193a8dbdb73586a1acf8109d06f25b92d"}, - {file = "cryptography-41.0.0-cp37-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:84a165379cb9d411d58ed739e4af3396e544eac190805a54ba2e0322feb55c46"}, - {file = "cryptography-41.0.0-cp37-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:4ab14d567f7bbe7f1cdff1c53d5324ed4d3fc8bd17c481b395db224fb405c237"}, - {file = "cryptography-41.0.0-cp37-abi3-manylinux_2_28_x86_64.whl", hash = "sha256:9f65e842cb02550fac96536edb1d17f24c0a338fd84eaf582be25926e993dde4"}, - {file = "cryptography-41.0.0-cp37-abi3-musllinux_1_1_aarch64.whl", hash = "sha256:b7f2f5c525a642cecad24ee8670443ba27ac1fab81bba4cc24c7b6b41f2d0c75"}, - {file = "cryptography-41.0.0-cp37-abi3-musllinux_1_1_x86_64.whl", hash = "sha256:7d92f0248d38faa411d17f4107fc0bce0c42cae0b0ba5415505df72d751bf62d"}, - {file = "cryptography-41.0.0-cp37-abi3-win32.whl", hash = "sha256:34d405ea69a8b34566ba3dfb0521379b210ea5d560fafedf9f800a9a94a41928"}, - {file = "cryptography-41.0.0-cp37-abi3-win_amd64.whl", hash = "sha256:344c6de9f8bda3c425b3a41b319522ba3208551b70c2ae00099c205f0d9fd3be"}, - {file = "cryptography-41.0.0-pp38-pypy38_pp73-macosx_10_12_x86_64.whl", hash = "sha256:88ff107f211ea696455ea8d911389f6d2b276aabf3231bf72c8853d22db755c5"}, - {file = "cryptography-41.0.0-pp38-pypy38_pp73-manylinux_2_28_aarch64.whl", hash = "sha256:b846d59a8d5a9ba87e2c3d757ca019fa576793e8758174d3868aecb88d6fc8eb"}, - {file = "cryptography-41.0.0-pp38-pypy38_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:f5d0bf9b252f30a31664b6f64432b4730bb7038339bd18b1fafe129cfc2be9be"}, - {file = "cryptography-41.0.0-pp38-pypy38_pp73-win_amd64.whl", hash = "sha256:5c1f7293c31ebc72163a9a0df246f890d65f66b4a40d9ec80081969ba8c78cc9"}, - {file = "cryptography-41.0.0-pp39-pypy39_pp73-macosx_10_12_x86_64.whl", hash = "sha256:bf8fc66012ca857d62f6a347007e166ed59c0bc150cefa49f28376ebe7d992a2"}, - {file = "cryptography-41.0.0-pp39-pypy39_pp73-manylinux_2_28_aarch64.whl", hash = "sha256:a4fc68d1c5b951cfb72dfd54702afdbbf0fb7acdc9b7dc4301bbf2225a27714d"}, - {file = "cryptography-41.0.0-pp39-pypy39_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:14754bcdae909d66ff24b7b5f166d69340ccc6cb15731670435efd5719294895"}, - {file = "cryptography-41.0.0-pp39-pypy39_pp73-win_amd64.whl", hash = "sha256:0ddaee209d1cf1f180f1efa338a68c4621154de0afaef92b89486f5f96047c55"}, - {file = "cryptography-41.0.0.tar.gz", hash = "sha256:6b71f64beeea341c9b4f963b48ee3b62d62d57ba93eb120e1196b31dc1025e78"}, + {file = "cryptography-41.0.2-cp37-abi3-macosx_10_12_universal2.whl", hash = "sha256:01f1d9e537f9a15b037d5d9ee442b8c22e3ae11ce65ea1f3316a41c78756b711"}, + {file = "cryptography-41.0.2-cp37-abi3-macosx_10_12_x86_64.whl", hash = "sha256:079347de771f9282fbfe0e0236c716686950c19dee1b76240ab09ce1624d76d7"}, + {file = "cryptography-41.0.2-cp37-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:439c3cc4c0d42fa999b83ded80a9a1fb54d53c58d6e59234cfe97f241e6c781d"}, + {file = "cryptography-41.0.2-cp37-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f14ad275364c8b4e525d018f6716537ae7b6d369c094805cae45300847e0894f"}, + {file = "cryptography-41.0.2-cp37-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:84609ade00a6ec59a89729e87a503c6e36af98ddcd566d5f3be52e29ba993182"}, + {file = "cryptography-41.0.2-cp37-abi3-manylinux_2_28_x86_64.whl", hash = "sha256:49c3222bb8f8e800aead2e376cbef687bc9e3cb9b58b29a261210456a7783d83"}, + {file = "cryptography-41.0.2-cp37-abi3-musllinux_1_1_aarch64.whl", hash = "sha256:d73f419a56d74fef257955f51b18d046f3506270a5fd2ac5febbfa259d6c0fa5"}, + {file = "cryptography-41.0.2-cp37-abi3-musllinux_1_1_x86_64.whl", hash = "sha256:2a034bf7d9ca894720f2ec1d8b7b5832d7e363571828037f9e0c4f18c1b58a58"}, + {file = "cryptography-41.0.2-cp37-abi3-win32.whl", hash = "sha256:d124682c7a23c9764e54ca9ab5b308b14b18eba02722b8659fb238546de83a76"}, + {file = "cryptography-41.0.2-cp37-abi3-win_amd64.whl", hash = "sha256:9c3fe6534d59d071ee82081ca3d71eed3210f76ebd0361798c74abc2bcf347d4"}, + {file = "cryptography-41.0.2-pp310-pypy310_pp73-macosx_10_12_x86_64.whl", hash = "sha256:a719399b99377b218dac6cf547b6ec54e6ef20207b6165126a280b0ce97e0d2a"}, + {file = "cryptography-41.0.2-pp310-pypy310_pp73-manylinux_2_28_aarch64.whl", hash = "sha256:182be4171f9332b6741ee818ec27daff9fb00349f706629f5cbf417bd50e66fd"}, + {file = "cryptography-41.0.2-pp310-pypy310_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:7a9a3bced53b7f09da251685224d6a260c3cb291768f54954e28f03ef14e3766"}, + {file = "cryptography-41.0.2-pp310-pypy310_pp73-win_amd64.whl", hash = "sha256:f0dc40e6f7aa37af01aba07277d3d64d5a03dc66d682097541ec4da03cc140ee"}, + {file = "cryptography-41.0.2-pp38-pypy38_pp73-macosx_10_12_x86_64.whl", hash = "sha256:674b669d5daa64206c38e507808aae49904c988fa0a71c935e7006a3e1e83831"}, + {file = "cryptography-41.0.2-pp38-pypy38_pp73-manylinux_2_28_aarch64.whl", hash = "sha256:7af244b012711a26196450d34f483357e42aeddb04128885d95a69bd8b14b69b"}, + {file = "cryptography-41.0.2-pp38-pypy38_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:9b6d717393dbae53d4e52684ef4f022444fc1cce3c48c38cb74fca29e1f08eaa"}, + {file = "cryptography-41.0.2-pp38-pypy38_pp73-win_amd64.whl", hash = "sha256:192255f539d7a89f2102d07d7375b1e0a81f7478925b3bc2e0549ebf739dae0e"}, + {file = "cryptography-41.0.2-pp39-pypy39_pp73-macosx_10_12_x86_64.whl", hash = "sha256:f772610fe364372de33d76edcd313636a25684edb94cee53fd790195f5989d14"}, + {file = "cryptography-41.0.2-pp39-pypy39_pp73-manylinux_2_28_aarch64.whl", hash = "sha256:b332cba64d99a70c1e0836902720887fb4529ea49ea7f5462cf6640e095e11d2"}, + {file = "cryptography-41.0.2-pp39-pypy39_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:9a6673c1828db6270b76b22cc696f40cde9043eb90373da5c2f8f2158957f42f"}, + {file = "cryptography-41.0.2-pp39-pypy39_pp73-win_amd64.whl", hash = "sha256:342f3767e25876751e14f8459ad85e77e660537ca0a066e10e75df9c9e9099f0"}, + {file = "cryptography-41.0.2.tar.gz", hash = "sha256:7d230bf856164de164ecb615ccc14c7fc6de6906ddd5b491f3af90d3514c925c"}, ] [package.dependencies] @@ -689,21 +671,19 @@ test-randomorder = ["pytest-randomly"] [[package]] name = "distlib" -version = "0.3.6" +version = "0.3.7" description = "Distribution utilities" -category = "main" optional = false python-versions = "*" files = [ - {file = "distlib-0.3.6-py2.py3-none-any.whl", hash = "sha256:f35c4b692542ca110de7ef0bea44d73981caeb34ca0b9b6b2e6d7790dda8f80e"}, - {file = "distlib-0.3.6.tar.gz", hash = "sha256:14bad2d9b04d3a36127ac97f30b12a19268f211063d8f8ee4f47108896e11b46"}, + {file = "distlib-0.3.7-py2.py3-none-any.whl", hash = "sha256:2e24928bc811348f0feb63014e97aaae3037f2cf48712d51ae61df7fd6075057"}, + {file = "distlib-0.3.7.tar.gz", hash = "sha256:9dafe54b34a028eafd95039d5e5d4851a13734540f1331060d31c9916e7147a8"}, ] [[package]] name = "docutils" version = "0.20.1" description = "Docutils -- Python Documentation Utilities" -category = "dev" optional = false python-versions = ">=3.7" files = [ @@ -713,71 +693,74 @@ files = [ [[package]] name = "duckdb" -version = "0.8.0" +version = "0.8.1" description = "DuckDB embedded database" -category = "main" optional = true python-versions = "*" files = [ - {file = "duckdb-0.8.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:6455aee00af30770c20f4a8c5e4347918cf59b578f49ee996a13807b12911871"}, - {file = "duckdb-0.8.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:b8cf0622ae7f86d4ce72791f8928af4357a46824aadf1b6879c7936b3db65344"}, - {file = "duckdb-0.8.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:6132e8183ca3ae08a593e43c97cb189794077dedd48546e27ce43bd6a51a9c33"}, - {file = "duckdb-0.8.0-cp310-cp310-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:fe29e5343fa2a95f2cde4519a4f4533f4fd551a48d2d9a8ab5220d40ebf53610"}, - {file = "duckdb-0.8.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:945165987ca87c097dc0e578dcf47a100cad77e1c29f5dd8443d53ce159dc22e"}, - {file = "duckdb-0.8.0-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:673c60daf7ada1d9a8518286a6893ec45efabb64602954af5f3d98f42912fda6"}, - {file = "duckdb-0.8.0-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:d5075fe1ff97ae62331ca5c61e3597e6e9f7682a6fdd418c23ba5c4873ed5cd1"}, - {file = "duckdb-0.8.0-cp310-cp310-win32.whl", hash = "sha256:001f5102f45d3d67f389fa8520046c8f55a99e2c6d43b8e68b38ea93261c5395"}, - {file = "duckdb-0.8.0-cp310-cp310-win_amd64.whl", hash = "sha256:cb00800f2e1e865584b13221e0121fce9341bb3a39a93e569d563eaed281f528"}, - {file = "duckdb-0.8.0-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:b2707096d6df4321044fcde2c9f04da632d11a8be60957fd09d49a42fae71a29"}, - {file = "duckdb-0.8.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:b27df1b70ae74d2c88efb5ffca8490954fdc678099509a9c4404ca30acc53426"}, - {file = "duckdb-0.8.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:75a97c800271b52dd0f37696d074c50576dcb4b2750b6115932a98696a268070"}, - {file = "duckdb-0.8.0-cp311-cp311-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:804cac261a5e016506a6d67838a65d19b06a237f7949f1704f0e800eb708286a"}, - {file = "duckdb-0.8.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c6b9abca7fa6713e1d031c18485343b4de99742c7e1b85c10718aa2f31a4e2c6"}, - {file = "duckdb-0.8.0-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:51aa6d606d49072abcfeb3be209eb559ac94c1b5e70f58ac3adbb94aca9cd69f"}, - {file = "duckdb-0.8.0-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:7c8dc769aaf2be0a1c57995ca657e5b92c1c56fc8437edb720ca6cab571adf14"}, - {file = "duckdb-0.8.0-cp311-cp311-win32.whl", hash = "sha256:c4207d18b42387c4a035846d8878eb967070198be8ac26fd77797ce320d1a400"}, - {file = "duckdb-0.8.0-cp311-cp311-win_amd64.whl", hash = "sha256:0c392257547c20794c3072fcbca99a49ef0a49974005d755e93893e2b4875267"}, - {file = "duckdb-0.8.0-cp36-cp36m-macosx_10_9_x86_64.whl", hash = "sha256:2832379e122020814dbe869af7b9ddf3c9f21474cf345531145b099c63ffe17e"}, - {file = "duckdb-0.8.0-cp36-cp36m-win32.whl", hash = "sha256:914896526f7caba86b170f2c4f17f11fd06540325deeb0000cb4fb24ec732966"}, - {file = "duckdb-0.8.0-cp36-cp36m-win_amd64.whl", hash = "sha256:022ebda86d0e3204cdc206e4af45aa9f0ae0668b34c2c68cf88e08355af4a372"}, - {file = "duckdb-0.8.0-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:96a31c0f3f4ccbf0f5b18f94319f37691205d82f80aae48c6fe04860d743eb2c"}, - {file = "duckdb-0.8.0-cp37-cp37m-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:a07c73c6e6a8cf4ce1a634625e0d1b17e5b817242a8a530d26ed84508dfbdc26"}, - {file = "duckdb-0.8.0-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:424acbd6e857531b06448d757d7c2557938dbddbff0632092090efbf413b4699"}, - {file = "duckdb-0.8.0-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:c83cfd2a868f1acb0692b9c3fd5ef1d7da8faa1348c6eabf421fbf5d8c2f3eb8"}, - {file = "duckdb-0.8.0-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:5c6f6b2d8db56936f662c649539df81856b5a8cb769a31f9544edf18af2a11ff"}, - {file = "duckdb-0.8.0-cp37-cp37m-win32.whl", hash = "sha256:0bd6376b40a512172eaf4aa816813b1b9d68994292ca436ce626ccd5f77f8184"}, - {file = "duckdb-0.8.0-cp37-cp37m-win_amd64.whl", hash = "sha256:931221885bcf1e7dfce2400f11fd048a7beef566b775f1453bb1db89b828e810"}, - {file = "duckdb-0.8.0-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:42e7853d963d68e72403ea208bcf806b0f28c7b44db0aa85ce49bb124d56c133"}, - {file = "duckdb-0.8.0-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:fcc338399175be3d43366576600aef7d72e82114d415992a7a95aded98a0f3fd"}, - {file = "duckdb-0.8.0-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:03dd08a4624d6b581a59f9f9dbfd34902416398d16795ad19f92361cf21fd9b5"}, - {file = "duckdb-0.8.0-cp38-cp38-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:0c7c24ea0c9d8563dbd5ad49ccb54b7a9a3c7b8c2833d35e5d32a08549cacea5"}, - {file = "duckdb-0.8.0-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:cb58f6505cc0f34b4e976154302d26563d2e5d16b206758daaa04b65e55d9dd8"}, - {file = "duckdb-0.8.0-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:ef37ac7880100c4b3f913c8483a29a13f8289313b9a07df019fadfa8e7427544"}, - {file = "duckdb-0.8.0-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:c2a4f5ee913ca8a6a069c78f8944b9934ffdbc71fd935f9576fdcea2a6f476f1"}, - {file = "duckdb-0.8.0-cp38-cp38-win32.whl", hash = "sha256:73831c6d7aefcb5f4072cd677b9efebecbf6c578946d21710791e10a1fc41b9a"}, - {file = "duckdb-0.8.0-cp38-cp38-win_amd64.whl", hash = "sha256:faa36d2854734364d234f37d7ef4f3d763b73cd6b0f799cbc2a0e3b7e2575450"}, - {file = "duckdb-0.8.0-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:50a31ec237ed619e50f9ab79eb0ec5111eb9697d4475da6e0ab22c08495ce26b"}, - {file = "duckdb-0.8.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:351abb4cc2d229d043920c4bc2a4c29ca31a79fef7d7ef8f6011cf4331f297bf"}, - {file = "duckdb-0.8.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:568550a163aca6a787bef8313e358590254de3f4019025a8d68c3a61253fedc1"}, - {file = "duckdb-0.8.0-cp39-cp39-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:2b82617f0e7f9fc080eda217090d82b42d4fad083bc9f6d58dfda9cecb7e3b29"}, - {file = "duckdb-0.8.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d01c9be34d272532b75e8faedda0ff77fa76d1034cde60b8f5768ae85680d6d3"}, - {file = "duckdb-0.8.0-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:8549d6a6bf5f00c012b6916f605416226507e733a3ffc57451682afd6e674d1b"}, - {file = "duckdb-0.8.0-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:8d145c6d51e55743c3ed1a74cffa109d9e72f82b07e203b436cfa453c925313a"}, - {file = "duckdb-0.8.0-cp39-cp39-win32.whl", hash = "sha256:f8610dfd21e90d7b04e8598b244bf3ad68599fd6ba0daad3428c03cbfd74dced"}, - {file = "duckdb-0.8.0-cp39-cp39-win_amd64.whl", hash = "sha256:d0f0f104d30418808bafbe9bccdcd238588a07bd246b3cff13842d60bfd8e8ba"}, - {file = "duckdb-0.8.0.tar.gz", hash = "sha256:c68da35bab5072a64ada2646a5b343da620ddc75a7a6e84aa4a1e0628a7ec18f"}, + {file = "duckdb-0.8.1-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:14781d21580ee72aba1f5dcae7734674c9b6c078dd60470a08b2b420d15b996d"}, + {file = "duckdb-0.8.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:f13bf7ab0e56ddd2014ef762ae4ee5ea4df5a69545ce1191b8d7df8118ba3167"}, + {file = "duckdb-0.8.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:e4032042d8363e55365bbca3faafc6dc336ed2aad088f10ae1a534ebc5bcc181"}, + {file = "duckdb-0.8.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:31a71bd8f0b0ca77c27fa89b99349ef22599ffefe1e7684ae2e1aa2904a08684"}, + {file = "duckdb-0.8.1-cp310-cp310-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:24568d6e48f3dbbf4a933109e323507a46b9399ed24c5d4388c4987ddc694fd0"}, + {file = "duckdb-0.8.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:297226c0dadaa07f7c5ae7cbdb9adba9567db7b16693dbd1b406b739ce0d7924"}, + {file = "duckdb-0.8.1-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:5792cf777ece2c0591194006b4d3e531f720186102492872cb32ddb9363919cf"}, + {file = "duckdb-0.8.1-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:12803f9f41582b68921d6b21f95ba7a51e1d8f36832b7d8006186f58c3d1b344"}, + {file = "duckdb-0.8.1-cp310-cp310-win32.whl", hash = "sha256:d0953d5a2355ddc49095e7aef1392b7f59c5be5cec8cdc98b9d9dc1f01e7ce2b"}, + {file = "duckdb-0.8.1-cp310-cp310-win_amd64.whl", hash = "sha256:6e6583c98a7d6637e83bcadfbd86e1f183917ea539f23b6b41178f32f813a5eb"}, + {file = "duckdb-0.8.1-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:fad7ed0d4415f633d955ac24717fa13a500012b600751d4edb050b75fb940c25"}, + {file = "duckdb-0.8.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:81ae602f34d38d9c48dd60f94b89f28df3ef346830978441b83c5b4eae131d08"}, + {file = "duckdb-0.8.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:7d75cfe563aaa058d3b4ccaaa371c6271e00e3070df5de72361fd161b2fe6780"}, + {file = "duckdb-0.8.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8dbb55e7a3336f2462e5e916fc128c47fe1c03b6208d6bd413ac11ed95132aa0"}, + {file = "duckdb-0.8.1-cp311-cp311-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:a6df53efd63b6fdf04657385a791a4e3c4fb94bfd5db181c4843e2c46b04fef5"}, + {file = "duckdb-0.8.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1b188b80b70d1159b17c9baaf541c1799c1ce8b2af4add179a9eed8e2616be96"}, + {file = "duckdb-0.8.1-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:5ad481ee353f31250b45d64b4a104e53b21415577943aa8f84d0af266dc9af85"}, + {file = "duckdb-0.8.1-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:d1d1b1729993611b1892509d21c21628917625cdbe824a61ce891baadf684b32"}, + {file = "duckdb-0.8.1-cp311-cp311-win32.whl", hash = "sha256:2d8f9cc301e8455a4f89aa1088b8a2d628f0c1f158d4cf9bc78971ed88d82eea"}, + {file = "duckdb-0.8.1-cp311-cp311-win_amd64.whl", hash = "sha256:07457a43605223f62d93d2a5a66b3f97731f79bbbe81fdd5b79954306122f612"}, + {file = "duckdb-0.8.1-cp36-cp36m-macosx_10_9_x86_64.whl", hash = "sha256:d2c8062c3e978dbcd80d712ca3e307de8a06bd4f343aa457d7dd7294692a3842"}, + {file = "duckdb-0.8.1-cp36-cp36m-win32.whl", hash = "sha256:fad486c65ae944eae2de0d590a0a4fb91a9893df98411d66cab03359f9cba39b"}, + {file = "duckdb-0.8.1-cp36-cp36m-win_amd64.whl", hash = "sha256:86fa4506622c52d2df93089c8e7075f1c4d0ba56f4bf27faebde8725355edf32"}, + {file = "duckdb-0.8.1-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:60e07a62782f88420046e30cc0e3de842d0901c4fd5b8e4d28b73826ec0c3f5e"}, + {file = "duckdb-0.8.1-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f18563675977f8cbf03748efee0165b4c8ef64e0cbe48366f78e2914d82138bb"}, + {file = "duckdb-0.8.1-cp37-cp37m-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:16e179443832bea8439ae4dff93cf1e42c545144ead7a4ef5f473e373eea925a"}, + {file = "duckdb-0.8.1-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a413d5267cb41a1afe69d30dd6d4842c588256a6fed7554c7e07dad251ede095"}, + {file = "duckdb-0.8.1-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:3784680df59eadd683b0a4c2375d451a64470ca54bd171c01e36951962b1d332"}, + {file = "duckdb-0.8.1-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:67a1725c2b01f9b53571ecf3f92959b652f60156c1c48fb35798302e39b3c1a2"}, + {file = "duckdb-0.8.1-cp37-cp37m-win32.whl", hash = "sha256:197d37e2588c5ad063e79819054eedb7550d43bf1a557d03ba8f8f67f71acc42"}, + {file = "duckdb-0.8.1-cp37-cp37m-win_amd64.whl", hash = "sha256:3843feb79edf100800f5037c32d5d5a5474fb94b32ace66c707b96605e7c16b2"}, + {file = "duckdb-0.8.1-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:624c889b0f2d656794757b3cc4fc58030d5e285f5ad2ef9fba1ea34a01dab7fb"}, + {file = "duckdb-0.8.1-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:fcbe3742d77eb5add2d617d487266d825e663270ef90253366137a47eaab9448"}, + {file = "duckdb-0.8.1-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:47516c9299d09e9dbba097b9fb339b389313c4941da5c54109df01df0f05e78c"}, + {file = "duckdb-0.8.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:cf1ba718b7522d34399446ebd5d4b9fcac0b56b6ac07bfebf618fd190ec37c1d"}, + {file = "duckdb-0.8.1-cp38-cp38-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e36e35d38a9ae798fe8cf6a839e81494d5b634af89f4ec9483f4d0a313fc6bdb"}, + {file = "duckdb-0.8.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:23493313f88ce6e708a512daacad13e83e6d1ea0be204b175df1348f7fc78671"}, + {file = "duckdb-0.8.1-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:1fb9bf0b6f63616c8a4b9a6a32789045e98c108df100e6bac783dc1e36073737"}, + {file = "duckdb-0.8.1-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:12fc13ecd5eddd28b203b9e3999040d3a7374a8f4b833b04bd26b8c5685c2635"}, + {file = "duckdb-0.8.1-cp38-cp38-win32.whl", hash = "sha256:a12bf4b18306c9cb2c9ba50520317e6cf2de861f121d6f0678505fa83468c627"}, + {file = "duckdb-0.8.1-cp38-cp38-win_amd64.whl", hash = "sha256:e4e809358b9559c00caac4233e0e2014f3f55cd753a31c4bcbbd1b55ad0d35e4"}, + {file = "duckdb-0.8.1-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:7acedfc00d97fbdb8c3d120418c41ef3cb86ef59367f3a9a30dff24470d38680"}, + {file = "duckdb-0.8.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:99bfe264059cdc1e318769103f656f98e819cd4e231cd76c1d1a0327f3e5cef8"}, + {file = "duckdb-0.8.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:538b225f361066231bc6cd66c04a5561de3eea56115a5dd773e99e5d47eb1b89"}, + {file = "duckdb-0.8.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ae0be3f71a18cd8492d05d0fc1bc67d01d5a9457b04822d025b0fc8ee6efe32e"}, + {file = "duckdb-0.8.1-cp39-cp39-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:cd82ba63b58672e46c8ec60bc9946aa4dd7b77f21c1ba09633d8847ad9eb0d7b"}, + {file = "duckdb-0.8.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:780a34559aaec8354e83aa4b7b31b3555f1b2cf75728bf5ce11b89a950f5cdd9"}, + {file = "duckdb-0.8.1-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:01f0d4e9f7103523672bda8d3f77f440b3e0155dd3b2f24997bc0c77f8deb460"}, + {file = "duckdb-0.8.1-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:31f692decb98c2d57891da27180201d9e93bb470a3051fcf413e8da65bca37a5"}, + {file = "duckdb-0.8.1-cp39-cp39-win32.whl", hash = "sha256:e7fe93449cd309bbc67d1bf6f6392a6118e94a9a4479ab8a80518742e855370a"}, + {file = "duckdb-0.8.1-cp39-cp39-win_amd64.whl", hash = "sha256:81d670bc6807672f038332d9bf587037aabdd741b0810de191984325ed307abd"}, + {file = "duckdb-0.8.1.tar.gz", hash = "sha256:a54d37f4abc2afc4f92314aaa56ecf215a411f40af4bffe1e86bd25e62aceee9"}, ] [[package]] name = "exceptiongroup" -version = "1.1.1" +version = "1.1.2" description = "Backport of PEP 654 (exception groups)" -category = "dev" optional = false python-versions = ">=3.7" files = [ - {file = "exceptiongroup-1.1.1-py3-none-any.whl", hash = "sha256:232c37c63e4f682982c8b6459f33a8981039e5fb8756b2074364e5055c498c9e"}, - {file = "exceptiongroup-1.1.1.tar.gz", hash = "sha256:d484c3090ba2889ae2928419117447a14daf3c1231d5e30d0aae34f354f01785"}, + {file = "exceptiongroup-1.1.2-py3-none-any.whl", hash = "sha256:e346e69d186172ca7cf029c8c1d16235aa0e04035e5750b4b95039e65204328f"}, + {file = "exceptiongroup-1.1.2.tar.gz", hash = "sha256:12c3e887d6485d16943a309616de20ae5582633e0a2eda17f4e10fd61c1e8af5"}, ] [package.extras] @@ -785,43 +768,36 @@ test = ["pytest (>=6)"] [[package]] name = "fastavro" -version = "1.7.4" +version = "1.8.2" description = "Fast read/write of AVRO files" -category = "dev" optional = false -python-versions = ">=3.7" +python-versions = ">=3.8" files = [ - {file = "fastavro-1.7.4-cp310-cp310-macosx_11_0_x86_64.whl", hash = "sha256:7568e621b94e061974b2a96d70670d09910e0a71482dd8610b153c07bd768497"}, - {file = "fastavro-1.7.4-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d4ec994faf64b743647f0027fcc56b01dc15d46c0e48fa15828277cb02dbdcd6"}, - {file = "fastavro-1.7.4-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:727fdc1ddd12fcc6addab0b6df12ef999a6babe4b753db891f78aa2ee33edc77"}, - {file = "fastavro-1.7.4-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:b2f0cb3f7795fcb0042e0bbbe51204c28338a455986d68409b26dcbde64dd69a"}, - {file = "fastavro-1.7.4-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:bb0a8b5016a99be4b8ce3550889a1bd968c0fb3f521bcfbae24210c6342aee0c"}, - {file = "fastavro-1.7.4-cp310-cp310-win_amd64.whl", hash = "sha256:1d2040b2bf3dc1a75170ea44d1e7e09f84fb77f40ef2e6c6b9f2eaf710557083"}, - {file = "fastavro-1.7.4-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:5542423f46bb7fc9699c467cbf151c2713aa6976ef14f4f5ec3532d80d0bb616"}, - {file = "fastavro-1.7.4-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ec396e6ab6b272708c8b9a0142df01fff4c7a1f168050f292ab92fdaee0b0257"}, - {file = "fastavro-1.7.4-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:39b10d68c03371b79f461feca1c6c7e9d3f6aea2e9c7472b25cd749c57562aa1"}, - {file = "fastavro-1.7.4-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:f94d5168ec72f3cfcf2181df1c46ad240dc1fcf361717447d2c5237121b9df55"}, - {file = "fastavro-1.7.4-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:bad3dc279ed4ce747989259035cb3607f189ef7aff40339202f9321ca7f83d0b"}, - {file = "fastavro-1.7.4-cp311-cp311-win_amd64.whl", hash = "sha256:8480ff444d9c7abd0bf121dd68656bd2115caca8ed28e71936eff348fde706e0"}, - {file = "fastavro-1.7.4-cp37-cp37m-macosx_10_15_x86_64.whl", hash = "sha256:bd3d669f4ec6915c88bb80b7c14e01d2c3ceb93a61de5dcf33ff13972bba505e"}, - {file = "fastavro-1.7.4-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1a312b128536b81bdb79f27076f513b998abe7d13ee6fe52e99bc01f7ad9b06a"}, - {file = "fastavro-1.7.4-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:487054d1419f1bfa41e7f19c718cbdbbb254319d3fd5b9ac411054d6432b9d40"}, - {file = "fastavro-1.7.4-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:d2897fe7d1d5b27dcd33c43d68480de36e55a0e651d7731004a36162cd3eed9e"}, - {file = "fastavro-1.7.4-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:6d318b49fd648a1fd93394411fe23761b486ac65dadea7c52dbeb0d0bef30221"}, - {file = "fastavro-1.7.4-cp37-cp37m-win_amd64.whl", hash = "sha256:a117c3b122a8110c6ab99b3e66736790b4be19ceefb1edf0e732c33b3dc411c8"}, - {file = "fastavro-1.7.4-cp38-cp38-macosx_10_15_x86_64.whl", hash = "sha256:0cca15e1a1f829e40524004342e425acfb594cefbd3388b0a5d13542750623ac"}, - {file = "fastavro-1.7.4-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f9211ec7a18a46a2aee01a2a979fd79f05f36b11fdb1bc469c9d9fd8cec32579"}, - {file = "fastavro-1.7.4-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f16bde6b5fb51e15233bfcee0378f48d4221201ba45e497a8063f6d216b7aad7"}, - {file = "fastavro-1.7.4-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:aeca55c905ff4c667f2158564654a778918988811ae3eb28592767edcf5f5c4a"}, - {file = "fastavro-1.7.4-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:b244f3abc024fc043d6637284ba2ffee5a1291c08a0f361ea1af4d829f66f303"}, - {file = "fastavro-1.7.4-cp38-cp38-win_amd64.whl", hash = "sha256:b64e394c87cb99d0681727e1ae5d3633906a72abeab5ea0c692394aeb5a56607"}, - {file = "fastavro-1.7.4-cp39-cp39-macosx_11_0_x86_64.whl", hash = "sha256:8c8115bdb1c862354d9abd0ea23eab85793bbff139087f2607bd4b83e8ae07ab"}, - {file = "fastavro-1.7.4-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b27dd08f2338a478185c6ba23308002f334642ce83a6aeaf8308271efef88062"}, - {file = "fastavro-1.7.4-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f087c246afab8bac08d86ef21be87cbf4f3779348fb960c081863fc3d570412c"}, - {file = "fastavro-1.7.4-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:b4077e17a2bab37af96e5ca52e61b6f2b85e4577e7a2903f6814642eb6a834f7"}, - {file = "fastavro-1.7.4-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:776511cecf2ea9da4edd0de5015c1562cd9063683cf94f79bc9e20bab8f06923"}, - {file = "fastavro-1.7.4-cp39-cp39-win_amd64.whl", hash = "sha256:a7ea5565fe2c145e074ce9ba75fafd5479a86b34a8dbd00dd1835cf192290e14"}, - {file = "fastavro-1.7.4.tar.gz", hash = "sha256:6450f47ac4db95ec3a9e6434fec1f8a3c4c8c941de16205832ca8c67dd23d0d2"}, + {file = "fastavro-1.8.2-cp310-cp310-macosx_11_0_x86_64.whl", hash = "sha256:0e08964b2e9a455d831f2557402a683d4c4d45206f2ab9ade7c69d3dc14e0e58"}, + {file = "fastavro-1.8.2-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:401a70b1e5c7161420c6019e0c8afa88f7c8a373468591f5ec37639a903c2509"}, + {file = "fastavro-1.8.2-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:eef1ed3eaa4240c05698d02d8d0c010b9a03780eda37b492da6cd4c9d37e04ec"}, + {file = "fastavro-1.8.2-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:543185a672ff6306beb329b57a7b8a3a2dd1eb21a5ccc530150623d58d48bb98"}, + {file = "fastavro-1.8.2-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:ffbf8bae1edb50fe7beeffc3afa8e684686550c2e5d31bf01c25cfa213f581e1"}, + {file = "fastavro-1.8.2-cp310-cp310-win_amd64.whl", hash = "sha256:bb545eb9d876bc7b785e27e98e7720ada7eee7d7a1729798d2ed51517f13500a"}, + {file = "fastavro-1.8.2-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:2b837d3038c651046252bc92c1b9899bf21c7927a148a1ff89599c36c2a331ca"}, + {file = "fastavro-1.8.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d3510e96c0a47e4e914bd1a29c954eb662bfa24849ad92e597cb97cc79f21af7"}, + {file = "fastavro-1.8.2-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ccc0e74f2c2ab357f39bb73d67fcdb6dc10e23fdbbd399326139f72ec0fb99a3"}, + {file = "fastavro-1.8.2-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:add51c70d0ab1175601c75cd687bbe9d16ae312cd8899b907aafe0d79ee2bc1d"}, + {file = "fastavro-1.8.2-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:d9e2662f57e6453e9a2c9fb4f54b2a9e62e3e46f5a412ac00558112336d23883"}, + {file = "fastavro-1.8.2-cp311-cp311-win_amd64.whl", hash = "sha256:fea75cf53a93c56dd56e68abce8d314ef877b27451c870cd7ede7582d34c08a7"}, + {file = "fastavro-1.8.2-cp38-cp38-macosx_11_0_x86_64.whl", hash = "sha256:f489020bb8664c2737c03457ad5dbd490579ddab6f0a7b5c17fecfe982715a89"}, + {file = "fastavro-1.8.2-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a547625c138efd5e61300119241041906ee8cb426fc7aa789900f87af7ed330d"}, + {file = "fastavro-1.8.2-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:53beb458f30c9ad4aa7bff4a42243ff990ffb713b6ce0cd9b360cbc3d648fe52"}, + {file = "fastavro-1.8.2-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:7b1b2cbd2dd851452306beed0ab9bdaeeab1cc8ad46f84b47cd81eeaff6dd6b8"}, + {file = "fastavro-1.8.2-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:d29e9baee0b2f37ecd09bde3b487cf900431fd548c85be3e4fe1b9a0b2a917f1"}, + {file = "fastavro-1.8.2-cp38-cp38-win_amd64.whl", hash = "sha256:66e132c710663230292bc63e2cb79cf95b16ccb94a5fc99bb63694b24e312fc5"}, + {file = "fastavro-1.8.2-cp39-cp39-macosx_11_0_x86_64.whl", hash = "sha256:38aca63ce604039bcdf2edd14912d00287bdbf8b76f9aa42b28e6ca0bf950092"}, + {file = "fastavro-1.8.2-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:9787835f6449ee94713e7993a700432fce3763024791ffa8a58dc91ef9d1f950"}, + {file = "fastavro-1.8.2-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:536cb448bc83811056be02749fd9df37a69621678f02597d272970a769e9b40c"}, + {file = "fastavro-1.8.2-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:e9d5027cf7d9968f8f819958b41bfedb933323ea6d6a0485eefacaa1afd91f54"}, + {file = "fastavro-1.8.2-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:792adfc0c80c7f1109e0ab4b0decef20691fdf0a45091d397a0563872eb56d42"}, + {file = "fastavro-1.8.2-cp39-cp39-win_amd64.whl", hash = "sha256:650b22766259f7dd7519dfa4e4658f0e233c319efa130b9cf0c36a500e09cc57"}, + {file = "fastavro-1.8.2.tar.gz", hash = "sha256:ab9d9226d4b66b6b3d0661a57cd45259b0868fed1c0cd4fac95249b9e0973320"}, ] [package.extras] @@ -832,114 +808,98 @@ zstandard = ["zstandard"] [[package]] name = "filelock" -version = "3.12.0" +version = "3.12.2" description = "A platform independent file lock." -category = "main" optional = false python-versions = ">=3.7" files = [ - {file = "filelock-3.12.0-py3-none-any.whl", hash = "sha256:ad98852315c2ab702aeb628412cbf7e95b7ce8c3bf9565670b4eaecf1db370a9"}, - {file = "filelock-3.12.0.tar.gz", hash = "sha256:fc03ae43288c013d2ea83c8597001b1129db351aad9c57fe2409327916b8e718"}, + {file = "filelock-3.12.2-py3-none-any.whl", hash = "sha256:cbb791cdea2a72f23da6ac5b5269ab0a0d161e9ef0100e653b69049a7706d1ec"}, + {file = "filelock-3.12.2.tar.gz", hash = "sha256:002740518d8aa59a26b0c76e10fb8c6e15eae825d34b6fdf670333fd7b938d81"}, ] [package.extras] -docs = ["furo (>=2023.3.27)", "sphinx (>=6.1.3)", "sphinx-autodoc-typehints (>=1.23,!=1.23.4)"] -testing = ["covdefaults (>=2.3)", "coverage (>=7.2.3)", "diff-cover (>=7.5)", "pytest (>=7.3.1)", "pytest-cov (>=4)", "pytest-mock (>=3.10)", "pytest-timeout (>=2.1)"] +docs = ["furo (>=2023.5.20)", "sphinx (>=7.0.1)", "sphinx-autodoc-typehints (>=1.23,!=1.23.4)"] +testing = ["covdefaults (>=2.3)", "coverage (>=7.2.7)", "diff-cover (>=7.5)", "pytest (>=7.3.1)", "pytest-cov (>=4.1)", "pytest-mock (>=3.10)", "pytest-timeout (>=2.1)"] [[package]] name = "frozenlist" -version = "1.3.3" +version = "1.4.0" description = "A list-like structure which implements collections.abc.MutableSequence" -category = "main" optional = true -python-versions = ">=3.7" +python-versions = ">=3.8" files = [ - {file = "frozenlist-1.3.3-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:ff8bf625fe85e119553b5383ba0fb6aa3d0ec2ae980295aaefa552374926b3f4"}, - {file = "frozenlist-1.3.3-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:dfbac4c2dfcc082fcf8d942d1e49b6aa0766c19d3358bd86e2000bf0fa4a9cf0"}, - {file = "frozenlist-1.3.3-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:b1c63e8d377d039ac769cd0926558bb7068a1f7abb0f003e3717ee003ad85530"}, - {file = "frozenlist-1.3.3-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:7fdfc24dcfce5b48109867c13b4cb15e4660e7bd7661741a391f821f23dfdca7"}, - {file = "frozenlist-1.3.3-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:2c926450857408e42f0bbc295e84395722ce74bae69a3b2aa2a65fe22cb14b99"}, - {file = "frozenlist-1.3.3-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:1841e200fdafc3d51f974d9d377c079a0694a8f06de2e67b48150328d66d5483"}, - {file = "frozenlist-1.3.3-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f470c92737afa7d4c3aacc001e335062d582053d4dbe73cda126f2d7031068dd"}, - {file = "frozenlist-1.3.3-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:783263a4eaad7c49983fe4b2e7b53fa9770c136c270d2d4bbb6d2192bf4d9caf"}, - {file = "frozenlist-1.3.3-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:924620eef691990dfb56dc4709f280f40baee568c794b5c1885800c3ecc69816"}, - {file = "frozenlist-1.3.3-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:ae4dc05c465a08a866b7a1baf360747078b362e6a6dbeb0c57f234db0ef88ae0"}, - {file = "frozenlist-1.3.3-cp310-cp310-musllinux_1_1_ppc64le.whl", hash = "sha256:bed331fe18f58d844d39ceb398b77d6ac0b010d571cba8267c2e7165806b00ce"}, - {file = "frozenlist-1.3.3-cp310-cp310-musllinux_1_1_s390x.whl", hash = "sha256:02c9ac843e3390826a265e331105efeab489ffaf4dd86384595ee8ce6d35ae7f"}, - {file = "frozenlist-1.3.3-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:9545a33965d0d377b0bc823dcabf26980e77f1b6a7caa368a365a9497fb09420"}, - {file = "frozenlist-1.3.3-cp310-cp310-win32.whl", hash = "sha256:d5cd3ab21acbdb414bb6c31958d7b06b85eeb40f66463c264a9b343a4e238642"}, - {file = "frozenlist-1.3.3-cp310-cp310-win_amd64.whl", hash = "sha256:b756072364347cb6aa5b60f9bc18e94b2f79632de3b0190253ad770c5df17db1"}, - {file = "frozenlist-1.3.3-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:b4395e2f8d83fbe0c627b2b696acce67868793d7d9750e90e39592b3626691b7"}, - {file = "frozenlist-1.3.3-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:14143ae966a6229350021384870458e4777d1eae4c28d1a7aa47f24d030e6678"}, - {file = "frozenlist-1.3.3-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:5d8860749e813a6f65bad8285a0520607c9500caa23fea6ee407e63debcdbef6"}, - {file = "frozenlist-1.3.3-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:23d16d9f477bb55b6154654e0e74557040575d9d19fe78a161bd33d7d76808e8"}, - {file = "frozenlist-1.3.3-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:eb82dbba47a8318e75f679690190c10a5e1f447fbf9df41cbc4c3afd726d88cb"}, - {file = "frozenlist-1.3.3-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:9309869032abb23d196cb4e4db574232abe8b8be1339026f489eeb34a4acfd91"}, - {file = "frozenlist-1.3.3-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:a97b4fe50b5890d36300820abd305694cb865ddb7885049587a5678215782a6b"}, - {file = "frozenlist-1.3.3-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c188512b43542b1e91cadc3c6c915a82a5eb95929134faf7fd109f14f9892ce4"}, - {file = "frozenlist-1.3.3-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:303e04d422e9b911a09ad499b0368dc551e8c3cd15293c99160c7f1f07b59a48"}, - {file = "frozenlist-1.3.3-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:0771aed7f596c7d73444c847a1c16288937ef988dc04fb9f7be4b2aa91db609d"}, - {file = "frozenlist-1.3.3-cp311-cp311-musllinux_1_1_ppc64le.whl", hash = "sha256:66080ec69883597e4d026f2f71a231a1ee9887835902dbe6b6467d5a89216cf6"}, - {file = "frozenlist-1.3.3-cp311-cp311-musllinux_1_1_s390x.whl", hash = "sha256:41fe21dc74ad3a779c3d73a2786bdf622ea81234bdd4faf90b8b03cad0c2c0b4"}, - {file = "frozenlist-1.3.3-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:f20380df709d91525e4bee04746ba612a4df0972c1b8f8e1e8af997e678c7b81"}, - {file = "frozenlist-1.3.3-cp311-cp311-win32.whl", hash = "sha256:f30f1928162e189091cf4d9da2eac617bfe78ef907a761614ff577ef4edfb3c8"}, - {file = "frozenlist-1.3.3-cp311-cp311-win_amd64.whl", hash = "sha256:a6394d7dadd3cfe3f4b3b186e54d5d8504d44f2d58dcc89d693698e8b7132b32"}, - {file = "frozenlist-1.3.3-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:8df3de3a9ab8325f94f646609a66cbeeede263910c5c0de0101079ad541af332"}, - {file = "frozenlist-1.3.3-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0693c609e9742c66ba4870bcee1ad5ff35462d5ffec18710b4ac89337ff16e27"}, - {file = "frozenlist-1.3.3-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:cd4210baef299717db0a600d7a3cac81d46ef0e007f88c9335db79f8979c0d3d"}, - {file = "frozenlist-1.3.3-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:394c9c242113bfb4b9aa36e2b80a05ffa163a30691c7b5a29eba82e937895d5e"}, - {file = "frozenlist-1.3.3-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:6327eb8e419f7d9c38f333cde41b9ae348bec26d840927332f17e887a8dcb70d"}, - {file = "frozenlist-1.3.3-cp37-cp37m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2e24900aa13212e75e5b366cb9065e78bbf3893d4baab6052d1aca10d46d944c"}, - {file = "frozenlist-1.3.3-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:3843f84a6c465a36559161e6c59dce2f2ac10943040c2fd021cfb70d58c4ad56"}, - {file = "frozenlist-1.3.3-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:84610c1502b2461255b4c9b7d5e9c48052601a8957cd0aea6ec7a7a1e1fb9420"}, - {file = "frozenlist-1.3.3-cp37-cp37m-musllinux_1_1_ppc64le.whl", hash = "sha256:c21b9aa40e08e4f63a2f92ff3748e6b6c84d717d033c7b3438dd3123ee18f70e"}, - {file = "frozenlist-1.3.3-cp37-cp37m-musllinux_1_1_s390x.whl", hash = "sha256:efce6ae830831ab6a22b9b4091d411698145cb9b8fc869e1397ccf4b4b6455cb"}, - {file = "frozenlist-1.3.3-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:40de71985e9042ca00b7953c4f41eabc3dc514a2d1ff534027f091bc74416401"}, - {file = "frozenlist-1.3.3-cp37-cp37m-win32.whl", hash = "sha256:180c00c66bde6146a860cbb81b54ee0df350d2daf13ca85b275123bbf85de18a"}, - {file = "frozenlist-1.3.3-cp37-cp37m-win_amd64.whl", hash = "sha256:9bbbcedd75acdfecf2159663b87f1bb5cfc80e7cd99f7ddd9d66eb98b14a8411"}, - {file = "frozenlist-1.3.3-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:034a5c08d36649591be1cbb10e09da9f531034acfe29275fc5454a3b101ce41a"}, - {file = "frozenlist-1.3.3-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:ba64dc2b3b7b158c6660d49cdb1d872d1d0bf4e42043ad8d5006099479a194e5"}, - {file = "frozenlist-1.3.3-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:47df36a9fe24054b950bbc2db630d508cca3aa27ed0566c0baf661225e52c18e"}, - {file = "frozenlist-1.3.3-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:008a054b75d77c995ea26629ab3a0c0d7281341f2fa7e1e85fa6153ae29ae99c"}, - {file = "frozenlist-1.3.3-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:841ea19b43d438a80b4de62ac6ab21cfe6827bb8a9dc62b896acc88eaf9cecba"}, - {file = "frozenlist-1.3.3-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:e235688f42b36be2b6b06fc37ac2126a73b75fb8d6bc66dd632aa35286238703"}, - {file = "frozenlist-1.3.3-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:ca713d4af15bae6e5d79b15c10c8522859a9a89d3b361a50b817c98c2fb402a2"}, - {file = "frozenlist-1.3.3-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9ac5995f2b408017b0be26d4a1d7c61bce106ff3d9e3324374d66b5964325448"}, - {file = "frozenlist-1.3.3-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:a4ae8135b11652b08a8baf07631d3ebfe65a4c87909dbef5fa0cdde440444ee4"}, - {file = "frozenlist-1.3.3-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:4ea42116ceb6bb16dbb7d526e242cb6747b08b7710d9782aa3d6732bd8d27649"}, - {file = "frozenlist-1.3.3-cp38-cp38-musllinux_1_1_ppc64le.whl", hash = "sha256:810860bb4bdce7557bc0febb84bbd88198b9dbc2022d8eebe5b3590b2ad6c842"}, - {file = "frozenlist-1.3.3-cp38-cp38-musllinux_1_1_s390x.whl", hash = "sha256:ee78feb9d293c323b59a6f2dd441b63339a30edf35abcb51187d2fc26e696d13"}, - {file = "frozenlist-1.3.3-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:0af2e7c87d35b38732e810befb9d797a99279cbb85374d42ea61c1e9d23094b3"}, - {file = "frozenlist-1.3.3-cp38-cp38-win32.whl", hash = "sha256:899c5e1928eec13fd6f6d8dc51be23f0d09c5281e40d9cf4273d188d9feeaf9b"}, - {file = "frozenlist-1.3.3-cp38-cp38-win_amd64.whl", hash = "sha256:7f44e24fa70f6fbc74aeec3e971f60a14dde85da364aa87f15d1be94ae75aeef"}, - {file = "frozenlist-1.3.3-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:2b07ae0c1edaa0a36339ec6cce700f51b14a3fc6545fdd32930d2c83917332cf"}, - {file = "frozenlist-1.3.3-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:ebb86518203e12e96af765ee89034a1dbb0c3c65052d1b0c19bbbd6af8a145e1"}, - {file = "frozenlist-1.3.3-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:5cf820485f1b4c91e0417ea0afd41ce5cf5965011b3c22c400f6d144296ccbc0"}, - {file = "frozenlist-1.3.3-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5c11e43016b9024240212d2a65043b70ed8dfd3b52678a1271972702d990ac6d"}, - {file = "frozenlist-1.3.3-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:8fa3c6e3305aa1146b59a09b32b2e04074945ffcfb2f0931836d103a2c38f936"}, - {file = "frozenlist-1.3.3-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:352bd4c8c72d508778cf05ab491f6ef36149f4d0cb3c56b1b4302852255d05d5"}, - {file = "frozenlist-1.3.3-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:65a5e4d3aa679610ac6e3569e865425b23b372277f89b5ef06cf2cdaf1ebf22b"}, - {file = "frozenlist-1.3.3-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b1e2c1185858d7e10ff045c496bbf90ae752c28b365fef2c09cf0fa309291669"}, - {file = "frozenlist-1.3.3-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:f163d2fd041c630fed01bc48d28c3ed4a3b003c00acd396900e11ee5316b56bb"}, - {file = "frozenlist-1.3.3-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:05cdb16d09a0832eedf770cb7bd1fe57d8cf4eaf5aced29c4e41e3f20b30a784"}, - {file = "frozenlist-1.3.3-cp39-cp39-musllinux_1_1_ppc64le.whl", hash = "sha256:8bae29d60768bfa8fb92244b74502b18fae55a80eac13c88eb0b496d4268fd2d"}, - {file = "frozenlist-1.3.3-cp39-cp39-musllinux_1_1_s390x.whl", hash = "sha256:eedab4c310c0299961ac285591acd53dc6723a1ebd90a57207c71f6e0c2153ab"}, - {file = "frozenlist-1.3.3-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:3bbdf44855ed8f0fbcd102ef05ec3012d6a4fd7c7562403f76ce6a52aeffb2b1"}, - {file = "frozenlist-1.3.3-cp39-cp39-win32.whl", hash = "sha256:efa568b885bca461f7c7b9e032655c0c143d305bf01c30caf6db2854a4532b38"}, - {file = "frozenlist-1.3.3-cp39-cp39-win_amd64.whl", hash = "sha256:cfe33efc9cb900a4c46f91a5ceba26d6df370ffddd9ca386eb1d4f0ad97b9ea9"}, - {file = "frozenlist-1.3.3.tar.gz", hash = "sha256:58bcc55721e8a90b88332d6cd441261ebb22342e238296bb330968952fbb3a6a"}, + {file = "frozenlist-1.4.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:764226ceef3125e53ea2cb275000e309c0aa5464d43bd72abd661e27fffc26ab"}, + {file = "frozenlist-1.4.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:d6484756b12f40003c6128bfcc3fa9f0d49a687e171186c2d85ec82e3758c559"}, + {file = "frozenlist-1.4.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:9ac08e601308e41eb533f232dbf6b7e4cea762f9f84f6357136eed926c15d12c"}, + {file = "frozenlist-1.4.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d081f13b095d74b67d550de04df1c756831f3b83dc9881c38985834387487f1b"}, + {file = "frozenlist-1.4.0-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:71932b597f9895f011f47f17d6428252fc728ba2ae6024e13c3398a087c2cdea"}, + {file = "frozenlist-1.4.0-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:981b9ab5a0a3178ff413bca62526bb784249421c24ad7381e39d67981be2c326"}, + {file = "frozenlist-1.4.0-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e41f3de4df3e80de75845d3e743b3f1c4c8613c3997a912dbf0229fc61a8b963"}, + {file = "frozenlist-1.4.0-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6918d49b1f90821e93069682c06ffde41829c346c66b721e65a5c62b4bab0300"}, + {file = "frozenlist-1.4.0-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:0e5c8764c7829343d919cc2dfc587a8db01c4f70a4ebbc49abde5d4b158b007b"}, + {file = "frozenlist-1.4.0-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:8d0edd6b1c7fb94922bf569c9b092ee187a83f03fb1a63076e7774b60f9481a8"}, + {file = "frozenlist-1.4.0-cp310-cp310-musllinux_1_1_ppc64le.whl", hash = "sha256:e29cda763f752553fa14c68fb2195150bfab22b352572cb36c43c47bedba70eb"}, + {file = "frozenlist-1.4.0-cp310-cp310-musllinux_1_1_s390x.whl", hash = "sha256:0c7c1b47859ee2cac3846fde1c1dc0f15da6cec5a0e5c72d101e0f83dcb67ff9"}, + {file = "frozenlist-1.4.0-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:901289d524fdd571be1c7be054f48b1f88ce8dddcbdf1ec698b27d4b8b9e5d62"}, + {file = "frozenlist-1.4.0-cp310-cp310-win32.whl", hash = "sha256:1a0848b52815006ea6596c395f87449f693dc419061cc21e970f139d466dc0a0"}, + {file = "frozenlist-1.4.0-cp310-cp310-win_amd64.whl", hash = "sha256:b206646d176a007466358aa21d85cd8600a415c67c9bd15403336c331a10d956"}, + {file = "frozenlist-1.4.0-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:de343e75f40e972bae1ef6090267f8260c1446a1695e77096db6cfa25e759a95"}, + {file = "frozenlist-1.4.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:ad2a9eb6d9839ae241701d0918f54c51365a51407fd80f6b8289e2dfca977cc3"}, + {file = "frozenlist-1.4.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:bd7bd3b3830247580de99c99ea2a01416dfc3c34471ca1298bccabf86d0ff4dc"}, + {file = "frozenlist-1.4.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:bdf1847068c362f16b353163391210269e4f0569a3c166bc6a9f74ccbfc7e839"}, + {file = "frozenlist-1.4.0-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:38461d02d66de17455072c9ba981d35f1d2a73024bee7790ac2f9e361ef1cd0c"}, + {file = "frozenlist-1.4.0-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:d5a32087d720c608f42caed0ef36d2b3ea61a9d09ee59a5142d6070da9041b8f"}, + {file = "frozenlist-1.4.0-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:dd65632acaf0d47608190a71bfe46b209719bf2beb59507db08ccdbe712f969b"}, + {file = "frozenlist-1.4.0-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:261b9f5d17cac914531331ff1b1d452125bf5daa05faf73b71d935485b0c510b"}, + {file = "frozenlist-1.4.0-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:b89ac9768b82205936771f8d2eb3ce88503b1556324c9f903e7156669f521472"}, + {file = "frozenlist-1.4.0-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:008eb8b31b3ea6896da16c38c1b136cb9fec9e249e77f6211d479db79a4eaf01"}, + {file = "frozenlist-1.4.0-cp311-cp311-musllinux_1_1_ppc64le.whl", hash = "sha256:e74b0506fa5aa5598ac6a975a12aa8928cbb58e1f5ac8360792ef15de1aa848f"}, + {file = "frozenlist-1.4.0-cp311-cp311-musllinux_1_1_s390x.whl", hash = "sha256:490132667476f6781b4c9458298b0c1cddf237488abd228b0b3650e5ecba7467"}, + {file = "frozenlist-1.4.0-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:76d4711f6f6d08551a7e9ef28c722f4a50dd0fc204c56b4bcd95c6cc05ce6fbb"}, + {file = "frozenlist-1.4.0-cp311-cp311-win32.whl", hash = "sha256:a02eb8ab2b8f200179b5f62b59757685ae9987996ae549ccf30f983f40602431"}, + {file = "frozenlist-1.4.0-cp311-cp311-win_amd64.whl", hash = "sha256:515e1abc578dd3b275d6a5114030b1330ba044ffba03f94091842852f806f1c1"}, + {file = "frozenlist-1.4.0-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:f0ed05f5079c708fe74bf9027e95125334b6978bf07fd5ab923e9e55e5fbb9d3"}, + {file = "frozenlist-1.4.0-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:ca265542ca427bf97aed183c1676e2a9c66942e822b14dc6e5f42e038f92a503"}, + {file = "frozenlist-1.4.0-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:491e014f5c43656da08958808588cc6c016847b4360e327a62cb308c791bd2d9"}, + {file = "frozenlist-1.4.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:17ae5cd0f333f94f2e03aaf140bb762c64783935cc764ff9c82dff626089bebf"}, + {file = "frozenlist-1.4.0-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:1e78fb68cf9c1a6aa4a9a12e960a5c9dfbdb89b3695197aa7064705662515de2"}, + {file = "frozenlist-1.4.0-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:d5655a942f5f5d2c9ed93d72148226d75369b4f6952680211972a33e59b1dfdc"}, + {file = "frozenlist-1.4.0-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c11b0746f5d946fecf750428a95f3e9ebe792c1ee3b1e96eeba145dc631a9672"}, + {file = "frozenlist-1.4.0-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e66d2a64d44d50d2543405fb183a21f76b3b5fd16f130f5c99187c3fb4e64919"}, + {file = "frozenlist-1.4.0-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:88f7bc0fcca81f985f78dd0fa68d2c75abf8272b1f5c323ea4a01a4d7a614efc"}, + {file = "frozenlist-1.4.0-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:5833593c25ac59ede40ed4de6d67eb42928cca97f26feea219f21d0ed0959b79"}, + {file = "frozenlist-1.4.0-cp38-cp38-musllinux_1_1_ppc64le.whl", hash = "sha256:fec520865f42e5c7f050c2a79038897b1c7d1595e907a9e08e3353293ffc948e"}, + {file = "frozenlist-1.4.0-cp38-cp38-musllinux_1_1_s390x.whl", hash = "sha256:b826d97e4276750beca7c8f0f1a4938892697a6bcd8ec8217b3312dad6982781"}, + {file = "frozenlist-1.4.0-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:ceb6ec0a10c65540421e20ebd29083c50e6d1143278746a4ef6bcf6153171eb8"}, + {file = "frozenlist-1.4.0-cp38-cp38-win32.whl", hash = "sha256:2b8bcf994563466db019fab287ff390fffbfdb4f905fc77bc1c1d604b1c689cc"}, + {file = "frozenlist-1.4.0-cp38-cp38-win_amd64.whl", hash = "sha256:a6c8097e01886188e5be3e6b14e94ab365f384736aa1fca6a0b9e35bd4a30bc7"}, + {file = "frozenlist-1.4.0-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:6c38721585f285203e4b4132a352eb3daa19121a035f3182e08e437cface44bf"}, + {file = "frozenlist-1.4.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:a0c6da9aee33ff0b1a451e867da0c1f47408112b3391dd43133838339e410963"}, + {file = "frozenlist-1.4.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:93ea75c050c5bb3d98016b4ba2497851eadf0ac154d88a67d7a6816206f6fa7f"}, + {file = "frozenlist-1.4.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f61e2dc5ad442c52b4887f1fdc112f97caeff4d9e6ebe78879364ac59f1663e1"}, + {file = "frozenlist-1.4.0-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:aa384489fefeb62321b238e64c07ef48398fe80f9e1e6afeff22e140e0850eef"}, + {file = "frozenlist-1.4.0-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:10ff5faaa22786315ef57097a279b833ecab1a0bfb07d604c9cbb1c4cdc2ed87"}, + {file = "frozenlist-1.4.0-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:007df07a6e3eb3e33e9a1fe6a9db7af152bbd8a185f9aaa6ece10a3529e3e1c6"}, + {file = "frozenlist-1.4.0-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7f4f399d28478d1f604c2ff9119907af9726aed73680e5ed1ca634d377abb087"}, + {file = "frozenlist-1.4.0-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:c5374b80521d3d3f2ec5572e05adc94601985cc526fb276d0c8574a6d749f1b3"}, + {file = "frozenlist-1.4.0-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:ce31ae3e19f3c902de379cf1323d90c649425b86de7bbdf82871b8a2a0615f3d"}, + {file = "frozenlist-1.4.0-cp39-cp39-musllinux_1_1_ppc64le.whl", hash = "sha256:7211ef110a9194b6042449431e08c4d80c0481e5891e58d429df5899690511c2"}, + {file = "frozenlist-1.4.0-cp39-cp39-musllinux_1_1_s390x.whl", hash = "sha256:556de4430ce324c836789fa4560ca62d1591d2538b8ceb0b4f68fb7b2384a27a"}, + {file = "frozenlist-1.4.0-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:7645a8e814a3ee34a89c4a372011dcd817964ce8cb273c8ed6119d706e9613e3"}, + {file = "frozenlist-1.4.0-cp39-cp39-win32.whl", hash = "sha256:19488c57c12d4e8095a922f328df3f179c820c212940a498623ed39160bc3c2f"}, + {file = "frozenlist-1.4.0-cp39-cp39-win_amd64.whl", hash = "sha256:6221d84d463fb110bdd7619b69cb43878a11d51cbb9394ae3105d082d5199167"}, + {file = "frozenlist-1.4.0.tar.gz", hash = "sha256:09163bdf0b2907454042edb19f887c6d33806adc71fbd54afc14908bfdc22251"}, ] [[package]] name = "fsspec" -version = "2023.5.0" +version = "2023.6.0" description = "File-system specification" -category = "main" optional = false python-versions = ">=3.8" files = [ - {file = "fsspec-2023.5.0-py3-none-any.whl", hash = "sha256:51a4ad01a5bb66fcc58036e288c0d53d3975a0df2a5dc59a93b59bade0391f2a"}, - {file = "fsspec-2023.5.0.tar.gz", hash = "sha256:b3b56e00fb93ea321bc9e5d9cf6f8522a0198b20eb24e02774d329e9c6fb84ce"}, + {file = "fsspec-2023.6.0-py3-none-any.whl", hash = "sha256:1cbad1faef3e391fba6dc005ae9b5bdcbf43005c9167ce78c915549c352c869a"}, + {file = "fsspec-2023.6.0.tar.gz", hash = "sha256:d0b2f935446169753e7a5c5c55681c54ea91996cc67be93c39a154fb3a2742af"}, ] [package.extras] @@ -967,134 +927,144 @@ ssh = ["paramiko"] tqdm = ["tqdm"] [[package]] -name = "grpcio" -version = "1.49.1" -description = "HTTP/2-based RPC framework" -category = "main" +name = "greenlet" +version = "2.0.2" +description = "Lightweight in-process concurrent programming" optional = true -python-versions = ">=3.7" -files = [ - {file = "grpcio-1.49.1-cp310-cp310-linux_armv7l.whl", hash = "sha256:fd86040232e805b8e6378b2348c928490ee595b058ce9aaa27ed8e4b0f172b20"}, - {file = "grpcio-1.49.1-cp310-cp310-macosx_10_10_x86_64.whl", hash = "sha256:6fd0c9cede9552bf00f8c5791d257d5bf3790d7057b26c59df08be5e7a1e021d"}, - {file = "grpcio-1.49.1-cp310-cp310-manylinux_2_17_aarch64.whl", hash = "sha256:d0d402e158d4e84e49c158cb5204119d55e1baf363ee98d6cb5dce321c3a065d"}, - {file = "grpcio-1.49.1-cp310-cp310-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:822ceec743d42a627e64ea266059a62d214c5a3cdfcd0d7fe2b7a8e4e82527c7"}, - {file = "grpcio-1.49.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2106d9c16527f0a85e2eea6e6b91a74fc99579c60dd810d8690843ea02bc0f5f"}, - {file = "grpcio-1.49.1-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:52dd02b7e7868233c571b49bc38ebd347c3bb1ff8907bb0cb74cb5f00c790afc"}, - {file = "grpcio-1.49.1-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:120fecba2ec5d14b5a15d11063b39783fda8dc8d24addd83196acb6582cabd9b"}, - {file = "grpcio-1.49.1-cp310-cp310-win32.whl", hash = "sha256:f1a3b88e3c53c1a6e6bed635ec1bbb92201bb6a1f2db186179f7f3f244829788"}, - {file = "grpcio-1.49.1-cp310-cp310-win_amd64.whl", hash = "sha256:a7d0017b92d3850abea87c1bdec6ea41104e71c77bca44c3e17f175c6700af62"}, - {file = "grpcio-1.49.1-cp311-cp311-linux_armv7l.whl", hash = "sha256:9fb17ff8c0d56099ac6ebfa84f670c5a62228d6b5c695cf21c02160c2ac1446b"}, - {file = "grpcio-1.49.1-cp311-cp311-macosx_10_10_x86_64.whl", hash = "sha256:075f2d06e3db6b48a2157a1bcd52d6cbdca980dd18988fe6afdb41795d51625f"}, - {file = "grpcio-1.49.1-cp311-cp311-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:46d93a1b4572b461a227f1db6b8d35a88952db1c47e5fadcf8b8a2f0e1dd9201"}, - {file = "grpcio-1.49.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:dc79b2b37d779ac42341ddef40ad5bf0966a64af412c89fc2b062e3ddabb093f"}, - {file = "grpcio-1.49.1-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:5f8b3a971c7820ea9878f3fd70086240a36aeee15d1b7e9ecbc2743b0e785568"}, - {file = "grpcio-1.49.1-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:49b301740cf5bc8fed4fee4c877570189ae3951432d79fa8e524b09353659811"}, - {file = "grpcio-1.49.1-cp311-cp311-win32.whl", hash = "sha256:1c66a25afc6c71d357867b341da594a5587db5849b48f4b7d5908d236bb62ede"}, - {file = "grpcio-1.49.1-cp311-cp311-win_amd64.whl", hash = "sha256:6b6c3a95d27846f4145d6967899b3ab25fffc6ae99544415e1adcacef84842d2"}, - {file = "grpcio-1.49.1-cp37-cp37m-linux_armv7l.whl", hash = "sha256:1cc400c8a2173d1c042997d98a9563e12d9bb3fb6ad36b7f355bc77c7663b8af"}, - {file = "grpcio-1.49.1-cp37-cp37m-macosx_10_10_x86_64.whl", hash = "sha256:34f736bd4d0deae90015c0e383885b431444fe6b6c591dea288173df20603146"}, - {file = "grpcio-1.49.1-cp37-cp37m-manylinux_2_17_aarch64.whl", hash = "sha256:196082b9c89ebf0961dcd77cb114bed8171964c8e3063b9da2fb33536a6938ed"}, - {file = "grpcio-1.49.1-cp37-cp37m-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:8c9f89c42749890618cd3c2464e1fbf88446e3d2f67f1e334c8e5db2f3272bbd"}, - {file = "grpcio-1.49.1-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:64419cb8a5b612cdb1550c2fd4acbb7d4fb263556cf4625f25522337e461509e"}, - {file = "grpcio-1.49.1-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:8a5272061826e6164f96e3255405ef6f73b88fd3e8bef464c7d061af8585ac62"}, - {file = "grpcio-1.49.1-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:ea9d0172445241ad7cb49577314e39d0af2c5267395b3561d7ced5d70458a9f3"}, - {file = "grpcio-1.49.1-cp37-cp37m-win32.whl", hash = "sha256:2070e87d95991473244c72d96d13596c751cb35558e11f5df5414981e7ed2492"}, - {file = "grpcio-1.49.1-cp37-cp37m-win_amd64.whl", hash = "sha256:4fcedcab49baaa9db4a2d240ac81f2d57eb0052b1c6a9501b46b8ae912720fbf"}, - {file = "grpcio-1.49.1-cp38-cp38-linux_armv7l.whl", hash = "sha256:afbb3475cf7f4f7d380c2ca37ee826e51974f3e2665613996a91d6a58583a534"}, - {file = "grpcio-1.49.1-cp38-cp38-macosx_10_10_x86_64.whl", hash = "sha256:a4f9ba141380abde6c3adc1727f21529137a2552002243fa87c41a07e528245c"}, - {file = "grpcio-1.49.1-cp38-cp38-manylinux_2_17_aarch64.whl", hash = "sha256:cf0a1fb18a7204b9c44623dfbd1465b363236ce70c7a4ed30402f9f60d8b743b"}, - {file = "grpcio-1.49.1-cp38-cp38-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:17bb6fe72784b630728c6cff9c9d10ccc3b6d04e85da6e0a7b27fb1d135fac62"}, - {file = "grpcio-1.49.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:18305d5a082d1593b005a895c10041f833b16788e88b02bb81061f5ebcc465df"}, - {file = "grpcio-1.49.1-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:b6a1b39e59ac5a3067794a0e498911cf2e37e4b19ee9e9977dc5e7051714f13f"}, - {file = "grpcio-1.49.1-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:0e20d59aafc086b1cc68400463bddda6e41d3e5ed30851d1e2e0f6a2e7e342d3"}, - {file = "grpcio-1.49.1-cp38-cp38-win32.whl", hash = "sha256:e1e83233d4680863a421f3ee4a7a9b80d33cd27ee9ed7593bc93f6128302d3f2"}, - {file = "grpcio-1.49.1-cp38-cp38-win_amd64.whl", hash = "sha256:221d42c654d2a41fa31323216279c73ed17d92f533bc140a3390cc1bd78bf63c"}, - {file = "grpcio-1.49.1-cp39-cp39-linux_armv7l.whl", hash = "sha256:fa9e6e61391e99708ac87fc3436f6b7b9c6b845dc4639b406e5e61901e1aacde"}, - {file = "grpcio-1.49.1-cp39-cp39-macosx_10_10_x86_64.whl", hash = "sha256:9b449e966ef518ce9c860d21f8afe0b0f055220d95bc710301752ac1db96dd6a"}, - {file = "grpcio-1.49.1-cp39-cp39-manylinux_2_17_aarch64.whl", hash = "sha256:aa34d2ad9f24e47fa9a3172801c676e4037d862247e39030165fe83821a7aafd"}, - {file = "grpcio-1.49.1-cp39-cp39-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:5207f4eed1b775d264fcfe379d8541e1c43b878f2b63c0698f8f5c56c40f3d68"}, - {file = "grpcio-1.49.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0b24a74651438d45619ac67004638856f76cc13d78b7478f2457754cbcb1c8ad"}, - {file = "grpcio-1.49.1-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:fe763781669790dc8b9618e7e677c839c87eae6cf28b655ee1fa69ae04eea03f"}, - {file = "grpcio-1.49.1-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:2f2ff7ba0f8f431f32d4b4bc3a3713426949d3533b08466c4ff1b2b475932ca8"}, - {file = "grpcio-1.49.1-cp39-cp39-win32.whl", hash = "sha256:08ff74aec8ff457a89b97152d36cb811dcc1d17cd5a92a65933524e363327394"}, - {file = "grpcio-1.49.1-cp39-cp39-win_amd64.whl", hash = "sha256:274ffbb39717918c514b35176510ae9be06e1d93121e84d50b350861dcb9a705"}, - {file = "grpcio-1.49.1.tar.gz", hash = "sha256:d4725fc9ec8e8822906ae26bb26f5546891aa7fbc3443de970cc556d43a5c99f"}, +python-versions = ">=2.7,!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*,!=3.4.*" +files = [ + {file = "greenlet-2.0.2-cp27-cp27m-macosx_10_14_x86_64.whl", hash = "sha256:bdfea8c661e80d3c1c99ad7c3ff74e6e87184895bbaca6ee8cc61209f8b9b85d"}, + {file = "greenlet-2.0.2-cp27-cp27m-manylinux2010_x86_64.whl", hash = "sha256:9d14b83fab60d5e8abe587d51c75b252bcc21683f24699ada8fb275d7712f5a9"}, + {file = "greenlet-2.0.2-cp27-cp27m-win32.whl", hash = "sha256:6c3acb79b0bfd4fe733dff8bc62695283b57949ebcca05ae5c129eb606ff2d74"}, + {file = "greenlet-2.0.2-cp27-cp27m-win_amd64.whl", hash = "sha256:283737e0da3f08bd637b5ad058507e578dd462db259f7f6e4c5c365ba4ee9343"}, + {file = "greenlet-2.0.2-cp27-cp27mu-manylinux2010_x86_64.whl", hash = "sha256:d27ec7509b9c18b6d73f2f5ede2622441de812e7b1a80bbd446cb0633bd3d5ae"}, + {file = "greenlet-2.0.2-cp310-cp310-macosx_11_0_x86_64.whl", hash = "sha256:30bcf80dda7f15ac77ba5af2b961bdd9dbc77fd4ac6105cee85b0d0a5fcf74df"}, + {file = "greenlet-2.0.2-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:26fbfce90728d82bc9e6c38ea4d038cba20b7faf8a0ca53a9c07b67318d46088"}, + {file = "greenlet-2.0.2-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:9190f09060ea4debddd24665d6804b995a9c122ef5917ab26e1566dcc712ceeb"}, + {file = "greenlet-2.0.2-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d75209eed723105f9596807495d58d10b3470fa6732dd6756595e89925ce2470"}, + {file = "greenlet-2.0.2-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:3a51c9751078733d88e013587b108f1b7a1fb106d402fb390740f002b6f6551a"}, + {file = "greenlet-2.0.2-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:76ae285c8104046b3a7f06b42f29c7b73f77683df18c49ab5af7983994c2dd91"}, + {file = "greenlet-2.0.2-cp310-cp310-win_amd64.whl", hash = "sha256:2d4686f195e32d36b4d7cf2d166857dbd0ee9f3d20ae349b6bf8afc8485b3645"}, + {file = "greenlet-2.0.2-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:c4302695ad8027363e96311df24ee28978162cdcdd2006476c43970b384a244c"}, + {file = "greenlet-2.0.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c48f54ef8e05f04d6eff74b8233f6063cb1ed960243eacc474ee73a2ea8573ca"}, + {file = "greenlet-2.0.2-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:a1846f1b999e78e13837c93c778dcfc3365902cfb8d1bdb7dd73ead37059f0d0"}, + {file = "greenlet-2.0.2-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3a06ad5312349fec0ab944664b01d26f8d1f05009566339ac6f63f56589bc1a2"}, + {file = "greenlet-2.0.2-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:eff4eb9b7eb3e4d0cae3d28c283dc16d9bed6b193c2e1ace3ed86ce48ea8df19"}, + {file = "greenlet-2.0.2-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:5454276c07d27a740c5892f4907c86327b632127dd9abec42ee62e12427ff7e3"}, + {file = "greenlet-2.0.2-cp311-cp311-win_amd64.whl", hash = "sha256:7cafd1208fdbe93b67c7086876f061f660cfddc44f404279c1585bbf3cdc64c5"}, + {file = "greenlet-2.0.2-cp35-cp35m-macosx_10_14_x86_64.whl", hash = "sha256:910841381caba4f744a44bf81bfd573c94e10b3045ee00de0cbf436fe50673a6"}, + {file = "greenlet-2.0.2-cp35-cp35m-manylinux2010_x86_64.whl", hash = "sha256:18a7f18b82b52ee85322d7a7874e676f34ab319b9f8cce5de06067384aa8ff43"}, + {file = "greenlet-2.0.2-cp35-cp35m-win32.whl", hash = "sha256:03a8f4f3430c3b3ff8d10a2a86028c660355ab637cee9333d63d66b56f09d52a"}, + {file = "greenlet-2.0.2-cp35-cp35m-win_amd64.whl", hash = "sha256:4b58adb399c4d61d912c4c331984d60eb66565175cdf4a34792cd9600f21b394"}, + {file = "greenlet-2.0.2-cp36-cp36m-macosx_10_14_x86_64.whl", hash = "sha256:703f18f3fda276b9a916f0934d2fb6d989bf0b4fb5a64825260eb9bfd52d78f0"}, + {file = "greenlet-2.0.2-cp36-cp36m-manylinux2010_x86_64.whl", hash = "sha256:32e5b64b148966d9cccc2c8d35a671409e45f195864560829f395a54226408d3"}, + {file = "greenlet-2.0.2-cp36-cp36m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:2dd11f291565a81d71dab10b7033395b7a3a5456e637cf997a6f33ebdf06f8db"}, + {file = "greenlet-2.0.2-cp36-cp36m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:e0f72c9ddb8cd28532185f54cc1453f2c16fb417a08b53a855c4e6a418edd099"}, + {file = "greenlet-2.0.2-cp36-cp36m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:cd021c754b162c0fb55ad5d6b9d960db667faad0fa2ff25bb6e1301b0b6e6a75"}, + {file = "greenlet-2.0.2-cp36-cp36m-musllinux_1_1_aarch64.whl", hash = "sha256:3c9b12575734155d0c09d6c3e10dbd81665d5c18e1a7c6597df72fd05990c8cf"}, + {file = "greenlet-2.0.2-cp36-cp36m-musllinux_1_1_x86_64.whl", hash = "sha256:b9ec052b06a0524f0e35bd8790686a1da006bd911dd1ef7d50b77bfbad74e292"}, + {file = "greenlet-2.0.2-cp36-cp36m-win32.whl", hash = "sha256:dbfcfc0218093a19c252ca8eb9aee3d29cfdcb586df21049b9d777fd32c14fd9"}, + {file = "greenlet-2.0.2-cp36-cp36m-win_amd64.whl", hash = "sha256:9f35ec95538f50292f6d8f2c9c9f8a3c6540bbfec21c9e5b4b751e0a7c20864f"}, + {file = "greenlet-2.0.2-cp37-cp37m-macosx_10_15_x86_64.whl", hash = "sha256:d5508f0b173e6aa47273bdc0a0b5ba055b59662ba7c7ee5119528f466585526b"}, + {file = "greenlet-2.0.2-cp37-cp37m-manylinux2010_x86_64.whl", hash = "sha256:f82d4d717d8ef19188687aa32b8363e96062911e63ba22a0cff7802a8e58e5f1"}, + {file = "greenlet-2.0.2-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c9c59a2120b55788e800d82dfa99b9e156ff8f2227f07c5e3012a45a399620b7"}, + {file = "greenlet-2.0.2-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:2780572ec463d44c1d3ae850239508dbeb9fed38e294c68d19a24d925d9223ca"}, + {file = "greenlet-2.0.2-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:937e9020b514ceedb9c830c55d5c9872abc90f4b5862f89c0887033ae33c6f73"}, + {file = "greenlet-2.0.2-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:36abbf031e1c0f79dd5d596bfaf8e921c41df2bdf54ee1eed921ce1f52999a86"}, + {file = "greenlet-2.0.2-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:18e98fb3de7dba1c0a852731c3070cf022d14f0d68b4c87a19cc1016f3bb8b33"}, + {file = "greenlet-2.0.2-cp37-cp37m-win32.whl", hash = "sha256:3f6ea9bd35eb450837a3d80e77b517ea5bc56b4647f5502cd28de13675ee12f7"}, + {file = "greenlet-2.0.2-cp37-cp37m-win_amd64.whl", hash = "sha256:7492e2b7bd7c9b9916388d9df23fa49d9b88ac0640db0a5b4ecc2b653bf451e3"}, + {file = "greenlet-2.0.2-cp38-cp38-macosx_10_15_x86_64.whl", hash = "sha256:b864ba53912b6c3ab6bcb2beb19f19edd01a6bfcbdfe1f37ddd1778abfe75a30"}, + {file = "greenlet-2.0.2-cp38-cp38-manylinux2010_x86_64.whl", hash = "sha256:ba2956617f1c42598a308a84c6cf021a90ff3862eddafd20c3333d50f0edb45b"}, + {file = "greenlet-2.0.2-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:fc3a569657468b6f3fb60587e48356fe512c1754ca05a564f11366ac9e306526"}, + {file = "greenlet-2.0.2-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:8eab883b3b2a38cc1e050819ef06a7e6344d4a990d24d45bc6f2cf959045a45b"}, + {file = "greenlet-2.0.2-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:acd2162a36d3de67ee896c43effcd5ee3de247eb00354db411feb025aa319857"}, + {file = "greenlet-2.0.2-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:0bf60faf0bc2468089bdc5edd10555bab6e85152191df713e2ab1fcc86382b5a"}, + {file = "greenlet-2.0.2-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:b0ef99cdbe2b682b9ccbb964743a6aca37905fda5e0452e5ee239b1654d37f2a"}, + {file = "greenlet-2.0.2-cp38-cp38-win32.whl", hash = "sha256:b80f600eddddce72320dbbc8e3784d16bd3fb7b517e82476d8da921f27d4b249"}, + {file = "greenlet-2.0.2-cp38-cp38-win_amd64.whl", hash = "sha256:4d2e11331fc0c02b6e84b0d28ece3a36e0548ee1a1ce9ddde03752d9b79bba40"}, + {file = "greenlet-2.0.2-cp39-cp39-macosx_11_0_x86_64.whl", hash = "sha256:88d9ab96491d38a5ab7c56dd7a3cc37d83336ecc564e4e8816dbed12e5aaefc8"}, + {file = "greenlet-2.0.2-cp39-cp39-manylinux2010_x86_64.whl", hash = "sha256:561091a7be172ab497a3527602d467e2b3fbe75f9e783d8b8ce403fa414f71a6"}, + {file = "greenlet-2.0.2-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:971ce5e14dc5e73715755d0ca2975ac88cfdaefcaab078a284fea6cfabf866df"}, + {file = "greenlet-2.0.2-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:be4ed120b52ae4d974aa40215fcdfde9194d63541c7ded40ee12eb4dda57b76b"}, + {file = "greenlet-2.0.2-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:94c817e84245513926588caf1152e3b559ff794d505555211ca041f032abbb6b"}, + {file = "greenlet-2.0.2-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:1a819eef4b0e0b96bb0d98d797bef17dc1b4a10e8d7446be32d1da33e095dbb8"}, + {file = "greenlet-2.0.2-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:7efde645ca1cc441d6dc4b48c0f7101e8d86b54c8530141b09fd31cef5149ec9"}, + {file = "greenlet-2.0.2-cp39-cp39-win32.whl", hash = "sha256:ea9872c80c132f4663822dd2a08d404073a5a9b5ba6155bea72fb2a79d1093b5"}, + {file = "greenlet-2.0.2-cp39-cp39-win_amd64.whl", hash = "sha256:db1a39669102a1d8d12b57de2bb7e2ec9066a6f2b3da35ae511ff93b01b5d564"}, + {file = "greenlet-2.0.2.tar.gz", hash = "sha256:e7c8dc13af7db097bed64a051d2dd49e9f0af495c26995c00a9ee842690d34c0"}, ] -[package.dependencies] -six = ">=1.5.2" - [package.extras] -protobuf = ["grpcio-tools (>=1.49.1)"] +docs = ["Sphinx", "docutils (<0.18)"] +test = ["objgraph", "psutil"] [[package]] name = "grpcio" -version = "1.51.3" +version = "1.56.2" description = "HTTP/2-based RPC framework" -category = "main" optional = true python-versions = ">=3.7" files = [ - {file = "grpcio-1.51.3-cp310-cp310-linux_armv7l.whl", hash = "sha256:f601aaeae18dab81930fb8d4f916b0da21e89bb4b5f7367ef793f46b4a76b7b0"}, - {file = "grpcio-1.51.3-cp310-cp310-macosx_12_0_universal2.whl", hash = "sha256:eef0450a4b5ed11feab639bf3eb1b6e23d0efa9b911bf7b06fb60e14f5f8a585"}, - {file = "grpcio-1.51.3-cp310-cp310-manylinux_2_17_aarch64.whl", hash = "sha256:82b0ad8ac825d4bb31bff9f638557c045f4a6d824d84b21e893968286f88246b"}, - {file = "grpcio-1.51.3-cp310-cp310-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3667c06e37d6cd461afdd51cefe6537702f3d1dc5ff4cac07e88d8b4795dc16f"}, - {file = "grpcio-1.51.3-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3709048fe0aa23dda09b3e69849a12055790171dab9e399a72ea8f9dfbf9ac80"}, - {file = "grpcio-1.51.3-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:200d69857f9910f7458b39b9bcf83ee4a180591b40146ba9e49314e3a7419313"}, - {file = "grpcio-1.51.3-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:cd9a5e68e79c5f031500e67793048a90209711e0854a9ddee8a3ce51728de4e5"}, - {file = "grpcio-1.51.3-cp310-cp310-win32.whl", hash = "sha256:6604f614016127ae10969176bbf12eb0e03d2fb3d643f050b3b69e160d144fb4"}, - {file = "grpcio-1.51.3-cp310-cp310-win_amd64.whl", hash = "sha256:e95c7ccd4c5807adef1602005513bf7c7d14e5a41daebcf9d8d30d8bf51b8f81"}, - {file = "grpcio-1.51.3-cp311-cp311-linux_armv7l.whl", hash = "sha256:5e77ee138100f0bb55cbd147840f87ee6241dbd25f09ea7cd8afe7efff323449"}, - {file = "grpcio-1.51.3-cp311-cp311-macosx_10_10_universal2.whl", hash = "sha256:68a7514b754e38e8de9075f7bb4dee919919515ec68628c43a894027e40ddec4"}, - {file = "grpcio-1.51.3-cp311-cp311-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3c1b9f8afa62ff265d86a4747a2990ec5a96e4efce5d5888f245a682d66eca47"}, - {file = "grpcio-1.51.3-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8de30f0b417744288cec65ec8cf84b8a57995cf7f1e84ccad2704d93f05d0aae"}, - {file = "grpcio-1.51.3-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:b69c7adc7ed60da1cb1b502853db61f453fc745f940cbcc25eb97c99965d8f41"}, - {file = "grpcio-1.51.3-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:d81528ffe0e973dc840ec73a4132fd18b8203ad129d7410155d951a0a7e4f5d0"}, - {file = "grpcio-1.51.3-cp311-cp311-win32.whl", hash = "sha256:040eb421613b57c696063abde405916dd830203c184c9000fc8c3b3b3c950325"}, - {file = "grpcio-1.51.3-cp311-cp311-win_amd64.whl", hash = "sha256:2a8e17286c4240137d933b8ca506465472248b4ce0fe46f3404459e708b65b68"}, - {file = "grpcio-1.51.3-cp37-cp37m-linux_armv7l.whl", hash = "sha256:d5cd1389669a847555df54177b911d9ff6f17345b2a6f19388707b7a9f724c88"}, - {file = "grpcio-1.51.3-cp37-cp37m-macosx_10_10_universal2.whl", hash = "sha256:be1bf35ce82cdbcac14e39d5102d8de4079a1c1a6a06b68e41fcd9ef64f9dd28"}, - {file = "grpcio-1.51.3-cp37-cp37m-manylinux_2_17_aarch64.whl", hash = "sha256:5eed34994c095e2bf7194ffac7381c6068b057ef1e69f8f08db77771350a7566"}, - {file = "grpcio-1.51.3-cp37-cp37m-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3f9a7d88082b2a17ae7bd3c2354d13bab0453899e0851733f6afa6918373f476"}, - {file = "grpcio-1.51.3-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:36c8abbc5f837111e7bd619612eedc223c290b0903b952ce0c7b00840ea70f14"}, - {file = "grpcio-1.51.3-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:165b05af77e6aecb4210ae7663e25acf234ba78a7c1c157fa5f2efeb0d6ec53c"}, - {file = "grpcio-1.51.3-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:54e36c2ee304ff15f2bfbdc43d2b56c63331c52d818c364e5b5214e5bc2ad9f6"}, - {file = "grpcio-1.51.3-cp37-cp37m-win32.whl", hash = "sha256:cd0daac21d9ef5e033a5100c1d3aa055bbed28bfcf070b12d8058045c4e821b1"}, - {file = "grpcio-1.51.3-cp37-cp37m-win_amd64.whl", hash = "sha256:2fdd6333ce96435408565a9dbbd446212cd5d62e4d26f6a3c0feb1e3c35f1cc8"}, - {file = "grpcio-1.51.3-cp38-cp38-linux_armv7l.whl", hash = "sha256:54b0c29bdd9a3b1e1b61443ab152f060fc719f1c083127ab08d03fac5efd51be"}, - {file = "grpcio-1.51.3-cp38-cp38-macosx_10_10_universal2.whl", hash = "sha256:ffaaf7e93fcb437356b5a4b23bf36e8a3d0221399ff77fd057e4bc77776a24be"}, - {file = "grpcio-1.51.3-cp38-cp38-manylinux_2_17_aarch64.whl", hash = "sha256:eafbe7501a3268d05f2e450e1ddaffb950d842a8620c13ec328b501d25d2e2c3"}, - {file = "grpcio-1.51.3-cp38-cp38-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:881ecb34feabf31c6b3b9bbbddd1a5b57e69f805041e5a2c6c562a28574f71c4"}, - {file = "grpcio-1.51.3-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e860a3222139b41d430939bbec2ec9c3f6c740938bf7a04471a9a8caaa965a2e"}, - {file = "grpcio-1.51.3-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:49ede0528e9dac7e8a9fe30b16c73b630ddd9a576bf4b675eb6b0c53ee5ca00f"}, - {file = "grpcio-1.51.3-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:6972b009638b40a448d10e1bc18e2223143b8a7aa20d7def0d78dd4af4126d12"}, - {file = "grpcio-1.51.3-cp38-cp38-win32.whl", hash = "sha256:5694448256e3cdfe5bd358f1574a3f2f51afa20cc834713c4b9788d60b7cc646"}, - {file = "grpcio-1.51.3-cp38-cp38-win_amd64.whl", hash = "sha256:3ea4341efe603b049e8c9a5f13c696ca37fcdf8a23ca35f650428ad3606381d9"}, - {file = "grpcio-1.51.3-cp39-cp39-linux_armv7l.whl", hash = "sha256:6c677581ce129f5fa228b8f418cee10bd28dd449f3a544ea73c8ba590ee49d0b"}, - {file = "grpcio-1.51.3-cp39-cp39-macosx_10_10_universal2.whl", hash = "sha256:30e09b5e0531685e176f49679b6a3b190762cc225f4565e55a899f5e14b3aa62"}, - {file = "grpcio-1.51.3-cp39-cp39-manylinux_2_17_aarch64.whl", hash = "sha256:c831f31336e81243f85b6daff3e5e8a123302ce0ea1f2726ad752fd7a59f3aee"}, - {file = "grpcio-1.51.3-cp39-cp39-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:2cd2e4cefb724cab1ba2df4b7535a9980531b9ec51b4dbb5f137a1f3a3754ef0"}, - {file = "grpcio-1.51.3-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f7a0d0bf44438869d307f85a54f25a896ad6b4b0ca12370f76892ad732928d87"}, - {file = "grpcio-1.51.3-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:c02abd55409bfb293371554adf6a4401197ec2133dd97727c01180889014ba4d"}, - {file = "grpcio-1.51.3-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:2f8ff75e61e1227ba7a3f16b2eadbcc11d0a54096d52ab75a6b88cfbe56f55d1"}, - {file = "grpcio-1.51.3-cp39-cp39-win32.whl", hash = "sha256:6c99a73a6260bdf844b2e5ddad02dcd530310f80e1fa72c300fa19c1c7496962"}, - {file = "grpcio-1.51.3-cp39-cp39-win_amd64.whl", hash = "sha256:22bdfac4f7f27acdd4da359b5e7e1973dc74bf1ed406729b07d0759fde2f064b"}, - {file = "grpcio-1.51.3.tar.gz", hash = "sha256:be7b2265b7527bb12109a7727581e274170766d5b3c9258d4e466f4872522d7a"}, + {file = "grpcio-1.56.2-cp310-cp310-linux_armv7l.whl", hash = "sha256:bf0b9959e673505ee5869950642428046edb91f99942607c2ecf635f8a4b31c9"}, + {file = "grpcio-1.56.2-cp310-cp310-macosx_12_0_universal2.whl", hash = "sha256:5144feb20fe76e73e60c7d73ec3bf54f320247d1ebe737d10672480371878b48"}, + {file = "grpcio-1.56.2-cp310-cp310-manylinux_2_17_aarch64.whl", hash = "sha256:a72797549935c9e0b9bc1def1768c8b5a709538fa6ab0678e671aec47ebfd55e"}, + {file = "grpcio-1.56.2-cp310-cp310-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c3f3237a57e42f79f1e560726576aedb3a7ef931f4e3accb84ebf6acc485d316"}, + {file = "grpcio-1.56.2-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:900bc0096c2ca2d53f2e5cebf98293a7c32f532c4aeb926345e9747452233950"}, + {file = "grpcio-1.56.2-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:97e0efaebbfd222bcaac2f1735c010c1d3b167112d9d237daebbeedaaccf3d1d"}, + {file = "grpcio-1.56.2-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:c0c85c5cbe8b30a32fa6d802588d55ffabf720e985abe9590c7c886919d875d4"}, + {file = "grpcio-1.56.2-cp310-cp310-win32.whl", hash = "sha256:06e84ad9ae7668a109e970c7411e7992751a116494cba7c4fb877656527f9a57"}, + {file = "grpcio-1.56.2-cp310-cp310-win_amd64.whl", hash = "sha256:10954662f77dc36c9a1fb5cc4a537f746580d6b5734803be1e587252682cda8d"}, + {file = "grpcio-1.56.2-cp311-cp311-linux_armv7l.whl", hash = "sha256:c435f5ce1705de48e08fcbcfaf8aee660d199c90536e3e06f2016af7d6a938dd"}, + {file = "grpcio-1.56.2-cp311-cp311-macosx_10_10_universal2.whl", hash = "sha256:6108e5933eb8c22cd3646e72d5b54772c29f57482fd4c41a0640aab99eb5071d"}, + {file = "grpcio-1.56.2-cp311-cp311-manylinux_2_17_aarch64.whl", hash = "sha256:8391cea5ce72f4a12368afd17799474015d5d3dc00c936a907eb7c7eaaea98a5"}, + {file = "grpcio-1.56.2-cp311-cp311-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:750de923b456ca8c0f1354d6befca45d1f3b3a789e76efc16741bd4132752d95"}, + {file = "grpcio-1.56.2-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fda2783c12f553cdca11c08e5af6eecbd717280dc8fbe28a110897af1c15a88c"}, + {file = "grpcio-1.56.2-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:9e04d4e4cfafa7c5264e535b5d28e786f0571bea609c3f0aaab13e891e933e9c"}, + {file = "grpcio-1.56.2-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:89a49cc5ad08a38b6141af17e00d1dd482dc927c7605bc77af457b5a0fca807c"}, + {file = "grpcio-1.56.2-cp311-cp311-win32.whl", hash = "sha256:6a007a541dff984264981fbafeb052bfe361db63578948d857907df9488d8774"}, + {file = "grpcio-1.56.2-cp311-cp311-win_amd64.whl", hash = "sha256:af4063ef2b11b96d949dccbc5a987272f38d55c23c4c01841ea65a517906397f"}, + {file = "grpcio-1.56.2-cp37-cp37m-linux_armv7l.whl", hash = "sha256:a6ff459dac39541e6a2763a4439c4ca6bc9ecb4acc05a99b79246751f9894756"}, + {file = "grpcio-1.56.2-cp37-cp37m-macosx_10_10_universal2.whl", hash = "sha256:f20fd21f7538f8107451156dd1fe203300b79a9ddceba1ee0ac8132521a008ed"}, + {file = "grpcio-1.56.2-cp37-cp37m-manylinux_2_17_aarch64.whl", hash = "sha256:d1fbad1f9077372b6587ec589c1fc120b417b6c8ad72d3e3cc86bbbd0a3cee93"}, + {file = "grpcio-1.56.2-cp37-cp37m-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:6ee26e9dfb3996aff7c870f09dc7ad44a5f6732b8bdb5a5f9905737ac6fd4ef1"}, + {file = "grpcio-1.56.2-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a4c60abd950d6de3e4f1ddbc318075654d275c29c846ab6a043d6ed2c52e4c8c"}, + {file = "grpcio-1.56.2-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:1c31e52a04e62c8577a7bf772b3e7bed4df9c9e0dd90f92b6ffa07c16cab63c9"}, + {file = "grpcio-1.56.2-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:345356b307cce5d14355e8e055b4ca5f99bc857c33a3dc1ddbc544fca9cd0475"}, + {file = "grpcio-1.56.2-cp37-cp37m-win_amd64.whl", hash = "sha256:42e63904ee37ae46aa23de50dac8b145b3596f43598fa33fe1098ab2cbda6ff5"}, + {file = "grpcio-1.56.2-cp38-cp38-linux_armv7l.whl", hash = "sha256:7c5ede2e2558f088c49a1ddda19080e4c23fb5d171de80a726b61b567e3766ed"}, + {file = "grpcio-1.56.2-cp38-cp38-macosx_10_10_universal2.whl", hash = "sha256:33971197c47965cc1d97d78d842163c283e998223b151bab0499b951fd2c0b12"}, + {file = "grpcio-1.56.2-cp38-cp38-manylinux_2_17_aarch64.whl", hash = "sha256:d39f5d4af48c138cb146763eda14eb7d8b3ccbbec9fe86fb724cd16e0e914c64"}, + {file = "grpcio-1.56.2-cp38-cp38-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:ded637176addc1d3eef35331c39acc598bac550d213f0a1bedabfceaa2244c87"}, + {file = "grpcio-1.56.2-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c90da4b124647547a68cf2f197174ada30c7bb9523cb976665dfd26a9963d328"}, + {file = "grpcio-1.56.2-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:3ccb621749a81dc7755243665a70ce45536ec413ef5818e013fe8dfbf5aa497b"}, + {file = "grpcio-1.56.2-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:4eb37dd8dd1aa40d601212afa27ca5be255ba792e2e0b24d67b8af5e012cdb7d"}, + {file = "grpcio-1.56.2-cp38-cp38-win32.whl", hash = "sha256:ddb4a6061933bd9332b74eac0da25f17f32afa7145a33a0f9711ad74f924b1b8"}, + {file = "grpcio-1.56.2-cp38-cp38-win_amd64.whl", hash = "sha256:8940d6de7068af018dfa9a959a3510e9b7b543f4c405e88463a1cbaa3b2b379a"}, + {file = "grpcio-1.56.2-cp39-cp39-linux_armv7l.whl", hash = "sha256:51173e8fa6d9a2d85c14426bdee5f5c4a0654fd5fddcc21fe9d09ab0f6eb8b35"}, + {file = "grpcio-1.56.2-cp39-cp39-macosx_10_10_universal2.whl", hash = "sha256:373b48f210f43327a41e397391715cd11cfce9ded2fe76a5068f9bacf91cc226"}, + {file = "grpcio-1.56.2-cp39-cp39-manylinux_2_17_aarch64.whl", hash = "sha256:42a3bbb2bc07aef72a7d97e71aabecaf3e4eb616d39e5211e2cfe3689de860ca"}, + {file = "grpcio-1.56.2-cp39-cp39-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:5344be476ac37eb9c9ad09c22f4ea193c1316bf074f1daf85bddb1b31fda5116"}, + {file = "grpcio-1.56.2-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c3fa3ab0fb200a2c66493828ed06ccd1a94b12eddbfb985e7fd3e5723ff156c6"}, + {file = "grpcio-1.56.2-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:b975b85d1d5efc36cf8b237c5f3849b64d1ba33d6282f5e991f28751317504a1"}, + {file = "grpcio-1.56.2-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:cbdf2c498e077282cd427cfd88bdce4668019791deef0be8155385ab2ba7837f"}, + {file = "grpcio-1.56.2-cp39-cp39-win32.whl", hash = "sha256:139f66656a762572ae718fa0d1f2dce47c05e9fbf7a16acd704c354405b97df9"}, + {file = "grpcio-1.56.2-cp39-cp39-win_amd64.whl", hash = "sha256:830215173ad45d670140ff99aac3b461f9be9a6b11bee1a17265aaaa746a641a"}, + {file = "grpcio-1.56.2.tar.gz", hash = "sha256:0ff789ae7d8ddd76d2ac02e7d13bfef6fc4928ac01e1dcaa182be51b6bcc0aaa"}, ] [package.extras] -protobuf = ["grpcio-tools (>=1.51.3)"] +protobuf = ["grpcio-tools (>=1.56.2)"] [[package]] name = "identify" -version = "2.5.24" +version = "2.5.26" description = "File identification library for Python" -category = "dev" optional = false -python-versions = ">=3.7" +python-versions = ">=3.8" files = [ - {file = "identify-2.5.24-py2.py3-none-any.whl", hash = "sha256:986dbfb38b1140e763e413e6feb44cd731faf72d1909543178aa79b0e258265d"}, - {file = "identify-2.5.24.tar.gz", hash = "sha256:0aac67d5b4812498056d28a9a512a483f5085cc28640b02b258a59dac34301d4"}, + {file = "identify-2.5.26-py2.py3-none-any.whl", hash = "sha256:c22a8ead0d4ca11f1edd6c9418c3220669b3b7533ada0a0ffa6cc0ef85cf9b54"}, + {file = "identify-2.5.26.tar.gz", hash = "sha256:7243800bce2f58404ed41b7c002e53d4d22bcf3ae1b7900c2d7aefd95394bf7f"}, ] [package.extras] @@ -1104,7 +1074,6 @@ license = ["ukkonen"] name = "idna" version = "3.4" description = "Internationalized Domain Names in Applications (IDNA)" -category = "main" optional = false python-versions = ">=3.5" files = [ @@ -1114,14 +1083,13 @@ files = [ [[package]] name = "importlib-metadata" -version = "6.6.0" +version = "6.8.0" description = "Read metadata from Python packages" -category = "dev" optional = false -python-versions = ">=3.7" +python-versions = ">=3.8" files = [ - {file = "importlib_metadata-6.6.0-py3-none-any.whl", hash = "sha256:43dd286a2cd8995d5eaef7fee2066340423b818ed3fd70adf0bad5f1fac53fed"}, - {file = "importlib_metadata-6.6.0.tar.gz", hash = "sha256:92501cdf9cc66ebd3e612f1b4f0c0765dfa42f0fa38ffb319b6bd84dd675d705"}, + {file = "importlib_metadata-6.8.0-py3-none-any.whl", hash = "sha256:3ebb78df84a805d7698245025b975d9d67053cd94c79245ba4b3eb694abe68bb"}, + {file = "importlib_metadata-6.8.0.tar.gz", hash = "sha256:dbace7892d8c0c4ac1ad096662232f831d4e64f4c4545bd53016a3e9d4654743"}, ] [package.dependencies] @@ -1130,32 +1098,30 @@ zipp = ">=0.5" [package.extras] docs = ["furo", "jaraco.packaging (>=9)", "jaraco.tidelift (>=1.4)", "rst.linker (>=1.9)", "sphinx (>=3.5)", "sphinx-lint"] perf = ["ipython"] -testing = ["flake8 (<5)", "flufl.flake8", "importlib-resources (>=1.3)", "packaging", "pyfakefs", "pytest (>=6)", "pytest-black (>=0.3.7)", "pytest-checkdocs (>=2.4)", "pytest-cov", "pytest-enabler (>=1.3)", "pytest-flake8", "pytest-mypy (>=0.9.1)", "pytest-perf (>=0.9.2)"] +testing = ["flufl.flake8", "importlib-resources (>=1.3)", "packaging", "pyfakefs", "pytest (>=6)", "pytest-black (>=0.3.7)", "pytest-checkdocs (>=2.4)", "pytest-cov", "pytest-enabler (>=2.2)", "pytest-mypy (>=0.9.1)", "pytest-perf (>=0.9.2)", "pytest-ruff"] [[package]] name = "importlib-resources" -version = "5.12.0" +version = "6.0.0" description = "Read resources from Python packages" -category = "main" optional = true -python-versions = ">=3.7" +python-versions = ">=3.8" files = [ - {file = "importlib_resources-5.12.0-py3-none-any.whl", hash = "sha256:7b1deeebbf351c7578e09bf2f63fa2ce8b5ffec296e0d349139d43cca061a81a"}, - {file = "importlib_resources-5.12.0.tar.gz", hash = "sha256:4be82589bf5c1d7999aedf2a45159d10cb3ca4f19b2271f8792bc8e6da7b22f6"}, + {file = "importlib_resources-6.0.0-py3-none-any.whl", hash = "sha256:d952faee11004c045f785bb5636e8f885bed30dc3c940d5d42798a2a4541c185"}, + {file = "importlib_resources-6.0.0.tar.gz", hash = "sha256:4cf94875a8368bd89531a756df9a9ebe1f150e0f885030b461237bc7f2d905f2"}, ] [package.dependencies] zipp = {version = ">=3.1.0", markers = "python_version < \"3.10\""} [package.extras] -docs = ["furo", "jaraco.packaging (>=9)", "jaraco.tidelift (>=1.4)", "rst.linker (>=1.9)", "sphinx (>=3.5)", "sphinx-lint"] -testing = ["flake8 (<5)", "pytest (>=6)", "pytest-black (>=0.3.7)", "pytest-checkdocs (>=2.4)", "pytest-cov", "pytest-enabler (>=1.3)", "pytest-flake8", "pytest-mypy (>=0.9.1)"] +docs = ["furo", "jaraco.packaging (>=9.3)", "jaraco.tidelift (>=1.4)", "rst.linker (>=1.9)", "sphinx (>=3.5)", "sphinx-lint"] +testing = ["pytest (>=6)", "pytest-black (>=0.3.7)", "pytest-checkdocs (>=2.4)", "pytest-cov", "pytest-enabler (>=2.2)", "pytest-mypy (>=0.9.1)", "pytest-ruff"] [[package]] name = "iniconfig" version = "2.0.0" description = "brain-dead simple config-ini parsing" -category = "dev" optional = false python-versions = ">=3.7" files = [ @@ -1167,7 +1133,6 @@ files = [ name = "isodate" version = "0.6.1" description = "An ISO 8601 date/time/duration parser and formatter" -category = "main" optional = true python-versions = "*" files = [ @@ -1182,7 +1147,6 @@ six = "*" name = "jinja2" version = "3.1.2" description = "A very fast and expressive template engine." -category = "dev" optional = false python-versions = ">=3.7" files = [ @@ -1200,7 +1164,6 @@ i18n = ["Babel (>=2.7)"] name = "jmespath" version = "1.0.1" description = "JSON Matching Expressions" -category = "main" optional = false python-versions = ">=3.7" files = [ @@ -1210,36 +1173,51 @@ files = [ [[package]] name = "jsonschema" -version = "4.17.3" +version = "4.18.4" description = "An implementation of JSON Schema validation for Python" -category = "main" optional = true -python-versions = ">=3.7" +python-versions = ">=3.8" files = [ - {file = "jsonschema-4.17.3-py3-none-any.whl", hash = "sha256:a870ad254da1a8ca84b6a2905cac29d265f805acc57af304784962a2aa6508f6"}, - {file = "jsonschema-4.17.3.tar.gz", hash = "sha256:0f864437ab8b6076ba6707453ef8f98a6a0d512a80e93f8abdb676f737ecb60d"}, + {file = "jsonschema-4.18.4-py3-none-any.whl", hash = "sha256:971be834317c22daaa9132340a51c01b50910724082c2c1a2ac87eeec153a3fe"}, + {file = "jsonschema-4.18.4.tar.gz", hash = "sha256:fb3642735399fa958c0d2aad7057901554596c63349f4f6b283c493cf692a25d"}, ] [package.dependencies] -attrs = ">=17.4.0" +attrs = ">=22.2.0" importlib-resources = {version = ">=1.4.0", markers = "python_version < \"3.9\""} +jsonschema-specifications = ">=2023.03.6" pkgutil-resolve-name = {version = ">=1.3.10", markers = "python_version < \"3.9\""} -pyrsistent = ">=0.14.0,<0.17.0 || >0.17.0,<0.17.1 || >0.17.1,<0.17.2 || >0.17.2" +referencing = ">=0.28.4" +rpds-py = ">=0.7.1" [package.extras] format = ["fqdn", "idna", "isoduration", "jsonpointer (>1.13)", "rfc3339-validator", "rfc3987", "uri-template", "webcolors (>=1.11)"] format-nongpl = ["fqdn", "idna", "isoduration", "jsonpointer (>1.13)", "rfc3339-validator", "rfc3986-validator (>0.1.0)", "uri-template", "webcolors (>=1.11)"] +[[package]] +name = "jsonschema-specifications" +version = "2023.7.1" +description = "The JSON Schema meta-schemas and vocabularies, exposed as a Registry" +optional = true +python-versions = ">=3.8" +files = [ + {file = "jsonschema_specifications-2023.7.1-py3-none-any.whl", hash = "sha256:05adf340b659828a004220a9613be00fa3f223f2b82002e273dee62fd50524b1"}, + {file = "jsonschema_specifications-2023.7.1.tar.gz", hash = "sha256:c91a50404e88a1f6ba40636778e2ee08f6e24c5613fe4c53ac24578a5a7f72bb"}, +] + +[package.dependencies] +importlib-resources = {version = ">=1.4.0", markers = "python_version < \"3.9\""} +referencing = ">=0.28.0" + [[package]] name = "markdown-it-py" -version = "2.2.0" +version = "3.0.0" description = "Python port of markdown-it. Markdown parsing, done right!" -category = "main" optional = false -python-versions = ">=3.7" +python-versions = ">=3.8" files = [ - {file = "markdown-it-py-2.2.0.tar.gz", hash = "sha256:7c9a5e412688bc771c67432cbfebcdd686c93ce6484913dccf06cb5a0bea35a1"}, - {file = "markdown_it_py-2.2.0-py3-none-any.whl", hash = "sha256:5a35f8d1870171d9acc47b99612dc146129b631baf04970128b568f190d0cc30"}, + {file = "markdown-it-py-3.0.0.tar.gz", hash = "sha256:e3f60a94fa066dc52ec76661e37c851cb232d92f9886b15cb560aaada2df8feb"}, + {file = "markdown_it_py-3.0.0-py3-none-any.whl", hash = "sha256:355216845c60bd96232cd8d8c40e8f9765cc86f46880e43a8fd22dc1a1a8cab1"}, ] [package.dependencies] @@ -1252,74 +1230,72 @@ compare = ["commonmark (>=0.9,<1.0)", "markdown (>=3.4,<4.0)", "mistletoe (>=1.0 linkify = ["linkify-it-py (>=1,<3)"] plugins = ["mdit-py-plugins"] profiling = ["gprof2dot"] -rtd = ["attrs", "myst-parser", "pyyaml", "sphinx", "sphinx-copybutton", "sphinx-design", "sphinx_book_theme"] +rtd = ["jupyter_sphinx", "mdit-py-plugins", "myst-parser", "pyyaml", "sphinx", "sphinx-copybutton", "sphinx-design", "sphinx_book_theme"] testing = ["coverage", "pytest", "pytest-cov", "pytest-regressions"] [[package]] name = "markupsafe" -version = "2.1.2" +version = "2.1.3" description = "Safely add untrusted strings to HTML/XML markup." -category = "dev" optional = false python-versions = ">=3.7" files = [ - {file = "MarkupSafe-2.1.2-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:665a36ae6f8f20a4676b53224e33d456a6f5a72657d9c83c2aa00765072f31f7"}, - {file = "MarkupSafe-2.1.2-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:340bea174e9761308703ae988e982005aedf427de816d1afe98147668cc03036"}, - {file = "MarkupSafe-2.1.2-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:22152d00bf4a9c7c83960521fc558f55a1adbc0631fbb00a9471e097b19d72e1"}, - {file = "MarkupSafe-2.1.2-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:28057e985dace2f478e042eaa15606c7efccb700797660629da387eb289b9323"}, - {file = "MarkupSafe-2.1.2-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:ca244fa73f50a800cf8c3ebf7fd93149ec37f5cb9596aa8873ae2c1d23498601"}, - {file = "MarkupSafe-2.1.2-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:d9d971ec1e79906046aa3ca266de79eac42f1dbf3612a05dc9368125952bd1a1"}, - {file = "MarkupSafe-2.1.2-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:7e007132af78ea9df29495dbf7b5824cb71648d7133cf7848a2a5dd00d36f9ff"}, - {file = "MarkupSafe-2.1.2-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:7313ce6a199651c4ed9d7e4cfb4aa56fe923b1adf9af3b420ee14e6d9a73df65"}, - {file = "MarkupSafe-2.1.2-cp310-cp310-win32.whl", hash = "sha256:c4a549890a45f57f1ebf99c067a4ad0cb423a05544accaf2b065246827ed9603"}, - {file = "MarkupSafe-2.1.2-cp310-cp310-win_amd64.whl", hash = "sha256:835fb5e38fd89328e9c81067fd642b3593c33e1e17e2fdbf77f5676abb14a156"}, - {file = "MarkupSafe-2.1.2-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:2ec4f2d48ae59bbb9d1f9d7efb9236ab81429a764dedca114f5fdabbc3788013"}, - {file = "MarkupSafe-2.1.2-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:608e7073dfa9e38a85d38474c082d4281f4ce276ac0010224eaba11e929dd53a"}, - {file = "MarkupSafe-2.1.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:65608c35bfb8a76763f37036547f7adfd09270fbdbf96608be2bead319728fcd"}, - {file = "MarkupSafe-2.1.2-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f2bfb563d0211ce16b63c7cb9395d2c682a23187f54c3d79bfec33e6705473c6"}, - {file = "MarkupSafe-2.1.2-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:da25303d91526aac3672ee6d49a2f3db2d9502a4a60b55519feb1a4c7714e07d"}, - {file = "MarkupSafe-2.1.2-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:9cad97ab29dfc3f0249b483412c85c8ef4766d96cdf9dcf5a1e3caa3f3661cf1"}, - {file = "MarkupSafe-2.1.2-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:085fd3201e7b12809f9e6e9bc1e5c96a368c8523fad5afb02afe3c051ae4afcc"}, - {file = "MarkupSafe-2.1.2-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:1bea30e9bf331f3fef67e0a3877b2288593c98a21ccb2cf29b74c581a4eb3af0"}, - {file = "MarkupSafe-2.1.2-cp311-cp311-win32.whl", hash = "sha256:7df70907e00c970c60b9ef2938d894a9381f38e6b9db73c5be35e59d92e06625"}, - {file = "MarkupSafe-2.1.2-cp311-cp311-win_amd64.whl", hash = "sha256:e55e40ff0cc8cc5c07996915ad367fa47da6b3fc091fdadca7f5403239c5fec3"}, - {file = "MarkupSafe-2.1.2-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:a6e40afa7f45939ca356f348c8e23048e02cb109ced1eb8420961b2f40fb373a"}, - {file = "MarkupSafe-2.1.2-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:cf877ab4ed6e302ec1d04952ca358b381a882fbd9d1b07cccbfd61783561f98a"}, - {file = "MarkupSafe-2.1.2-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:63ba06c9941e46fa389d389644e2d8225e0e3e5ebcc4ff1ea8506dce646f8c8a"}, - {file = "MarkupSafe-2.1.2-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f1cd098434e83e656abf198f103a8207a8187c0fc110306691a2e94a78d0abb2"}, - {file = "MarkupSafe-2.1.2-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:55f44b440d491028addb3b88f72207d71eeebfb7b5dbf0643f7c023ae1fba619"}, - {file = "MarkupSafe-2.1.2-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:a6f2fcca746e8d5910e18782f976489939d54a91f9411c32051b4aab2bd7c513"}, - {file = "MarkupSafe-2.1.2-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:0b462104ba25f1ac006fdab8b6a01ebbfbce9ed37fd37fd4acd70c67c973e460"}, - {file = "MarkupSafe-2.1.2-cp37-cp37m-win32.whl", hash = "sha256:7668b52e102d0ed87cb082380a7e2e1e78737ddecdde129acadb0eccc5423859"}, - {file = "MarkupSafe-2.1.2-cp37-cp37m-win_amd64.whl", hash = "sha256:6d6607f98fcf17e534162f0709aaad3ab7a96032723d8ac8750ffe17ae5a0666"}, - {file = "MarkupSafe-2.1.2-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:a806db027852538d2ad7555b203300173dd1b77ba116de92da9afbc3a3be3eed"}, - {file = "MarkupSafe-2.1.2-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:a4abaec6ca3ad8660690236d11bfe28dfd707778e2442b45addd2f086d6ef094"}, - {file = "MarkupSafe-2.1.2-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f03a532d7dee1bed20bc4884194a16160a2de9ffc6354b3878ec9682bb623c54"}, - {file = "MarkupSafe-2.1.2-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4cf06cdc1dda95223e9d2d3c58d3b178aa5dacb35ee7e3bbac10e4e1faacb419"}, - {file = "MarkupSafe-2.1.2-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:22731d79ed2eb25059ae3df1dfc9cb1546691cc41f4e3130fe6bfbc3ecbbecfa"}, - {file = "MarkupSafe-2.1.2-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:f8ffb705ffcf5ddd0e80b65ddf7bed7ee4f5a441ea7d3419e861a12eaf41af58"}, - {file = "MarkupSafe-2.1.2-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:8db032bf0ce9022a8e41a22598eefc802314e81b879ae093f36ce9ddf39ab1ba"}, - {file = "MarkupSafe-2.1.2-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:2298c859cfc5463f1b64bd55cb3e602528db6fa0f3cfd568d3605c50678f8f03"}, - {file = "MarkupSafe-2.1.2-cp38-cp38-win32.whl", hash = "sha256:50c42830a633fa0cf9e7d27664637532791bfc31c731a87b202d2d8ac40c3ea2"}, - {file = "MarkupSafe-2.1.2-cp38-cp38-win_amd64.whl", hash = "sha256:bb06feb762bade6bf3c8b844462274db0c76acc95c52abe8dbed28ae3d44a147"}, - {file = "MarkupSafe-2.1.2-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:99625a92da8229df6d44335e6fcc558a5037dd0a760e11d84be2260e6f37002f"}, - {file = "MarkupSafe-2.1.2-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:8bca7e26c1dd751236cfb0c6c72d4ad61d986e9a41bbf76cb445f69488b2a2bd"}, - {file = "MarkupSafe-2.1.2-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:40627dcf047dadb22cd25ea7ecfe9cbf3bbbad0482ee5920b582f3809c97654f"}, - {file = "MarkupSafe-2.1.2-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:40dfd3fefbef579ee058f139733ac336312663c6706d1163b82b3003fb1925c4"}, - {file = "MarkupSafe-2.1.2-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:090376d812fb6ac5f171e5938e82e7f2d7adc2b629101cec0db8b267815c85e2"}, - {file = "MarkupSafe-2.1.2-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:2e7821bffe00aa6bd07a23913b7f4e01328c3d5cc0b40b36c0bd81d362faeb65"}, - {file = "MarkupSafe-2.1.2-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:c0a33bc9f02c2b17c3ea382f91b4db0e6cde90b63b296422a939886a7a80de1c"}, - {file = "MarkupSafe-2.1.2-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:b8526c6d437855442cdd3d87eede9c425c4445ea011ca38d937db299382e6fa3"}, - {file = "MarkupSafe-2.1.2-cp39-cp39-win32.whl", hash = "sha256:137678c63c977754abe9086a3ec011e8fd985ab90631145dfb9294ad09c102a7"}, - {file = "MarkupSafe-2.1.2-cp39-cp39-win_amd64.whl", hash = "sha256:0576fe974b40a400449768941d5d0858cc624e3249dfd1e0c33674e5c7ca7aed"}, - {file = "MarkupSafe-2.1.2.tar.gz", hash = "sha256:abcabc8c2b26036d62d4c746381a6f7cf60aafcc653198ad678306986b09450d"}, + {file = "MarkupSafe-2.1.3-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:cd0f502fe016460680cd20aaa5a76d241d6f35a1c3350c474bac1273803893fa"}, + {file = "MarkupSafe-2.1.3-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:e09031c87a1e51556fdcb46e5bd4f59dfb743061cf93c4d6831bf894f125eb57"}, + {file = "MarkupSafe-2.1.3-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:68e78619a61ecf91e76aa3e6e8e33fc4894a2bebe93410754bd28fce0a8a4f9f"}, + {file = "MarkupSafe-2.1.3-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:65c1a9bcdadc6c28eecee2c119465aebff8f7a584dd719facdd9e825ec61ab52"}, + {file = "MarkupSafe-2.1.3-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:525808b8019e36eb524b8c68acdd63a37e75714eac50e988180b169d64480a00"}, + {file = "MarkupSafe-2.1.3-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:962f82a3086483f5e5f64dbad880d31038b698494799b097bc59c2edf392fce6"}, + {file = "MarkupSafe-2.1.3-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:aa7bd130efab1c280bed0f45501b7c8795f9fdbeb02e965371bbef3523627779"}, + {file = "MarkupSafe-2.1.3-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:c9c804664ebe8f83a211cace637506669e7890fec1b4195b505c214e50dd4eb7"}, + {file = "MarkupSafe-2.1.3-cp310-cp310-win32.whl", hash = "sha256:10bbfe99883db80bdbaff2dcf681dfc6533a614f700da1287707e8a5d78a8431"}, + {file = "MarkupSafe-2.1.3-cp310-cp310-win_amd64.whl", hash = "sha256:1577735524cdad32f9f694208aa75e422adba74f1baee7551620e43a3141f559"}, + {file = "MarkupSafe-2.1.3-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:ad9e82fb8f09ade1c3e1b996a6337afac2b8b9e365f926f5a61aacc71adc5b3c"}, + {file = "MarkupSafe-2.1.3-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:3c0fae6c3be832a0a0473ac912810b2877c8cb9d76ca48de1ed31e1c68386575"}, + {file = "MarkupSafe-2.1.3-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b076b6226fb84157e3f7c971a47ff3a679d837cf338547532ab866c57930dbee"}, + {file = "MarkupSafe-2.1.3-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bfce63a9e7834b12b87c64d6b155fdd9b3b96191b6bd334bf37db7ff1fe457f2"}, + {file = "MarkupSafe-2.1.3-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:338ae27d6b8745585f87218a3f23f1512dbf52c26c28e322dbe54bcede54ccb9"}, + {file = "MarkupSafe-2.1.3-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:e4dd52d80b8c83fdce44e12478ad2e85c64ea965e75d66dbeafb0a3e77308fcc"}, + {file = "MarkupSafe-2.1.3-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:df0be2b576a7abbf737b1575f048c23fb1d769f267ec4358296f31c2479db8f9"}, + {file = "MarkupSafe-2.1.3-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:5bbe06f8eeafd38e5d0a4894ffec89378b6c6a625ff57e3028921f8ff59318ac"}, + {file = "MarkupSafe-2.1.3-cp311-cp311-win32.whl", hash = "sha256:dd15ff04ffd7e05ffcb7fe79f1b98041b8ea30ae9234aed2a9168b5797c3effb"}, + {file = "MarkupSafe-2.1.3-cp311-cp311-win_amd64.whl", hash = "sha256:134da1eca9ec0ae528110ccc9e48041e0828d79f24121a1a146161103c76e686"}, + {file = "MarkupSafe-2.1.3-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:8e254ae696c88d98da6555f5ace2279cf7cd5b3f52be2b5cf97feafe883b58d2"}, + {file = "MarkupSafe-2.1.3-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:cb0932dc158471523c9637e807d9bfb93e06a95cbf010f1a38b98623b929ef2b"}, + {file = "MarkupSafe-2.1.3-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9402b03f1a1b4dc4c19845e5c749e3ab82d5078d16a2a4c2cd2df62d57bb0707"}, + {file = "MarkupSafe-2.1.3-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:ca379055a47383d02a5400cb0d110cef0a776fc644cda797db0c5696cfd7e18e"}, + {file = "MarkupSafe-2.1.3-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:b7ff0f54cb4ff66dd38bebd335a38e2c22c41a8ee45aa608efc890ac3e3931bc"}, + {file = "MarkupSafe-2.1.3-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:c011a4149cfbcf9f03994ec2edffcb8b1dc2d2aede7ca243746df97a5d41ce48"}, + {file = "MarkupSafe-2.1.3-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:56d9f2ecac662ca1611d183feb03a3fa4406469dafe241673d521dd5ae92a155"}, + {file = "MarkupSafe-2.1.3-cp37-cp37m-win32.whl", hash = "sha256:8758846a7e80910096950b67071243da3e5a20ed2546e6392603c096778d48e0"}, + {file = "MarkupSafe-2.1.3-cp37-cp37m-win_amd64.whl", hash = "sha256:787003c0ddb00500e49a10f2844fac87aa6ce977b90b0feaaf9de23c22508b24"}, + {file = "MarkupSafe-2.1.3-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:2ef12179d3a291be237280175b542c07a36e7f60718296278d8593d21ca937d4"}, + {file = "MarkupSafe-2.1.3-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:2c1b19b3aaacc6e57b7e25710ff571c24d6c3613a45e905b1fde04d691b98ee0"}, + {file = "MarkupSafe-2.1.3-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8afafd99945ead6e075b973fefa56379c5b5c53fd8937dad92c662da5d8fd5ee"}, + {file = "MarkupSafe-2.1.3-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8c41976a29d078bb235fea9b2ecd3da465df42a562910f9022f1a03107bd02be"}, + {file = "MarkupSafe-2.1.3-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d080e0a5eb2529460b30190fcfcc4199bd7f827663f858a226a81bc27beaa97e"}, + {file = "MarkupSafe-2.1.3-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:69c0f17e9f5a7afdf2cc9fb2d1ce6aabdb3bafb7f38017c0b77862bcec2bbad8"}, + {file = "MarkupSafe-2.1.3-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:504b320cd4b7eff6f968eddf81127112db685e81f7e36e75f9f84f0df46041c3"}, + {file = "MarkupSafe-2.1.3-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:42de32b22b6b804f42c5d98be4f7e5e977ecdd9ee9b660fda1a3edf03b11792d"}, + {file = "MarkupSafe-2.1.3-cp38-cp38-win32.whl", hash = "sha256:ceb01949af7121f9fc39f7d27f91be8546f3fb112c608bc4029aef0bab86a2a5"}, + {file = "MarkupSafe-2.1.3-cp38-cp38-win_amd64.whl", hash = "sha256:1b40069d487e7edb2676d3fbdb2b0829ffa2cd63a2ec26c4938b2d34391b4ecc"}, + {file = "MarkupSafe-2.1.3-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:8023faf4e01efadfa183e863fefde0046de576c6f14659e8782065bcece22198"}, + {file = "MarkupSafe-2.1.3-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:6b2b56950d93e41f33b4223ead100ea0fe11f8e6ee5f641eb753ce4b77a7042b"}, + {file = "MarkupSafe-2.1.3-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:9dcdfd0eaf283af041973bff14a2e143b8bd64e069f4c383416ecd79a81aab58"}, + {file = "MarkupSafe-2.1.3-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:05fb21170423db021895e1ea1e1f3ab3adb85d1c2333cbc2310f2a26bc77272e"}, + {file = "MarkupSafe-2.1.3-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:282c2cb35b5b673bbcadb33a585408104df04f14b2d9b01d4c345a3b92861c2c"}, + {file = "MarkupSafe-2.1.3-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:ab4a0df41e7c16a1392727727e7998a467472d0ad65f3ad5e6e765015df08636"}, + {file = "MarkupSafe-2.1.3-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:7ef3cb2ebbf91e330e3bb937efada0edd9003683db6b57bb108c4001f37a02ea"}, + {file = "MarkupSafe-2.1.3-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:0a4e4a1aff6c7ac4cd55792abf96c915634c2b97e3cc1c7129578aa68ebd754e"}, + {file = "MarkupSafe-2.1.3-cp39-cp39-win32.whl", hash = "sha256:fec21693218efe39aa7f8599346e90c705afa52c5b31ae019b2e57e8f6542bb2"}, + {file = "MarkupSafe-2.1.3-cp39-cp39-win_amd64.whl", hash = "sha256:3fd4abcb888d15a94f32b75d8fd18ee162ca0c064f35b11134be77050296d6ba"}, + {file = "MarkupSafe-2.1.3.tar.gz", hash = "sha256:af598ed32d6ae86f1b747b82783958b1a4ab8f617b06fe68795c7f026abbdcad"}, ] [[package]] name = "mdurl" version = "0.1.2" description = "Markdown URL utilities" -category = "main" optional = false python-versions = ">=3.7" files = [ @@ -1331,7 +1307,6 @@ files = [ name = "mmhash3" version = "3.0.1" description = "Python wrapper for MurmurHash (MurmurHash3), a set of fast and robust hash functions." -category = "main" optional = false python-versions = "*" files = [ @@ -1373,14 +1348,13 @@ files = [ [[package]] name = "moto" -version = "4.1.10" +version = "4.1.13" description = "" -category = "dev" optional = false python-versions = ">=3.7" files = [ - {file = "moto-4.1.10-py2.py3-none-any.whl", hash = "sha256:86ee1f6b23a0c3dcf5c829fb1210fa3f293a2e8f1c95412538c0ea4a2f437021"}, - {file = "moto-4.1.10.tar.gz", hash = "sha256:65e15fc3e80a9589f50f8a5267b9a99f327224ed4d5cc44fe20edcbdb9248e06"}, + {file = "moto-4.1.13-py2.py3-none-any.whl", hash = "sha256:9650d05d89b6f97043695548fbc0d8fb293f4177daaebbcee00bb0d171367f1a"}, + {file = "moto-4.1.13.tar.gz", hash = "sha256:dd3e2ad920ab8b058c4f62fa7c195b788bd1f018cc701a1868ff5d5c4de6ed47"}, ] [package.dependencies] @@ -1395,17 +1369,17 @@ werkzeug = ">=0.5,<2.2.0 || >2.2.0,<2.2.1 || >2.2.1" xmltodict = "*" [package.extras] -all = ["PyYAML (>=5.1)", "aws-xray-sdk (>=0.93,!=0.96)", "cfn-lint (>=0.40.0)", "docker (>=3.0.0)", "ecdsa (!=0.15)", "graphql-core", "jsondiff (>=1.1.2)", "openapi-spec-validator (>=0.2.8)", "py-partiql-parser (==0.3.0)", "pyparsing (>=3.0.7)", "python-jose[cryptography] (>=3.1.0,<4.0.0)", "setuptools", "sshpubkeys (>=3.1.0)"] +all = ["PyYAML (>=5.1)", "aws-xray-sdk (>=0.93,!=0.96)", "cfn-lint (>=0.40.0)", "docker (>=3.0.0)", "ecdsa (!=0.15)", "graphql-core", "jsondiff (>=1.1.2)", "openapi-spec-validator (>=0.2.8)", "py-partiql-parser (==0.3.3)", "pyparsing (>=3.0.7)", "python-jose[cryptography] (>=3.1.0,<4.0.0)", "setuptools", "sshpubkeys (>=3.1.0)"] apigateway = ["PyYAML (>=5.1)", "ecdsa (!=0.15)", "openapi-spec-validator (>=0.2.8)", "python-jose[cryptography] (>=3.1.0,<4.0.0)"] apigatewayv2 = ["PyYAML (>=5.1)"] appsync = ["graphql-core"] awslambda = ["docker (>=3.0.0)"] batch = ["docker (>=3.0.0)"] -cloudformation = ["PyYAML (>=5.1)", "aws-xray-sdk (>=0.93,!=0.96)", "cfn-lint (>=0.40.0)", "docker (>=3.0.0)", "ecdsa (!=0.15)", "graphql-core", "jsondiff (>=1.1.2)", "openapi-spec-validator (>=0.2.8)", "py-partiql-parser (==0.3.0)", "pyparsing (>=3.0.7)", "python-jose[cryptography] (>=3.1.0,<4.0.0)", "setuptools", "sshpubkeys (>=3.1.0)"] +cloudformation = ["PyYAML (>=5.1)", "aws-xray-sdk (>=0.93,!=0.96)", "cfn-lint (>=0.40.0)", "docker (>=3.0.0)", "ecdsa (!=0.15)", "graphql-core", "jsondiff (>=1.1.2)", "openapi-spec-validator (>=0.2.8)", "py-partiql-parser (==0.3.3)", "pyparsing (>=3.0.7)", "python-jose[cryptography] (>=3.1.0,<4.0.0)", "setuptools", "sshpubkeys (>=3.1.0)"] cognitoidp = ["ecdsa (!=0.15)", "python-jose[cryptography] (>=3.1.0,<4.0.0)"] ds = ["sshpubkeys (>=3.1.0)"] -dynamodb = ["docker (>=3.0.0)", "py-partiql-parser (==0.3.0)"] -dynamodbstreams = ["docker (>=3.0.0)", "py-partiql-parser (==0.3.0)"] +dynamodb = ["docker (>=3.0.0)", "py-partiql-parser (==0.3.3)"] +dynamodbstreams = ["docker (>=3.0.0)", "py-partiql-parser (==0.3.3)"] ebs = ["sshpubkeys (>=3.1.0)"] ec2 = ["sshpubkeys (>=3.1.0)"] efs = ["sshpubkeys (>=3.1.0)"] @@ -1413,25 +1387,24 @@ eks = ["sshpubkeys (>=3.1.0)"] glue = ["pyparsing (>=3.0.7)"] iotdata = ["jsondiff (>=1.1.2)"] route53resolver = ["sshpubkeys (>=3.1.0)"] -s3 = ["PyYAML (>=5.1)", "py-partiql-parser (==0.3.0)"] -server = ["PyYAML (>=5.1)", "aws-xray-sdk (>=0.93,!=0.96)", "cfn-lint (>=0.40.0)", "docker (>=3.0.0)", "ecdsa (!=0.15)", "flask (!=2.2.0,!=2.2.1)", "flask-cors", "graphql-core", "jsondiff (>=1.1.2)", "openapi-spec-validator (>=0.2.8)", "py-partiql-parser (==0.3.0)", "pyparsing (>=3.0.7)", "python-jose[cryptography] (>=3.1.0,<4.0.0)", "setuptools", "sshpubkeys (>=3.1.0)"] +s3 = ["PyYAML (>=5.1)", "py-partiql-parser (==0.3.3)"] +server = ["PyYAML (>=5.1)", "aws-xray-sdk (>=0.93,!=0.96)", "cfn-lint (>=0.40.0)", "docker (>=3.0.0)", "ecdsa (!=0.15)", "flask (!=2.2.0,!=2.2.1)", "flask-cors", "graphql-core", "jsondiff (>=1.1.2)", "openapi-spec-validator (>=0.2.8)", "py-partiql-parser (==0.3.3)", "pyparsing (>=3.0.7)", "python-jose[cryptography] (>=3.1.0,<4.0.0)", "setuptools", "sshpubkeys (>=3.1.0)"] ssm = ["PyYAML (>=5.1)"] xray = ["aws-xray-sdk (>=0.93,!=0.96)", "setuptools"] [[package]] name = "msal" -version = "1.22.0" +version = "1.23.0" description = "The Microsoft Authentication Library (MSAL) for Python library enables your app to access the Microsoft Cloud by supporting authentication of users with Microsoft Azure Active Directory accounts (AAD) and Microsoft Accounts (MSA) using industry standard OAuth2 and OpenID Connect." -category = "main" optional = true python-versions = "*" files = [ - {file = "msal-1.22.0-py2.py3-none-any.whl", hash = "sha256:9120b7eafdf061c92f7b3d744e5f325fca35873445fa8ffebb40b1086a13dd58"}, - {file = "msal-1.22.0.tar.gz", hash = "sha256:8a82f5375642c1625c89058018430294c109440dce42ea667d466c2cab520acd"}, + {file = "msal-1.23.0-py2.py3-none-any.whl", hash = "sha256:3342e0837a047007f9d479e814b559c3219767453d57920dc40a31986862048b"}, + {file = "msal-1.23.0.tar.gz", hash = "sha256:25c9a33acf84301f93d1fdbe9f1a9c60cd38af0d5fffdbfa378138fc7bc1e86b"}, ] [package.dependencies] -cryptography = ">=0.6,<43" +cryptography = ">=0.6,<44" PyJWT = {version = ">=1.0.0,<3", extras = ["crypto"]} requests = ">=2.0.0,<3" @@ -1442,7 +1415,6 @@ broker = ["pymsalruntime (>=0.13.2,<0.14)"] name = "msal-extensions" version = "1.0.0" description = "Microsoft Authentication Library extensions (MSAL EX) provides a persistence API that can save your data on disk, encrypted on Windows, macOS and Linux. Concurrent data access will be coordinated by a file lock mechanism." -category = "main" optional = true python-versions = "*" files = [ @@ -1461,7 +1433,6 @@ portalocker = [ name = "msgpack" version = "1.0.5" description = "MessagePack serializer" -category = "main" optional = true python-versions = "*" files = [ @@ -1534,7 +1505,6 @@ files = [ name = "multidict" version = "6.0.4" description = "multidict implementation" -category = "main" optional = true python-versions = ">=3.7" files = [ @@ -1618,7 +1588,6 @@ files = [ name = "nodeenv" version = "1.8.0" description = "Node.js virtual environment builder" -category = "dev" optional = false python-versions = ">=2.7,!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*,!=3.4.*,!=3.5.*,!=3.6.*" files = [ @@ -1631,47 +1600,45 @@ setuptools = "*" [[package]] name = "numpy" -version = "1.24.3" +version = "1.24.4" description = "Fundamental package for array computing in Python" -category = "main" optional = true python-versions = ">=3.8" files = [ - {file = "numpy-1.24.3-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:3c1104d3c036fb81ab923f507536daedc718d0ad5a8707c6061cdfd6d184e570"}, - {file = "numpy-1.24.3-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:202de8f38fc4a45a3eea4b63e2f376e5f2dc64ef0fa692838e31a808520efaf7"}, - {file = "numpy-1.24.3-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8535303847b89aa6b0f00aa1dc62867b5a32923e4d1681a35b5eef2d9591a463"}, - {file = "numpy-1.24.3-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2d926b52ba1367f9acb76b0df6ed21f0b16a1ad87c6720a1121674e5cf63e2b6"}, - {file = "numpy-1.24.3-cp310-cp310-win32.whl", hash = "sha256:f21c442fdd2805e91799fbe044a7b999b8571bb0ab0f7850d0cb9641a687092b"}, - {file = "numpy-1.24.3-cp310-cp310-win_amd64.whl", hash = "sha256:ab5f23af8c16022663a652d3b25dcdc272ac3f83c3af4c02eb8b824e6b3ab9d7"}, - {file = "numpy-1.24.3-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:9a7721ec204d3a237225db3e194c25268faf92e19338a35f3a224469cb6039a3"}, - {file = "numpy-1.24.3-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:d6cc757de514c00b24ae8cf5c876af2a7c3df189028d68c0cb4eaa9cd5afc2bf"}, - {file = "numpy-1.24.3-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:76e3f4e85fc5d4fd311f6e9b794d0c00e7002ec122be271f2019d63376f1d385"}, - {file = "numpy-1.24.3-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a1d3c026f57ceaad42f8231305d4653d5f05dc6332a730ae5c0bea3513de0950"}, - {file = "numpy-1.24.3-cp311-cp311-win32.whl", hash = "sha256:c91c4afd8abc3908e00a44b2672718905b8611503f7ff87390cc0ac3423fb096"}, - {file = "numpy-1.24.3-cp311-cp311-win_amd64.whl", hash = "sha256:5342cf6aad47943286afa6f1609cad9b4266a05e7f2ec408e2cf7aea7ff69d80"}, - {file = "numpy-1.24.3-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:7776ea65423ca6a15255ba1872d82d207bd1e09f6d0894ee4a64678dd2204078"}, - {file = "numpy-1.24.3-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:ae8d0be48d1b6ed82588934aaaa179875e7dc4f3d84da18d7eae6eb3f06c242c"}, - {file = "numpy-1.24.3-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ecde0f8adef7dfdec993fd54b0f78183051b6580f606111a6d789cd14c61ea0c"}, - {file = "numpy-1.24.3-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4749e053a29364d3452c034827102ee100986903263e89884922ef01a0a6fd2f"}, - {file = "numpy-1.24.3-cp38-cp38-win32.whl", hash = "sha256:d933fabd8f6a319e8530d0de4fcc2e6a61917e0b0c271fded460032db42a0fe4"}, - {file = "numpy-1.24.3-cp38-cp38-win_amd64.whl", hash = "sha256:56e48aec79ae238f6e4395886b5eaed058abb7231fb3361ddd7bfdf4eed54289"}, - {file = "numpy-1.24.3-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:4719d5aefb5189f50887773699eaf94e7d1e02bf36c1a9d353d9f46703758ca4"}, - {file = "numpy-1.24.3-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:0ec87a7084caa559c36e0a2309e4ecb1baa03b687201d0a847c8b0ed476a7187"}, - {file = "numpy-1.24.3-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ea8282b9bcfe2b5e7d491d0bf7f3e2da29700cec05b49e64d6246923329f2b02"}, - {file = "numpy-1.24.3-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:210461d87fb02a84ef243cac5e814aad2b7f4be953b32cb53327bb49fd77fbb4"}, - {file = "numpy-1.24.3-cp39-cp39-win32.whl", hash = "sha256:784c6da1a07818491b0ffd63c6bbe5a33deaa0e25a20e1b3ea20cf0e43f8046c"}, - {file = "numpy-1.24.3-cp39-cp39-win_amd64.whl", hash = "sha256:d5036197ecae68d7f491fcdb4df90082b0d4960ca6599ba2659957aafced7c17"}, - {file = "numpy-1.24.3-pp38-pypy38_pp73-macosx_10_9_x86_64.whl", hash = "sha256:352ee00c7f8387b44d19f4cada524586f07379c0d49270f87233983bc5087ca0"}, - {file = "numpy-1.24.3-pp38-pypy38_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1a7d6acc2e7524c9955e5c903160aa4ea083736fde7e91276b0e5d98e6332812"}, - {file = "numpy-1.24.3-pp38-pypy38_pp73-win_amd64.whl", hash = "sha256:35400e6a8d102fd07c71ed7dcadd9eb62ee9a6e84ec159bd48c28235bbb0f8e4"}, - {file = "numpy-1.24.3.tar.gz", hash = "sha256:ab344f1bf21f140adab8e47fdbc7c35a477dc01408791f8ba00d018dd0bc5155"}, + {file = "numpy-1.24.4-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:c0bfb52d2169d58c1cdb8cc1f16989101639b34c7d3ce60ed70b19c63eba0b64"}, + {file = "numpy-1.24.4-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:ed094d4f0c177b1b8e7aa9cba7d6ceed51c0e569a5318ac0ca9a090680a6a1b1"}, + {file = "numpy-1.24.4-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:79fc682a374c4a8ed08b331bef9c5f582585d1048fa6d80bc6c35bc384eee9b4"}, + {file = "numpy-1.24.4-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7ffe43c74893dbf38c2b0a1f5428760a1a9c98285553c89e12d70a96a7f3a4d6"}, + {file = "numpy-1.24.4-cp310-cp310-win32.whl", hash = "sha256:4c21decb6ea94057331e111a5bed9a79d335658c27ce2adb580fb4d54f2ad9bc"}, + {file = "numpy-1.24.4-cp310-cp310-win_amd64.whl", hash = "sha256:b4bea75e47d9586d31e892a7401f76e909712a0fd510f58f5337bea9572c571e"}, + {file = "numpy-1.24.4-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:f136bab9c2cfd8da131132c2cf6cc27331dd6fae65f95f69dcd4ae3c3639c810"}, + {file = "numpy-1.24.4-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:e2926dac25b313635e4d6cf4dc4e51c8c0ebfed60b801c799ffc4c32bf3d1254"}, + {file = "numpy-1.24.4-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:222e40d0e2548690405b0b3c7b21d1169117391c2e82c378467ef9ab4c8f0da7"}, + {file = "numpy-1.24.4-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7215847ce88a85ce39baf9e89070cb860c98fdddacbaa6c0da3ffb31b3350bd5"}, + {file = "numpy-1.24.4-cp311-cp311-win32.whl", hash = "sha256:4979217d7de511a8d57f4b4b5b2b965f707768440c17cb70fbf254c4b225238d"}, + {file = "numpy-1.24.4-cp311-cp311-win_amd64.whl", hash = "sha256:b7b1fc9864d7d39e28f41d089bfd6353cb5f27ecd9905348c24187a768c79694"}, + {file = "numpy-1.24.4-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:1452241c290f3e2a312c137a9999cdbf63f78864d63c79039bda65ee86943f61"}, + {file = "numpy-1.24.4-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:04640dab83f7c6c85abf9cd729c5b65f1ebd0ccf9de90b270cd61935eef0197f"}, + {file = "numpy-1.24.4-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a5425b114831d1e77e4b5d812b69d11d962e104095a5b9c3b641a218abcc050e"}, + {file = "numpy-1.24.4-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:dd80e219fd4c71fc3699fc1dadac5dcf4fd882bfc6f7ec53d30fa197b8ee22dc"}, + {file = "numpy-1.24.4-cp38-cp38-win32.whl", hash = "sha256:4602244f345453db537be5314d3983dbf5834a9701b7723ec28923e2889e0bb2"}, + {file = "numpy-1.24.4-cp38-cp38-win_amd64.whl", hash = "sha256:692f2e0f55794943c5bfff12b3f56f99af76f902fc47487bdfe97856de51a706"}, + {file = "numpy-1.24.4-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:2541312fbf09977f3b3ad449c4e5f4bb55d0dbf79226d7724211acc905049400"}, + {file = "numpy-1.24.4-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:9667575fb6d13c95f1b36aca12c5ee3356bf001b714fc354eb5465ce1609e62f"}, + {file = "numpy-1.24.4-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f3a86ed21e4f87050382c7bc96571755193c4c1392490744ac73d660e8f564a9"}, + {file = "numpy-1.24.4-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d11efb4dbecbdf22508d55e48d9c8384db795e1b7b51ea735289ff96613ff74d"}, + {file = "numpy-1.24.4-cp39-cp39-win32.whl", hash = "sha256:6620c0acd41dbcb368610bb2f4d83145674040025e5536954782467100aa8835"}, + {file = "numpy-1.24.4-cp39-cp39-win_amd64.whl", hash = "sha256:befe2bf740fd8373cf56149a5c23a0f601e82869598d41f8e188a0e9869926f8"}, + {file = "numpy-1.24.4-pp38-pypy38_pp73-macosx_10_9_x86_64.whl", hash = "sha256:31f13e25b4e304632a4619d0e0777662c2ffea99fcae2029556b17d8ff958aef"}, + {file = "numpy-1.24.4-pp38-pypy38_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:95f7ac6540e95bc440ad77f56e520da5bf877f87dca58bd095288dce8940532a"}, + {file = "numpy-1.24.4-pp38-pypy38_pp73-win_amd64.whl", hash = "sha256:e98f220aa76ca2a977fe435f5b04d7b3470c0a2e6312907b37ba6068f26787f2"}, + {file = "numpy-1.24.4.tar.gz", hash = "sha256:80f5e3a4e498641401868df4208b74581206afbee7cf7b8329daae82676d9463"}, ] [[package]] name = "packaging" version = "23.1" description = "Core utilities for Python packages" -category = "main" optional = false python-versions = ">=3.7" files = [ @@ -1681,37 +1648,36 @@ files = [ [[package]] name = "pandas" -version = "2.0.2" +version = "2.0.3" description = "Powerful data structures for data analysis, time series, and statistics" -category = "main" optional = true python-versions = ">=3.8" files = [ - {file = "pandas-2.0.2-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:9ebb9f1c22ddb828e7fd017ea265a59d80461d5a79154b49a4207bd17514d122"}, - {file = "pandas-2.0.2-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:1eb09a242184092f424b2edd06eb2b99d06dc07eeddff9929e8667d4ed44e181"}, - {file = "pandas-2.0.2-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c7319b6e68de14e6209460f72a8d1ef13c09fb3d3ef6c37c1e65b35d50b5c145"}, - {file = "pandas-2.0.2-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:dd46bde7309088481b1cf9c58e3f0e204b9ff9e3244f441accd220dd3365ce7c"}, - {file = "pandas-2.0.2-cp310-cp310-win32.whl", hash = "sha256:51a93d422fbb1bd04b67639ba4b5368dffc26923f3ea32a275d2cc450f1d1c86"}, - {file = "pandas-2.0.2-cp310-cp310-win_amd64.whl", hash = "sha256:66d00300f188fa5de73f92d5725ced162488f6dc6ad4cecfe4144ca29debe3b8"}, - {file = "pandas-2.0.2-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:02755de164da6827764ceb3bbc5f64b35cb12394b1024fdf88704d0fa06e0e2f"}, - {file = "pandas-2.0.2-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:0a1e0576611641acde15c2322228d138258f236d14b749ad9af498ab69089e2d"}, - {file = "pandas-2.0.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a6b5f14cd24a2ed06e14255ff40fe2ea0cfaef79a8dd68069b7ace74bd6acbba"}, - {file = "pandas-2.0.2-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:50e451932b3011b61d2961b4185382c92cc8c6ee4658dcd4f320687bb2d000ee"}, - {file = "pandas-2.0.2-cp311-cp311-win32.whl", hash = "sha256:7b21cb72958fc49ad757685db1919021d99650d7aaba676576c9e88d3889d456"}, - {file = "pandas-2.0.2-cp311-cp311-win_amd64.whl", hash = "sha256:c4af689352c4fe3d75b2834933ee9d0ccdbf5d7a8a7264f0ce9524e877820c08"}, - {file = "pandas-2.0.2-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:69167693cb8f9b3fc060956a5d0a0a8dbfed5f980d9fd2c306fb5b9c855c814c"}, - {file = "pandas-2.0.2-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:30a89d0fec4263ccbf96f68592fd668939481854d2ff9da709d32a047689393b"}, - {file = "pandas-2.0.2-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a18e5c72b989ff0f7197707ceddc99828320d0ca22ab50dd1b9e37db45b010c0"}, - {file = "pandas-2.0.2-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7376e13d28eb16752c398ca1d36ccfe52bf7e887067af9a0474de6331dd948d2"}, - {file = "pandas-2.0.2-cp38-cp38-win32.whl", hash = "sha256:6d6d10c2142d11d40d6e6c0a190b1f89f525bcf85564707e31b0a39e3b398e08"}, - {file = "pandas-2.0.2-cp38-cp38-win_amd64.whl", hash = "sha256:e69140bc2d29a8556f55445c15f5794490852af3de0f609a24003ef174528b79"}, - {file = "pandas-2.0.2-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:b42b120458636a981077cfcfa8568c031b3e8709701315e2bfa866324a83efa8"}, - {file = "pandas-2.0.2-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:f908a77cbeef9bbd646bd4b81214cbef9ac3dda4181d5092a4aa9797d1bc7774"}, - {file = "pandas-2.0.2-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:713f2f70abcdade1ddd68fc91577cb090b3544b07ceba78a12f799355a13ee44"}, - {file = "pandas-2.0.2-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:cf3f0c361a4270185baa89ec7ab92ecaa355fe783791457077473f974f654df5"}, - {file = "pandas-2.0.2-cp39-cp39-win32.whl", hash = "sha256:598e9020d85a8cdbaa1815eb325a91cfff2bb2b23c1442549b8a3668e36f0f77"}, - {file = "pandas-2.0.2-cp39-cp39-win_amd64.whl", hash = "sha256:77550c8909ebc23e56a89f91b40ad01b50c42cfbfab49b3393694a50549295ea"}, - {file = "pandas-2.0.2.tar.gz", hash = "sha256:dd5476b6c3fe410ee95926873f377b856dbc4e81a9c605a0dc05aaccc6a7c6c6"}, + {file = "pandas-2.0.3-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:e4c7c9f27a4185304c7caf96dc7d91bc60bc162221152de697c98eb0b2648dd8"}, + {file = "pandas-2.0.3-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:f167beed68918d62bffb6ec64f2e1d8a7d297a038f86d4aed056b9493fca407f"}, + {file = "pandas-2.0.3-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ce0c6f76a0f1ba361551f3e6dceaff06bde7514a374aa43e33b588ec10420183"}, + {file = "pandas-2.0.3-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ba619e410a21d8c387a1ea6e8a0e49bb42216474436245718d7f2e88a2f8d7c0"}, + {file = "pandas-2.0.3-cp310-cp310-win32.whl", hash = "sha256:3ef285093b4fe5058eefd756100a367f27029913760773c8bf1d2d8bebe5d210"}, + {file = "pandas-2.0.3-cp310-cp310-win_amd64.whl", hash = "sha256:9ee1a69328d5c36c98d8e74db06f4ad518a1840e8ccb94a4ba86920986bb617e"}, + {file = "pandas-2.0.3-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:b084b91d8d66ab19f5bb3256cbd5ea661848338301940e17f4492b2ce0801fe8"}, + {file = "pandas-2.0.3-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:37673e3bdf1551b95bf5d4ce372b37770f9529743d2498032439371fc7b7eb26"}, + {file = "pandas-2.0.3-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b9cb1e14fdb546396b7e1b923ffaeeac24e4cedd14266c3497216dd4448e4f2d"}, + {file = "pandas-2.0.3-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d9cd88488cceb7635aebb84809d087468eb33551097d600c6dad13602029c2df"}, + {file = "pandas-2.0.3-cp311-cp311-win32.whl", hash = "sha256:694888a81198786f0e164ee3a581df7d505024fbb1f15202fc7db88a71d84ebd"}, + {file = "pandas-2.0.3-cp311-cp311-win_amd64.whl", hash = "sha256:6a21ab5c89dcbd57f78d0ae16630b090eec626360085a4148693def5452d8a6b"}, + {file = "pandas-2.0.3-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:9e4da0d45e7f34c069fe4d522359df7d23badf83abc1d1cef398895822d11061"}, + {file = "pandas-2.0.3-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:32fca2ee1b0d93dd71d979726b12b61faa06aeb93cf77468776287f41ff8fdc5"}, + {file = "pandas-2.0.3-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:258d3624b3ae734490e4d63c430256e716f488c4fcb7c8e9bde2d3aa46c29089"}, + {file = "pandas-2.0.3-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9eae3dc34fa1aa7772dd3fc60270d13ced7346fcbcfee017d3132ec625e23bb0"}, + {file = "pandas-2.0.3-cp38-cp38-win32.whl", hash = "sha256:f3421a7afb1a43f7e38e82e844e2bca9a6d793d66c1a7f9f0ff39a795bbc5e02"}, + {file = "pandas-2.0.3-cp38-cp38-win_amd64.whl", hash = "sha256:69d7f3884c95da3a31ef82b7618af5710dba95bb885ffab339aad925c3e8ce78"}, + {file = "pandas-2.0.3-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:5247fb1ba347c1261cbbf0fcfba4a3121fbb4029d95d9ef4dc45406620b25c8b"}, + {file = "pandas-2.0.3-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:81af086f4543c9d8bb128328b5d32e9986e0c84d3ee673a2ac6fb57fd14f755e"}, + {file = "pandas-2.0.3-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1994c789bf12a7c5098277fb43836ce090f1073858c10f9220998ac74f37c69b"}, + {file = "pandas-2.0.3-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:5ec591c48e29226bcbb316e0c1e9423622bc7a4eaf1ef7c3c9fa1a3981f89641"}, + {file = "pandas-2.0.3-cp39-cp39-win32.whl", hash = "sha256:04dbdbaf2e4d46ca8da896e1805bc04eb85caa9a82e259e8eed00254d5e0c682"}, + {file = "pandas-2.0.3-cp39-cp39-win_amd64.whl", hash = "sha256:1168574b036cd8b93abc746171c9b4f1b83467438a5e45909fed645cf8692dbc"}, + {file = "pandas-2.0.3.tar.gz", hash = "sha256:c02f372a88e0d17f36d3093a644c73cfc1788e876a7c4bcb4020a77512e2043c"}, ] [package.dependencies] @@ -1725,7 +1691,7 @@ pytz = ">=2020.1" tzdata = ">=2022.1" [package.extras] -all = ["PyQt5 (>=5.15.1)", "SQLAlchemy (>=1.4.16)", "beautifulsoup4 (>=4.9.3)", "bottleneck (>=1.3.2)", "brotlipy (>=0.7.0)", "fastparquet (>=0.6.3)", "fsspec (>=2021.07.0)", "gcsfs (>=2021.07.0)", "html5lib (>=1.1)", "hypothesis (>=6.34.2)", "jinja2 (>=3.0.0)", "lxml (>=4.6.3)", "matplotlib (>=3.6.1)", "numba (>=0.53.1)", "numexpr (>=2.7.3)", "odfpy (>=1.4.1)", "openpyxl (>=3.0.7)", "pandas-gbq (>=0.15.0)", "psycopg2 (>=2.8.6)", "pyarrow (>=7.0.0)", "pymysql (>=1.0.2)", "pyreadstat (>=1.1.2)", "pytest (>=7.0.0)", "pytest-asyncio (>=0.17.0)", "pytest-xdist (>=2.2.0)", "python-snappy (>=0.6.0)", "pyxlsb (>=1.0.8)", "qtpy (>=2.2.0)", "s3fs (>=2021.08.0)", "scipy (>=1.7.1)", "tables (>=3.6.1)", "tabulate (>=0.8.9)", "xarray (>=0.21.0)", "xlrd (>=2.0.1)", "xlsxwriter (>=1.4.3)", "zstandard (>=0.15.2)"] +all = ["PyQt5 (>=5.15.1)", "SQLAlchemy (>=1.4.16)", "beautifulsoup4 (>=4.9.3)", "bottleneck (>=1.3.2)", "brotlipy (>=0.7.0)", "fastparquet (>=0.6.3)", "fsspec (>=2021.07.0)", "gcsfs (>=2021.07.0)", "html5lib (>=1.1)", "hypothesis (>=6.34.2)", "jinja2 (>=3.0.0)", "lxml (>=4.6.3)", "matplotlib (>=3.6.1)", "numba (>=0.53.1)", "numexpr (>=2.7.3)", "odfpy (>=1.4.1)", "openpyxl (>=3.0.7)", "pandas-gbq (>=0.15.0)", "psycopg2 (>=2.8.6)", "pyarrow (>=7.0.0)", "pymysql (>=1.0.2)", "pyreadstat (>=1.1.2)", "pytest (>=7.3.2)", "pytest-asyncio (>=0.17.0)", "pytest-xdist (>=2.2.0)", "python-snappy (>=0.6.0)", "pyxlsb (>=1.0.8)", "qtpy (>=2.2.0)", "s3fs (>=2021.08.0)", "scipy (>=1.7.1)", "tables (>=3.6.1)", "tabulate (>=0.8.9)", "xarray (>=0.21.0)", "xlrd (>=2.0.1)", "xlsxwriter (>=1.4.3)", "zstandard (>=0.15.2)"] aws = ["s3fs (>=2021.08.0)"] clipboard = ["PyQt5 (>=5.15.1)", "qtpy (>=2.2.0)"] compression = ["brotlipy (>=0.7.0)", "python-snappy (>=0.6.0)", "zstandard (>=0.15.2)"] @@ -1744,14 +1710,13 @@ plot = ["matplotlib (>=3.6.1)"] postgresql = ["SQLAlchemy (>=1.4.16)", "psycopg2 (>=2.8.6)"] spss = ["pyreadstat (>=1.1.2)"] sql-other = ["SQLAlchemy (>=1.4.16)"] -test = ["hypothesis (>=6.34.2)", "pytest (>=7.0.0)", "pytest-asyncio (>=0.17.0)", "pytest-xdist (>=2.2.0)"] +test = ["hypothesis (>=6.34.2)", "pytest (>=7.3.2)", "pytest-asyncio (>=0.17.0)", "pytest-xdist (>=2.2.0)"] xml = ["lxml (>=4.6.3)"] [[package]] name = "pkgutil-resolve-name" version = "1.3.10" description = "Resolve a name to an object." -category = "main" optional = true python-versions = ">=3.6" files = [ @@ -1761,30 +1726,28 @@ files = [ [[package]] name = "platformdirs" -version = "3.5.1" +version = "3.9.1" description = "A small Python package for determining appropriate platform-specific dirs, e.g. a \"user data dir\"." -category = "main" optional = false python-versions = ">=3.7" files = [ - {file = "platformdirs-3.5.1-py3-none-any.whl", hash = "sha256:e2378146f1964972c03c085bb5662ae80b2b8c06226c54b2ff4aa9483e8a13a5"}, - {file = "platformdirs-3.5.1.tar.gz", hash = "sha256:412dae91f52a6f84830f39a8078cecd0e866cb72294a5c66808e74d5e88d251f"}, + {file = "platformdirs-3.9.1-py3-none-any.whl", hash = "sha256:ad8291ae0ae5072f66c16945166cb11c63394c7a3ad1b1bc9828ca3162da8c2f"}, + {file = "platformdirs-3.9.1.tar.gz", hash = "sha256:1b42b450ad933e981d56e59f1b97495428c9bd60698baab9f3eb3d00d5822421"}, ] [package.extras] -docs = ["furo (>=2023.3.27)", "proselint (>=0.13)", "sphinx (>=6.2.1)", "sphinx-autodoc-typehints (>=1.23,!=1.23.4)"] -test = ["appdirs (==1.4.4)", "covdefaults (>=2.3)", "pytest (>=7.3.1)", "pytest-cov (>=4)", "pytest-mock (>=3.10)"] +docs = ["furo (>=2023.5.20)", "proselint (>=0.13)", "sphinx (>=7.0.1)", "sphinx-autodoc-typehints (>=1.23,!=1.23.4)"] +test = ["appdirs (==1.4.4)", "covdefaults (>=2.3)", "pytest (>=7.3.1)", "pytest-cov (>=4.1)", "pytest-mock (>=3.10)"] [[package]] name = "pluggy" -version = "1.0.0" +version = "1.2.0" description = "plugin and hook calling mechanisms for python" -category = "dev" optional = false -python-versions = ">=3.6" +python-versions = ">=3.7" files = [ - {file = "pluggy-1.0.0-py2.py3-none-any.whl", hash = "sha256:74134bbf457f031a36d68416e1509f34bd5ccc019f0bcc952c7b909d06b37bd3"}, - {file = "pluggy-1.0.0.tar.gz", hash = "sha256:4224373bacce55f955a878bf9cfa763c1e360858e330072059e10bad68531159"}, + {file = "pluggy-1.2.0-py3-none-any.whl", hash = "sha256:c2fd55a7d7a3863cba1a013e4e2414658b1d07b6bc57b3919e0c63c9abb99849"}, + {file = "pluggy-1.2.0.tar.gz", hash = "sha256:d12f0c4b579b15f5e054301bb226ee85eeeba08ffec228092f8defbaa3a4c4b3"}, ] [package.extras] @@ -1795,7 +1758,6 @@ testing = ["pytest", "pytest-benchmark"] name = "portalocker" version = "2.7.0" description = "Wraps the portalocker recipe for easy usage" -category = "main" optional = true python-versions = ">=3.5" files = [ @@ -1813,14 +1775,13 @@ tests = ["pytest (>=5.4.1)", "pytest-cov (>=2.8.1)", "pytest-mypy (>=0.8.0)", "p [[package]] name = "pre-commit" -version = "3.3.2" +version = "3.3.3" description = "A framework for managing and maintaining multi-language pre-commit hooks." -category = "dev" optional = false python-versions = ">=3.8" files = [ - {file = "pre_commit-3.3.2-py2.py3-none-any.whl", hash = "sha256:8056bc52181efadf4aac792b1f4f255dfd2fb5a350ded7335d251a68561e8cb6"}, - {file = "pre_commit-3.3.2.tar.gz", hash = "sha256:66e37bec2d882de1f17f88075047ef8962581f83c234ac08da21a0c58953d1f0"}, + {file = "pre_commit-3.3.3-py2.py3-none-any.whl", hash = "sha256:10badb65d6a38caff29703362271d7dca483d01da88f9d7e05d0b97171c136cb"}, + {file = "pre_commit-3.3.3.tar.gz", hash = "sha256:a2256f489cd913d575c145132ae196fe335da32d91a8294b7afe6622335dd023"}, ] [package.dependencies] @@ -1832,60 +1793,129 @@ virtualenv = ">=20.10.0" [[package]] name = "protobuf" -version = "4.23.2" +version = "4.23.4" description = "" -category = "main" optional = true python-versions = ">=3.7" files = [ - {file = "protobuf-4.23.2-cp310-abi3-win32.whl", hash = "sha256:384dd44cb4c43f2ccddd3645389a23ae61aeb8cfa15ca3a0f60e7c3ea09b28b3"}, - {file = "protobuf-4.23.2-cp310-abi3-win_amd64.whl", hash = "sha256:09310bce43353b46d73ba7e3bca78273b9bc50349509b9698e64d288c6372c2a"}, - {file = "protobuf-4.23.2-cp37-abi3-macosx_10_9_universal2.whl", hash = "sha256:b2cfab63a230b39ae603834718db74ac11e52bccaaf19bf20f5cce1a84cf76df"}, - {file = "protobuf-4.23.2-cp37-abi3-manylinux2014_aarch64.whl", hash = "sha256:c52cfcbfba8eb791255edd675c1fe6056f723bf832fa67f0442218f8817c076e"}, - {file = "protobuf-4.23.2-cp37-abi3-manylinux2014_x86_64.whl", hash = "sha256:86df87016d290143c7ce3be3ad52d055714ebaebb57cc659c387e76cfacd81aa"}, - {file = "protobuf-4.23.2-cp37-cp37m-win32.whl", hash = "sha256:281342ea5eb631c86697e1e048cb7e73b8a4e85f3299a128c116f05f5c668f8f"}, - {file = "protobuf-4.23.2-cp37-cp37m-win_amd64.whl", hash = "sha256:ce744938406de1e64b91410f473736e815f28c3b71201302612a68bf01517fea"}, - {file = "protobuf-4.23.2-cp38-cp38-win32.whl", hash = "sha256:6c081863c379bb1741be8f8193e893511312b1d7329b4a75445d1ea9955be69e"}, - {file = "protobuf-4.23.2-cp38-cp38-win_amd64.whl", hash = "sha256:25e3370eda26469b58b602e29dff069cfaae8eaa0ef4550039cc5ef8dc004511"}, - {file = "protobuf-4.23.2-cp39-cp39-win32.whl", hash = "sha256:efabbbbac1ab519a514579ba9ec52f006c28ae19d97915951f69fa70da2c9e91"}, - {file = "protobuf-4.23.2-cp39-cp39-win_amd64.whl", hash = "sha256:54a533b971288af3b9926e53850c7eb186886c0c84e61daa8444385a4720297f"}, - {file = "protobuf-4.23.2-py3-none-any.whl", hash = "sha256:8da6070310d634c99c0db7df48f10da495cc283fd9e9234877f0cd182d43ab7f"}, - {file = "protobuf-4.23.2.tar.gz", hash = "sha256:20874e7ca4436f683b64ebdbee2129a5a2c301579a67d1a7dda2cdf62fb7f5f7"}, + {file = "protobuf-4.23.4-cp310-abi3-win32.whl", hash = "sha256:5fea3c64d41ea5ecf5697b83e41d09b9589e6f20b677ab3c48e5f242d9b7897b"}, + {file = "protobuf-4.23.4-cp310-abi3-win_amd64.whl", hash = "sha256:7b19b6266d92ca6a2a87effa88ecc4af73ebc5cfde194dc737cf8ef23a9a3b12"}, + {file = "protobuf-4.23.4-cp37-abi3-macosx_10_9_universal2.whl", hash = "sha256:8547bf44fe8cec3c69e3042f5c4fb3e36eb2a7a013bb0a44c018fc1e427aafbd"}, + {file = "protobuf-4.23.4-cp37-abi3-manylinux2014_aarch64.whl", hash = "sha256:fee88269a090ada09ca63551bf2f573eb2424035bcf2cb1b121895b01a46594a"}, + {file = "protobuf-4.23.4-cp37-abi3-manylinux2014_x86_64.whl", hash = "sha256:effeac51ab79332d44fba74660d40ae79985901ac21bca408f8dc335a81aa597"}, + {file = "protobuf-4.23.4-cp37-cp37m-win32.whl", hash = "sha256:c3e0939433c40796ca4cfc0fac08af50b00eb66a40bbbc5dee711998fb0bbc1e"}, + {file = "protobuf-4.23.4-cp37-cp37m-win_amd64.whl", hash = "sha256:9053df6df8e5a76c84339ee4a9f5a2661ceee4a0dab019e8663c50ba324208b0"}, + {file = "protobuf-4.23.4-cp38-cp38-win32.whl", hash = "sha256:e1c915778d8ced71e26fcf43c0866d7499891bca14c4368448a82edc61fdbc70"}, + {file = "protobuf-4.23.4-cp38-cp38-win_amd64.whl", hash = "sha256:351cc90f7d10839c480aeb9b870a211e322bf05f6ab3f55fcb2f51331f80a7d2"}, + {file = "protobuf-4.23.4-cp39-cp39-win32.whl", hash = "sha256:6dd9b9940e3f17077e820b75851126615ee38643c2c5332aa7a359988820c720"}, + {file = "protobuf-4.23.4-cp39-cp39-win_amd64.whl", hash = "sha256:0a5759f5696895de8cc913f084e27fd4125e8fb0914bb729a17816a33819f474"}, + {file = "protobuf-4.23.4-py3-none-any.whl", hash = "sha256:e9d0be5bf34b275b9f87ba7407796556abeeba635455d036c7351f7c183ef8ff"}, + {file = "protobuf-4.23.4.tar.gz", hash = "sha256:ccd9430c0719dce806b93f89c91de7977304729e55377f872a92465d548329a9"}, +] + +[[package]] +name = "psycopg2-binary" +version = "2.9.6" +description = "psycopg2 - Python-PostgreSQL Database Adapter" +optional = true +python-versions = ">=3.6" +files = [ + {file = "psycopg2-binary-2.9.6.tar.gz", hash = "sha256:1f64dcfb8f6e0c014c7f55e51c9759f024f70ea572fbdef123f85318c297947c"}, + {file = "psycopg2_binary-2.9.6-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:d26e0342183c762de3276cca7a530d574d4e25121ca7d6e4a98e4f05cb8e4df7"}, + {file = "psycopg2_binary-2.9.6-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:c48d8f2db17f27d41fb0e2ecd703ea41984ee19362cbce52c097963b3a1b4365"}, + {file = "psycopg2_binary-2.9.6-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ffe9dc0a884a8848075e576c1de0290d85a533a9f6e9c4e564f19adf8f6e54a7"}, + {file = "psycopg2_binary-2.9.6-cp310-cp310-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:8a76e027f87753f9bd1ab5f7c9cb8c7628d1077ef927f5e2446477153a602f2c"}, + {file = "psycopg2_binary-2.9.6-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:6460c7a99fc939b849431f1e73e013d54aa54293f30f1109019c56a0b2b2ec2f"}, + {file = "psycopg2_binary-2.9.6-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ae102a98c547ee2288637af07393dd33f440c25e5cd79556b04e3fca13325e5f"}, + {file = "psycopg2_binary-2.9.6-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:9972aad21f965599ed0106f65334230ce826e5ae69fda7cbd688d24fa922415e"}, + {file = "psycopg2_binary-2.9.6-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:7a40c00dbe17c0af5bdd55aafd6ff6679f94a9be9513a4c7e071baf3d7d22a70"}, + {file = "psycopg2_binary-2.9.6-cp310-cp310-musllinux_1_1_ppc64le.whl", hash = "sha256:cacbdc5839bdff804dfebc058fe25684cae322987f7a38b0168bc1b2df703fb1"}, + {file = "psycopg2_binary-2.9.6-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:7f0438fa20fb6c7e202863e0d5ab02c246d35efb1d164e052f2f3bfe2b152bd0"}, + {file = "psycopg2_binary-2.9.6-cp310-cp310-win32.whl", hash = "sha256:b6c8288bb8a84b47e07013bb4850f50538aa913d487579e1921724631d02ea1b"}, + {file = "psycopg2_binary-2.9.6-cp310-cp310-win_amd64.whl", hash = "sha256:61b047a0537bbc3afae10f134dc6393823882eb263088c271331602b672e52e9"}, + {file = "psycopg2_binary-2.9.6-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:964b4dfb7c1c1965ac4c1978b0f755cc4bd698e8aa2b7667c575fb5f04ebe06b"}, + {file = "psycopg2_binary-2.9.6-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:afe64e9b8ea66866a771996f6ff14447e8082ea26e675a295ad3bdbffdd72afb"}, + {file = "psycopg2_binary-2.9.6-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:15e2ee79e7cf29582ef770de7dab3d286431b01c3bb598f8e05e09601b890081"}, + {file = "psycopg2_binary-2.9.6-cp311-cp311-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:dfa74c903a3c1f0d9b1c7e7b53ed2d929a4910e272add6700c38f365a6002820"}, + {file = "psycopg2_binary-2.9.6-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:b83456c2d4979e08ff56180a76429263ea254c3f6552cd14ada95cff1dec9bb8"}, + {file = "psycopg2_binary-2.9.6-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0645376d399bfd64da57148694d78e1f431b1e1ee1054872a5713125681cf1be"}, + {file = "psycopg2_binary-2.9.6-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:e99e34c82309dd78959ba3c1590975b5d3c862d6f279f843d47d26ff89d7d7e1"}, + {file = "psycopg2_binary-2.9.6-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:4ea29fc3ad9d91162c52b578f211ff1c931d8a38e1f58e684c45aa470adf19e2"}, + {file = "psycopg2_binary-2.9.6-cp311-cp311-musllinux_1_1_ppc64le.whl", hash = "sha256:4ac30da8b4f57187dbf449294d23b808f8f53cad6b1fc3623fa8a6c11d176dd0"}, + {file = "psycopg2_binary-2.9.6-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:e78e6e2a00c223e164c417628572a90093c031ed724492c763721c2e0bc2a8df"}, + {file = "psycopg2_binary-2.9.6-cp311-cp311-win32.whl", hash = "sha256:1876843d8e31c89c399e31b97d4b9725a3575bb9c2af92038464231ec40f9edb"}, + {file = "psycopg2_binary-2.9.6-cp311-cp311-win_amd64.whl", hash = "sha256:b4b24f75d16a89cc6b4cdff0eb6a910a966ecd476d1e73f7ce5985ff1328e9a6"}, + {file = "psycopg2_binary-2.9.6-cp36-cp36m-win32.whl", hash = "sha256:498807b927ca2510baea1b05cc91d7da4718a0f53cb766c154c417a39f1820a0"}, + {file = "psycopg2_binary-2.9.6-cp36-cp36m-win_amd64.whl", hash = "sha256:0d236c2825fa656a2d98bbb0e52370a2e852e5a0ec45fc4f402977313329174d"}, + {file = "psycopg2_binary-2.9.6-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:34b9ccdf210cbbb1303c7c4db2905fa0319391bd5904d32689e6dd5c963d2ea8"}, + {file = "psycopg2_binary-2.9.6-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:84d2222e61f313c4848ff05353653bf5f5cf6ce34df540e4274516880d9c3763"}, + {file = "psycopg2_binary-2.9.6-cp37-cp37m-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:30637a20623e2a2eacc420059be11527f4458ef54352d870b8181a4c3020ae6b"}, + {file = "psycopg2_binary-2.9.6-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:8122cfc7cae0da9a3077216528b8bb3629c43b25053284cc868744bfe71eb141"}, + {file = "psycopg2_binary-2.9.6-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:38601cbbfe600362c43714482f43b7c110b20cb0f8172422c616b09b85a750c5"}, + {file = "psycopg2_binary-2.9.6-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:c7e62ab8b332147a7593a385d4f368874d5fe4ad4e341770d4983442d89603e3"}, + {file = "psycopg2_binary-2.9.6-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:2ab652e729ff4ad76d400df2624d223d6e265ef81bb8aa17fbd63607878ecbee"}, + {file = "psycopg2_binary-2.9.6-cp37-cp37m-musllinux_1_1_ppc64le.whl", hash = "sha256:c83a74b68270028dc8ee74d38ecfaf9c90eed23c8959fca95bd703d25b82c88e"}, + {file = "psycopg2_binary-2.9.6-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:d4e6036decf4b72d6425d5b29bbd3e8f0ff1059cda7ac7b96d6ac5ed34ffbacd"}, + {file = "psycopg2_binary-2.9.6-cp37-cp37m-win32.whl", hash = "sha256:a8c28fd40a4226b4a84bdf2d2b5b37d2c7bd49486b5adcc200e8c7ec991dfa7e"}, + {file = "psycopg2_binary-2.9.6-cp37-cp37m-win_amd64.whl", hash = "sha256:51537e3d299be0db9137b321dfb6a5022caaab275775680e0c3d281feefaca6b"}, + {file = "psycopg2_binary-2.9.6-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:cf4499e0a83b7b7edcb8dabecbd8501d0d3a5ef66457200f77bde3d210d5debb"}, + {file = "psycopg2_binary-2.9.6-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:7e13a5a2c01151f1208d5207e42f33ba86d561b7a89fca67c700b9486a06d0e2"}, + {file = "psycopg2_binary-2.9.6-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0e0f754d27fddcfd74006455b6e04e6705d6c31a612ec69ddc040a5468e44b4e"}, + {file = "psycopg2_binary-2.9.6-cp38-cp38-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d57c3fd55d9058645d26ae37d76e61156a27722097229d32a9e73ed54819982a"}, + {file = "psycopg2_binary-2.9.6-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:71f14375d6f73b62800530b581aed3ada394039877818b2d5f7fc77e3bb6894d"}, + {file = "psycopg2_binary-2.9.6-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:441cc2f8869a4f0f4bb408475e5ae0ee1f3b55b33f350406150277f7f35384fc"}, + {file = "psycopg2_binary-2.9.6-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:65bee1e49fa6f9cf327ce0e01c4c10f39165ee76d35c846ade7cb0ec6683e303"}, + {file = "psycopg2_binary-2.9.6-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:af335bac6b666cc6aea16f11d486c3b794029d9df029967f9938a4bed59b6a19"}, + {file = "psycopg2_binary-2.9.6-cp38-cp38-musllinux_1_1_ppc64le.whl", hash = "sha256:cfec476887aa231b8548ece2e06d28edc87c1397ebd83922299af2e051cf2827"}, + {file = "psycopg2_binary-2.9.6-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:65c07febd1936d63bfde78948b76cd4c2a411572a44ac50719ead41947d0f26b"}, + {file = "psycopg2_binary-2.9.6-cp38-cp38-win32.whl", hash = "sha256:4dfb4be774c4436a4526d0c554af0cc2e02082c38303852a36f6456ece7b3503"}, + {file = "psycopg2_binary-2.9.6-cp38-cp38-win_amd64.whl", hash = "sha256:02c6e3cf3439e213e4ee930308dc122d6fb4d4bea9aef4a12535fbd605d1a2fe"}, + {file = "psycopg2_binary-2.9.6-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:e9182eb20f41417ea1dd8e8f7888c4d7c6e805f8a7c98c1081778a3da2bee3e4"}, + {file = "psycopg2_binary-2.9.6-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:8a6979cf527e2603d349a91060f428bcb135aea2be3201dff794813256c274f1"}, + {file = "psycopg2_binary-2.9.6-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8338a271cb71d8da40b023a35d9c1e919eba6cbd8fa20a54b748a332c355d896"}, + {file = "psycopg2_binary-2.9.6-cp39-cp39-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e3ed340d2b858d6e6fb5083f87c09996506af483227735de6964a6100b4e6a54"}, + {file = "psycopg2_binary-2.9.6-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:f81e65376e52f03422e1fb475c9514185669943798ed019ac50410fb4c4df232"}, + {file = "psycopg2_binary-2.9.6-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bfb13af3c5dd3a9588000910178de17010ebcccd37b4f9794b00595e3a8ddad3"}, + {file = "psycopg2_binary-2.9.6-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:4c727b597c6444a16e9119386b59388f8a424223302d0c06c676ec8b4bc1f963"}, + {file = "psycopg2_binary-2.9.6-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:4d67fbdaf177da06374473ef6f7ed8cc0a9dc640b01abfe9e8a2ccb1b1402c1f"}, + {file = "psycopg2_binary-2.9.6-cp39-cp39-musllinux_1_1_ppc64le.whl", hash = "sha256:0892ef645c2fabb0c75ec32d79f4252542d0caec1d5d949630e7d242ca4681a3"}, + {file = "psycopg2_binary-2.9.6-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:02c0f3757a4300cf379eb49f543fb7ac527fb00144d39246ee40e1df684ab514"}, + {file = "psycopg2_binary-2.9.6-cp39-cp39-win32.whl", hash = "sha256:c3dba7dab16709a33a847e5cd756767271697041fbe3fe97c215b1fc1f5c9848"}, + {file = "psycopg2_binary-2.9.6-cp39-cp39-win_amd64.whl", hash = "sha256:f6a88f384335bb27812293fdb11ac6aee2ca3f51d3c7820fe03de0a304ab6249"}, ] [[package]] name = "pyarrow" -version = "12.0.0" +version = "12.0.1" description = "Python library for Apache Arrow" -category = "main" optional = true python-versions = ">=3.7" files = [ - {file = "pyarrow-12.0.0-cp310-cp310-macosx_10_14_x86_64.whl", hash = "sha256:3b97649c8a9a09e1d8dc76513054f1331bd9ece78ee39365e6bf6bc7503c1e94"}, - {file = "pyarrow-12.0.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:bc4ea634dacb03936f50fcf59574a8e727f90c17c24527e488d8ceb52ae284de"}, - {file = "pyarrow-12.0.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1d568acfca3faa565d663e53ee34173be8e23a95f78f2abfdad198010ec8f745"}, - {file = "pyarrow-12.0.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1b50bb9a82dca38a002d7cbd802a16b1af0f8c50ed2ec94a319f5f2afc047ee9"}, - {file = "pyarrow-12.0.0-cp310-cp310-win_amd64.whl", hash = "sha256:3d1733b1ea086b3c101427d0e57e2be3eb964686e83c2363862a887bb5c41fa8"}, - {file = "pyarrow-12.0.0-cp311-cp311-macosx_10_14_x86_64.whl", hash = "sha256:a7cd32fe77f967fe08228bc100433273020e58dd6caced12627bcc0a7675a513"}, - {file = "pyarrow-12.0.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:92fb031e6777847f5c9b01eaa5aa0c9033e853ee80117dce895f116d8b0c3ca3"}, - {file = "pyarrow-12.0.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:280289ebfd4ac3570f6b776515baa01e4dcbf17122c401e4b7170a27c4be63fd"}, - {file = "pyarrow-12.0.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:272f147d4f8387bec95f17bb58dcfc7bc7278bb93e01cb7b08a0e93a8921e18e"}, - {file = "pyarrow-12.0.0-cp311-cp311-win_amd64.whl", hash = "sha256:0846ace49998825eda4722f8d7f83fa05601c832549c9087ea49d6d5397d8cec"}, - {file = "pyarrow-12.0.0-cp37-cp37m-macosx_10_14_x86_64.whl", hash = "sha256:993287136369aca60005ee7d64130f9466489c4f7425f5c284315b0a5401ccd9"}, - {file = "pyarrow-12.0.0-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:7a7b6a765ee4f88efd7d8348d9a1f804487d60799d0428b6ddf3344eaef37282"}, - {file = "pyarrow-12.0.0-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a1c4fce253d5bdc8d62f11cfa3da5b0b34b562c04ce84abb8bd7447e63c2b327"}, - {file = "pyarrow-12.0.0-cp37-cp37m-win_amd64.whl", hash = "sha256:e6be4d85707fc8e7a221c8ab86a40449ce62559ce25c94321df7c8500245888f"}, - {file = "pyarrow-12.0.0-cp38-cp38-macosx_10_14_x86_64.whl", hash = "sha256:ea830d9f66bfb82d30b5794642f83dd0e4a718846462d22328981e9eb149cba8"}, - {file = "pyarrow-12.0.0-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:7b5b9f60d9ef756db59bec8d90e4576b7df57861e6a3d6a8bf99538f68ca15b3"}, - {file = "pyarrow-12.0.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b99e559d27db36ad3a33868a475f03e3129430fc065accc839ef4daa12c6dab6"}, - {file = "pyarrow-12.0.0-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:5b0810864a593b89877120972d1f7af1d1c9389876dbed92b962ed81492d3ffc"}, - {file = "pyarrow-12.0.0-cp38-cp38-win_amd64.whl", hash = "sha256:23a77d97f4d101ddfe81b9c2ee03a177f0e590a7e68af15eafa06e8f3cf05976"}, - {file = "pyarrow-12.0.0-cp39-cp39-macosx_10_14_x86_64.whl", hash = "sha256:2cc63e746221cddb9001f7281dee95fd658085dd5b717b076950e1ccc607059c"}, - {file = "pyarrow-12.0.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:d8c26912607e26c2991826bbaf3cf2b9c8c3e17566598c193b492f058b40d3a4"}, - {file = "pyarrow-12.0.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0d8b90efc290e99a81d06015f3a46601c259ecc81ffb6d8ce288c91bd1b868c9"}, - {file = "pyarrow-12.0.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2466be046b81863be24db370dffd30a2e7894b4f9823fb60ef0a733c31ac6256"}, - {file = "pyarrow-12.0.0-cp39-cp39-win_amd64.whl", hash = "sha256:0e36425b1c1cbf5447718b3f1751bf86c58f2b3ad299f996cd9b1aa040967656"}, - {file = "pyarrow-12.0.0.tar.gz", hash = "sha256:19c812d303610ab5d664b7b1de4051ae23565f9f94d04cbea9e50569746ae1ee"}, + {file = "pyarrow-12.0.1-cp310-cp310-macosx_10_14_x86_64.whl", hash = "sha256:6d288029a94a9bb5407ceebdd7110ba398a00412c5b0155ee9813a40d246c5df"}, + {file = "pyarrow-12.0.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:345e1828efdbd9aa4d4de7d5676778aba384a2c3add896d995b23d368e60e5af"}, + {file = "pyarrow-12.0.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8d6009fdf8986332b2169314da482baed47ac053311c8934ac6651e614deacd6"}, + {file = "pyarrow-12.0.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2d3c4cbbf81e6dd23fe921bc91dc4619ea3b79bc58ef10bce0f49bdafb103daf"}, + {file = "pyarrow-12.0.1-cp310-cp310-win_amd64.whl", hash = "sha256:cdacf515ec276709ac8042c7d9bd5be83b4f5f39c6c037a17a60d7ebfd92c890"}, + {file = "pyarrow-12.0.1-cp311-cp311-macosx_10_14_x86_64.whl", hash = "sha256:749be7fd2ff260683f9cc739cb862fb11be376de965a2a8ccbf2693b098db6c7"}, + {file = "pyarrow-12.0.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:6895b5fb74289d055c43db3af0de6e16b07586c45763cb5e558d38b86a91e3a7"}, + {file = "pyarrow-12.0.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1887bdae17ec3b4c046fcf19951e71b6a619f39fa674f9881216173566c8f718"}, + {file = "pyarrow-12.0.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e2c9cb8eeabbadf5fcfc3d1ddea616c7ce893db2ce4dcef0ac13b099ad7ca082"}, + {file = "pyarrow-12.0.1-cp311-cp311-win_amd64.whl", hash = "sha256:ce4aebdf412bd0eeb800d8e47db854f9f9f7e2f5a0220440acf219ddfddd4f63"}, + {file = "pyarrow-12.0.1-cp37-cp37m-macosx_10_14_x86_64.whl", hash = "sha256:e0d8730c7f6e893f6db5d5b86eda42c0a130842d101992b581e2138e4d5663d3"}, + {file = "pyarrow-12.0.1-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:43364daec02f69fec89d2315f7fbfbeec956e0d991cbbef471681bd77875c40f"}, + {file = "pyarrow-12.0.1-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:051f9f5ccf585f12d7de836e50965b3c235542cc896959320d9776ab93f3b33d"}, + {file = "pyarrow-12.0.1-cp37-cp37m-win_amd64.whl", hash = "sha256:be2757e9275875d2a9c6e6052ac7957fbbfc7bc7370e4a036a9b893e96fedaba"}, + {file = "pyarrow-12.0.1-cp38-cp38-macosx_10_14_x86_64.whl", hash = "sha256:cf812306d66f40f69e684300f7af5111c11f6e0d89d6b733e05a3de44961529d"}, + {file = "pyarrow-12.0.1-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:459a1c0ed2d68671188b2118c63bac91eaef6fc150c77ddd8a583e3c795737bf"}, + {file = "pyarrow-12.0.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:85e705e33eaf666bbe508a16fd5ba27ca061e177916b7a317ba5a51bee43384c"}, + {file = "pyarrow-12.0.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9120c3eb2b1f6f516a3b7a9714ed860882d9ef98c4b17edcdc91d95b7528db60"}, + {file = "pyarrow-12.0.1-cp38-cp38-win_amd64.whl", hash = "sha256:c780f4dc40460015d80fcd6a6140de80b615349ed68ef9adb653fe351778c9b3"}, + {file = "pyarrow-12.0.1-cp39-cp39-macosx_10_14_x86_64.whl", hash = "sha256:a3c63124fc26bf5f95f508f5d04e1ece8cc23a8b0af2a1e6ab2b1ec3fdc91b24"}, + {file = "pyarrow-12.0.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:b13329f79fa4472324f8d32dc1b1216616d09bd1e77cfb13104dec5463632c36"}, + {file = "pyarrow-12.0.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:bb656150d3d12ec1396f6dde542db1675a95c0cc8366d507347b0beed96e87ca"}, + {file = "pyarrow-12.0.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6251e38470da97a5b2e00de5c6a049149f7b2bd62f12fa5dbb9ac674119ba71a"}, + {file = "pyarrow-12.0.1-cp39-cp39-win_amd64.whl", hash = "sha256:3de26da901216149ce086920547dfff5cd22818c9eab67ebc41e863a5883bac7"}, + {file = "pyarrow-12.0.1.tar.gz", hash = "sha256:cce317fc96e5b71107bf1f9f184d5e54e2bd14bbf3f9a3d62819961f0af86fec"}, ] [package.dependencies] @@ -1895,7 +1925,6 @@ numpy = ">=1.16.6" name = "pycparser" version = "2.21" description = "C parser in Python" -category = "main" optional = false python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*" files = [ @@ -1905,48 +1934,47 @@ files = [ [[package]] name = "pydantic" -version = "1.10.8" +version = "1.10.11" description = "Data validation and settings management using python type hints" -category = "main" optional = false python-versions = ">=3.7" files = [ - {file = "pydantic-1.10.8-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:1243d28e9b05003a89d72e7915fdb26ffd1d39bdd39b00b7dbe4afae4b557f9d"}, - {file = "pydantic-1.10.8-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:c0ab53b609c11dfc0c060d94335993cc2b95b2150e25583bec37a49b2d6c6c3f"}, - {file = "pydantic-1.10.8-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f9613fadad06b4f3bc5db2653ce2f22e0de84a7c6c293909b48f6ed37b83c61f"}, - {file = "pydantic-1.10.8-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:df7800cb1984d8f6e249351139667a8c50a379009271ee6236138a22a0c0f319"}, - {file = "pydantic-1.10.8-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:0c6fafa0965b539d7aab0a673a046466d23b86e4b0e8019d25fd53f4df62c277"}, - {file = "pydantic-1.10.8-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:e82d4566fcd527eae8b244fa952d99f2ca3172b7e97add0b43e2d97ee77f81ab"}, - {file = "pydantic-1.10.8-cp310-cp310-win_amd64.whl", hash = "sha256:ab523c31e22943713d80d8d342d23b6f6ac4b792a1e54064a8d0cf78fd64e800"}, - {file = "pydantic-1.10.8-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:666bdf6066bf6dbc107b30d034615d2627e2121506c555f73f90b54a463d1f33"}, - {file = "pydantic-1.10.8-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:35db5301b82e8661fa9c505c800d0990bc14e9f36f98932bb1d248c0ac5cada5"}, - {file = "pydantic-1.10.8-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f90c1e29f447557e9e26afb1c4dbf8768a10cc676e3781b6a577841ade126b85"}, - {file = "pydantic-1.10.8-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:93e766b4a8226e0708ef243e843105bf124e21331694367f95f4e3b4a92bbb3f"}, - {file = "pydantic-1.10.8-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:88f195f582851e8db960b4a94c3e3ad25692c1c1539e2552f3df7a9e972ef60e"}, - {file = "pydantic-1.10.8-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:34d327c81e68a1ecb52fe9c8d50c8a9b3e90d3c8ad991bfc8f953fb477d42fb4"}, - {file = "pydantic-1.10.8-cp311-cp311-win_amd64.whl", hash = "sha256:d532bf00f381bd6bc62cabc7d1372096b75a33bc197a312b03f5838b4fb84edd"}, - {file = "pydantic-1.10.8-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:7d5b8641c24886d764a74ec541d2fc2c7fb19f6da2a4001e6d580ba4a38f7878"}, - {file = "pydantic-1.10.8-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7b1f6cb446470b7ddf86c2e57cd119a24959af2b01e552f60705910663af09a4"}, - {file = "pydantic-1.10.8-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c33b60054b2136aef8cf190cd4c52a3daa20b2263917c49adad20eaf381e823b"}, - {file = "pydantic-1.10.8-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:1952526ba40b220b912cdc43c1c32bcf4a58e3f192fa313ee665916b26befb68"}, - {file = "pydantic-1.10.8-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:bb14388ec45a7a0dc429e87def6396f9e73c8c77818c927b6a60706603d5f2ea"}, - {file = "pydantic-1.10.8-cp37-cp37m-win_amd64.whl", hash = "sha256:16f8c3e33af1e9bb16c7a91fc7d5fa9fe27298e9f299cff6cb744d89d573d62c"}, - {file = "pydantic-1.10.8-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:1ced8375969673929809d7f36ad322934c35de4af3b5e5b09ec967c21f9f7887"}, - {file = "pydantic-1.10.8-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:93e6bcfccbd831894a6a434b0aeb1947f9e70b7468f274154d03d71fabb1d7c6"}, - {file = "pydantic-1.10.8-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:191ba419b605f897ede9892f6c56fb182f40a15d309ef0142212200a10af4c18"}, - {file = "pydantic-1.10.8-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:052d8654cb65174d6f9490cc9b9a200083a82cf5c3c5d3985db765757eb3b375"}, - {file = "pydantic-1.10.8-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:ceb6a23bf1ba4b837d0cfe378329ad3f351b5897c8d4914ce95b85fba96da5a1"}, - {file = "pydantic-1.10.8-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:6f2e754d5566f050954727c77f094e01793bcb5725b663bf628fa6743a5a9108"}, - {file = "pydantic-1.10.8-cp38-cp38-win_amd64.whl", hash = "sha256:6a82d6cda82258efca32b40040228ecf43a548671cb174a1e81477195ed3ed56"}, - {file = "pydantic-1.10.8-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:3e59417ba8a17265e632af99cc5f35ec309de5980c440c255ab1ca3ae96a3e0e"}, - {file = "pydantic-1.10.8-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:84d80219c3f8d4cad44575e18404099c76851bc924ce5ab1c4c8bb5e2a2227d0"}, - {file = "pydantic-1.10.8-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2e4148e635994d57d834be1182a44bdb07dd867fa3c2d1b37002000646cc5459"}, - {file = "pydantic-1.10.8-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:12f7b0bf8553e310e530e9f3a2f5734c68699f42218bf3568ef49cd9b0e44df4"}, - {file = "pydantic-1.10.8-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:42aa0c4b5c3025483240a25b09f3c09a189481ddda2ea3a831a9d25f444e03c1"}, - {file = "pydantic-1.10.8-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:17aef11cc1b997f9d574b91909fed40761e13fac438d72b81f902226a69dac01"}, - {file = "pydantic-1.10.8-cp39-cp39-win_amd64.whl", hash = "sha256:66a703d1983c675a6e0fed8953b0971c44dba48a929a2000a493c3772eb61a5a"}, - {file = "pydantic-1.10.8-py3-none-any.whl", hash = "sha256:7456eb22ed9aaa24ff3e7b4757da20d9e5ce2a81018c1b3ebd81a0b88a18f3b2"}, - {file = "pydantic-1.10.8.tar.gz", hash = "sha256:1410275520dfa70effadf4c21811d755e7ef9bb1f1d077a21958153a92c8d9ca"}, + {file = "pydantic-1.10.11-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:ff44c5e89315b15ff1f7fdaf9853770b810936d6b01a7bcecaa227d2f8fe444f"}, + {file = "pydantic-1.10.11-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:a6c098d4ab5e2d5b3984d3cb2527e2d6099d3de85630c8934efcfdc348a9760e"}, + {file = "pydantic-1.10.11-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:16928fdc9cb273c6af00d9d5045434c39afba5f42325fb990add2c241402d151"}, + {file = "pydantic-1.10.11-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:0588788a9a85f3e5e9ebca14211a496409cb3deca5b6971ff37c556d581854e7"}, + {file = "pydantic-1.10.11-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:e9baf78b31da2dc3d3f346ef18e58ec5f12f5aaa17ac517e2ffd026a92a87588"}, + {file = "pydantic-1.10.11-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:373c0840f5c2b5b1ccadd9286782852b901055998136287828731868027a724f"}, + {file = "pydantic-1.10.11-cp310-cp310-win_amd64.whl", hash = "sha256:c3339a46bbe6013ef7bdd2844679bfe500347ac5742cd4019a88312aa58a9847"}, + {file = "pydantic-1.10.11-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:08a6c32e1c3809fbc49debb96bf833164f3438b3696abf0fbeceb417d123e6eb"}, + {file = "pydantic-1.10.11-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:a451ccab49971af043ec4e0d207cbc8cbe53dbf148ef9f19599024076fe9c25b"}, + {file = "pydantic-1.10.11-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:5b02d24f7b2b365fed586ed73582c20f353a4c50e4be9ba2c57ab96f8091ddae"}, + {file = "pydantic-1.10.11-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3f34739a89260dfa420aa3cbd069fbcc794b25bbe5c0a214f8fb29e363484b66"}, + {file = "pydantic-1.10.11-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:e297897eb4bebde985f72a46a7552a7556a3dd11e7f76acda0c1093e3dbcf216"}, + {file = "pydantic-1.10.11-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:d185819a7a059550ecb85d5134e7d40f2565f3dd94cfd870132c5f91a89cf58c"}, + {file = "pydantic-1.10.11-cp311-cp311-win_amd64.whl", hash = "sha256:4400015f15c9b464c9db2d5d951b6a780102cfa5870f2c036d37c23b56f7fc1b"}, + {file = "pydantic-1.10.11-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:2417de68290434461a266271fc57274a138510dca19982336639484c73a07af6"}, + {file = "pydantic-1.10.11-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:331c031ba1554b974c98679bd0780d89670d6fd6f53f5d70b10bdc9addee1713"}, + {file = "pydantic-1.10.11-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:8268a735a14c308923e8958363e3a3404f6834bb98c11f5ab43251a4e410170c"}, + {file = "pydantic-1.10.11-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:44e51ba599c3ef227e168424e220cd3e544288c57829520dc90ea9cb190c3248"}, + {file = "pydantic-1.10.11-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:d7781f1d13b19700b7949c5a639c764a077cbbdd4322ed505b449d3ca8edcb36"}, + {file = "pydantic-1.10.11-cp37-cp37m-win_amd64.whl", hash = "sha256:7522a7666157aa22b812ce14c827574ddccc94f361237ca6ea8bb0d5c38f1629"}, + {file = "pydantic-1.10.11-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:bc64eab9b19cd794a380179ac0e6752335e9555d214cfcb755820333c0784cb3"}, + {file = "pydantic-1.10.11-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:8dc77064471780262b6a68fe67e013298d130414d5aaf9b562c33987dbd2cf4f"}, + {file = "pydantic-1.10.11-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fe429898f2c9dd209bd0632a606bddc06f8bce081bbd03d1c775a45886e2c1cb"}, + {file = "pydantic-1.10.11-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:192c608ad002a748e4a0bed2ddbcd98f9b56df50a7c24d9a931a8c5dd053bd3d"}, + {file = "pydantic-1.10.11-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:ef55392ec4bb5721f4ded1096241e4b7151ba6d50a50a80a2526c854f42e6a2f"}, + {file = "pydantic-1.10.11-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:41e0bb6efe86281623abbeeb0be64eab740c865388ee934cd3e6a358784aca6e"}, + {file = "pydantic-1.10.11-cp38-cp38-win_amd64.whl", hash = "sha256:265a60da42f9f27e0b1014eab8acd3e53bd0bad5c5b4884e98a55f8f596b2c19"}, + {file = "pydantic-1.10.11-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:469adf96c8e2c2bbfa655fc7735a2a82f4c543d9fee97bd113a7fb509bf5e622"}, + {file = "pydantic-1.10.11-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:e6cbfbd010b14c8a905a7b10f9fe090068d1744d46f9e0c021db28daeb8b6de1"}, + {file = "pydantic-1.10.11-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:abade85268cc92dff86d6effcd917893130f0ff516f3d637f50dadc22ae93999"}, + {file = "pydantic-1.10.11-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e9738b0f2e6c70f44ee0de53f2089d6002b10c33264abee07bdb5c7f03038303"}, + {file = "pydantic-1.10.11-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:787cf23e5a0cde753f2eabac1b2e73ae3844eb873fd1f5bdbff3048d8dbb7604"}, + {file = "pydantic-1.10.11-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:174899023337b9fc685ac8adaa7b047050616136ccd30e9070627c1aaab53a13"}, + {file = "pydantic-1.10.11-cp39-cp39-win_amd64.whl", hash = "sha256:1954f8778489a04b245a1e7b8b22a9d3ea8ef49337285693cf6959e4b757535e"}, + {file = "pydantic-1.10.11-py3-none-any.whl", hash = "sha256:008c5e266c8aada206d0627a011504e14268a62091450210eda7c07fabe6963e"}, + {file = "pydantic-1.10.11.tar.gz", hash = "sha256:f66d479cf7eb331372c470614be6511eae96f1f120344c25f3f9bb59fb1b5528"}, ] [package.dependencies] @@ -1960,7 +1988,6 @@ email = ["email-validator (>=1.0.3)"] name = "pygments" version = "2.15.1" description = "Pygments is a syntax highlighting package written in Python." -category = "main" optional = false python-versions = ">=3.7" files = [ @@ -1973,14 +2000,13 @@ plugins = ["importlib-metadata"] [[package]] name = "pyjwt" -version = "2.7.0" +version = "2.8.0" description = "JSON Web Token implementation in Python" -category = "main" optional = true python-versions = ">=3.7" files = [ - {file = "PyJWT-2.7.0-py3-none-any.whl", hash = "sha256:ba2b425b15ad5ef12f200dc67dd56af4e26de2331f965c5439994dad075876e1"}, - {file = "PyJWT-2.7.0.tar.gz", hash = "sha256:bd6ca4a3c4285c1a2d4349e5a035fdf8fb94e04ccd0fcbe6ba289dae9cc3e074"}, + {file = "PyJWT-2.8.0-py3-none-any.whl", hash = "sha256:59127c392cc44c2da5bb3192169a91f429924e17aff6534d70fdc02ab3e04320"}, + {file = "PyJWT-2.8.0.tar.gz", hash = "sha256:57e28d156e3d5c10088e0c68abb90bfac3df82b40a71bd0daa20c65ccd5c23de"}, ] [package.dependencies] @@ -1994,14 +2020,13 @@ tests = ["coverage[toml] (==5.0.4)", "pytest (>=6.0.0,<7.0.0)"] [[package]] name = "pyparsing" -version = "3.0.9" +version = "3.1.0" description = "pyparsing module - Classes and methods to define and execute parsing grammars" -category = "main" optional = false python-versions = ">=3.6.8" files = [ - {file = "pyparsing-3.0.9-py3-none-any.whl", hash = "sha256:5026bae9a10eeaefb61dab2f09052b9f4307d44aee4eda64b309723d8d206bbc"}, - {file = "pyparsing-3.0.9.tar.gz", hash = "sha256:2b020ecf7d21b687f219b71ecad3631f644a47f01403fa1d1036b0c6416d70fb"}, + {file = "pyparsing-3.1.0-py3-none-any.whl", hash = "sha256:d554a96d1a7d3ddaf7183104485bc19fd80543ad6ac5bdb6426719d766fb06c1"}, + {file = "pyparsing-3.1.0.tar.gz", hash = "sha256:edb662d6fe322d6e990b1594b5feaeadf806803359e3d4d42f11e295e588f0ea"}, ] [package.extras] @@ -2011,7 +2036,6 @@ diagrams = ["jinja2", "railroad-diagrams"] name = "pyproject-hooks" version = "1.0.0" description = "Wrappers to call pyproject.toml-based build backend hooks." -category = "dev" optional = false python-versions = ">=3.7" files = [ @@ -2022,53 +2046,15 @@ files = [ [package.dependencies] tomli = {version = ">=1.1.0", markers = "python_version < \"3.11\""} -[[package]] -name = "pyrsistent" -version = "0.19.3" -description = "Persistent/Functional/Immutable data structures" -category = "main" -optional = true -python-versions = ">=3.7" -files = [ - {file = "pyrsistent-0.19.3-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:20460ac0ea439a3e79caa1dbd560344b64ed75e85d8703943e0b66c2a6150e4a"}, - {file = "pyrsistent-0.19.3-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4c18264cb84b5e68e7085a43723f9e4c1fd1d935ab240ce02c0324a8e01ccb64"}, - {file = "pyrsistent-0.19.3-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:4b774f9288dda8d425adb6544e5903f1fb6c273ab3128a355c6b972b7df39dcf"}, - {file = "pyrsistent-0.19.3-cp310-cp310-win32.whl", hash = "sha256:5a474fb80f5e0d6c9394d8db0fc19e90fa540b82ee52dba7d246a7791712f74a"}, - {file = "pyrsistent-0.19.3-cp310-cp310-win_amd64.whl", hash = "sha256:49c32f216c17148695ca0e02a5c521e28a4ee6c5089f97e34fe24163113722da"}, - {file = "pyrsistent-0.19.3-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:f0774bf48631f3a20471dd7c5989657b639fd2d285b861237ea9e82c36a415a9"}, - {file = "pyrsistent-0.19.3-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3ab2204234c0ecd8b9368dbd6a53e83c3d4f3cab10ecaf6d0e772f456c442393"}, - {file = "pyrsistent-0.19.3-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e42296a09e83028b3476f7073fcb69ffebac0e66dbbfd1bd847d61f74db30f19"}, - {file = "pyrsistent-0.19.3-cp311-cp311-win32.whl", hash = "sha256:64220c429e42a7150f4bfd280f6f4bb2850f95956bde93c6fda1b70507af6ef3"}, - {file = "pyrsistent-0.19.3-cp311-cp311-win_amd64.whl", hash = "sha256:016ad1afadf318eb7911baa24b049909f7f3bb2c5b1ed7b6a8f21db21ea3faa8"}, - {file = "pyrsistent-0.19.3-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:c4db1bd596fefd66b296a3d5d943c94f4fac5bcd13e99bffe2ba6a759d959a28"}, - {file = "pyrsistent-0.19.3-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:aeda827381f5e5d65cced3024126529ddc4289d944f75e090572c77ceb19adbf"}, - {file = "pyrsistent-0.19.3-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:42ac0b2f44607eb92ae88609eda931a4f0dfa03038c44c772e07f43e738bcac9"}, - {file = "pyrsistent-0.19.3-cp37-cp37m-win32.whl", hash = "sha256:e8f2b814a3dc6225964fa03d8582c6e0b6650d68a232df41e3cc1b66a5d2f8d1"}, - {file = "pyrsistent-0.19.3-cp37-cp37m-win_amd64.whl", hash = "sha256:c9bb60a40a0ab9aba40a59f68214eed5a29c6274c83b2cc206a359c4a89fa41b"}, - {file = "pyrsistent-0.19.3-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:a2471f3f8693101975b1ff85ffd19bb7ca7dd7c38f8a81701f67d6b4f97b87d8"}, - {file = "pyrsistent-0.19.3-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:cc5d149f31706762c1f8bda2e8c4f8fead6e80312e3692619a75301d3dbb819a"}, - {file = "pyrsistent-0.19.3-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3311cb4237a341aa52ab8448c27e3a9931e2ee09561ad150ba94e4cfd3fc888c"}, - {file = "pyrsistent-0.19.3-cp38-cp38-win32.whl", hash = "sha256:f0e7c4b2f77593871e918be000b96c8107da48444d57005b6a6bc61fb4331b2c"}, - {file = "pyrsistent-0.19.3-cp38-cp38-win_amd64.whl", hash = "sha256:c147257a92374fde8498491f53ffa8f4822cd70c0d85037e09028e478cababb7"}, - {file = "pyrsistent-0.19.3-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:b735e538f74ec31378f5a1e3886a26d2ca6351106b4dfde376a26fc32a044edc"}, - {file = "pyrsistent-0.19.3-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:99abb85579e2165bd8522f0c0138864da97847875ecbd45f3e7e2af569bfc6f2"}, - {file = "pyrsistent-0.19.3-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3a8cb235fa6d3fd7aae6a4f1429bbb1fec1577d978098da1252f0489937786f3"}, - {file = "pyrsistent-0.19.3-cp39-cp39-win32.whl", hash = "sha256:c74bed51f9b41c48366a286395c67f4e894374306b197e62810e0fdaf2364da2"}, - {file = "pyrsistent-0.19.3-cp39-cp39-win_amd64.whl", hash = "sha256:878433581fc23e906d947a6814336eee031a00e6defba224234169ae3d3d6a98"}, - {file = "pyrsistent-0.19.3-py3-none-any.whl", hash = "sha256:ccf0d6bd208f8111179f0c26fdf84ed7c3891982f2edaeae7422575f47e66b64"}, - {file = "pyrsistent-0.19.3.tar.gz", hash = "sha256:1a2994773706bbb4995c31a97bc94f1418314923bd1048c6d964837040376440"}, -] - [[package]] name = "pytest" -version = "7.3.1" +version = "7.4.0" description = "pytest: simple powerful testing with Python" -category = "dev" optional = false python-versions = ">=3.7" files = [ - {file = "pytest-7.3.1-py3-none-any.whl", hash = "sha256:3799fa815351fea3a5e96ac7e503a96fa51cc9942c3753cda7651b93c1cfa362"}, - {file = "pytest-7.3.1.tar.gz", hash = "sha256:434afafd78b1d78ed0addf160ad2b77a30d35d4bdf8af234fe621919d9ed15e3"}, + {file = "pytest-7.4.0-py3-none-any.whl", hash = "sha256:78bf16451a2eb8c7a2ea98e32dc119fd2aa758f1d5d66dbf0a59d69a3969df32"}, + {file = "pytest-7.4.0.tar.gz", hash = "sha256:b4bf8c45bd59934ed84001ad51e11b4ee40d40a1229d2c79f9c592b0a3f6bd8a"}, ] [package.dependencies] @@ -2080,18 +2066,17 @@ pluggy = ">=0.12,<2.0" tomli = {version = ">=1.0.0", markers = "python_version < \"3.11\""} [package.extras] -testing = ["argcomplete", "attrs (>=19.2.0)", "hypothesis (>=3.56)", "mock", "nose", "pygments (>=2.7.2)", "requests", "xmlschema"] +testing = ["argcomplete", "attrs (>=19.2.0)", "hypothesis (>=3.56)", "mock", "nose", "pygments (>=2.7.2)", "requests", "setuptools", "xmlschema"] [[package]] name = "pytest-checkdocs" -version = "2.9.0" +version = "2.10.0" description = "check the README when running tests" -category = "dev" optional = false -python-versions = ">=3.7" +python-versions = ">=3.8" files = [ - {file = "pytest-checkdocs-2.9.0.tar.gz", hash = "sha256:54b3d1357a289e3f6336d31d4ceb72e99afe0fa4f552990f1b1f015f1925543b"}, - {file = "pytest_checkdocs-2.9.0-py3-none-any.whl", hash = "sha256:4231e5426915a02385e386b5eb4a70490b143d97cede4c7c77691bb320b5a73f"}, + {file = "pytest-checkdocs-2.10.0.tar.gz", hash = "sha256:0c1320dc88549a95b7482fba6dae37ef5e9bfb42fd7babcc1f8f1be3ae15994f"}, + {file = "pytest_checkdocs-2.10.0-py3-none-any.whl", hash = "sha256:93ae72cf71d4a3a8d5860a5f42c31131d639257a88796e769d6a008a081b10b0"}, ] [package.dependencies] @@ -2100,14 +2085,30 @@ docutils = ">=0.15" importlib-metadata = {version = ">=4", markers = "python_version < \"3.10\""} [package.extras] -docs = ["furo", "jaraco.packaging (>=9)", "rst.linker (>=1.9)", "sphinx (>=3.5)"] -testing = ["flake8 (<5)", "pytest (>=6)", "pytest-black (>=0.3.7)", "pytest-checkdocs (>=2.4)", "pytest-cov", "pytest-enabler (>=1.3)", "pytest-flake8", "pytest-mypy (>=0.9.1)", "types-docutils"] +docs = ["furo", "jaraco.packaging (>=9.3)", "rst.linker (>=1.9)", "sphinx (>=3.5)", "sphinx-lint"] +testing = ["pytest (>=6)", "pytest-black (>=0.3.7)", "pytest-checkdocs (>=2.4)", "pytest-cov", "pytest-enabler (>=2.2)", "pytest-mypy (>=0.9.1)", "pytest-ruff", "types-docutils"] + +[[package]] +name = "pytest-mock" +version = "3.11.1" +description = "Thin-wrapper around the mock package for easier use with pytest" +optional = false +python-versions = ">=3.7" +files = [ + {file = "pytest-mock-3.11.1.tar.gz", hash = "sha256:7f6b125602ac6d743e523ae0bfa71e1a697a2f5534064528c6ff84c2f7c2fc7f"}, + {file = "pytest_mock-3.11.1-py3-none-any.whl", hash = "sha256:21c279fff83d70763b05f8874cc9cfb3fcacd6d354247a976f9529d19f9acf39"}, +] + +[package.dependencies] +pytest = ">=5.0" + +[package.extras] +dev = ["pre-commit", "pytest-asyncio", "tox"] [[package]] name = "python-dateutil" version = "2.8.2" description = "Extensions to the standard Python datetime module" -category = "main" optional = false python-versions = "!=3.0.*,!=3.1.*,!=3.2.*,>=2.7" files = [ @@ -2122,7 +2123,6 @@ six = ">=1.5" name = "python-snappy" version = "0.6.1" description = "Python library for the snappy compression library from Google" -category = "main" optional = true python-versions = "*" files = [ @@ -2180,7 +2180,6 @@ files = [ name = "pytz" version = "2023.3" description = "World timezone definitions, modern and historical" -category = "main" optional = true python-versions = "*" files = [ @@ -2192,7 +2191,6 @@ files = [ name = "pywin32" version = "306" description = "Python for Window Extensions" -category = "main" optional = true python-versions = "*" files = [ @@ -2214,99 +2212,93 @@ files = [ [[package]] name = "pyyaml" -version = "6.0" +version = "6.0.1" description = "YAML parser and emitter for Python" -category = "main" optional = false python-versions = ">=3.6" files = [ - {file = "PyYAML-6.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:d4db7c7aef085872ef65a8fd7d6d09a14ae91f691dec3e87ee5ee0539d516f53"}, - {file = "PyYAML-6.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:9df7ed3b3d2e0ecfe09e14741b857df43adb5a3ddadc919a2d94fbdf78fea53c"}, - {file = "PyYAML-6.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:77f396e6ef4c73fdc33a9157446466f1cff553d979bd00ecb64385760c6babdc"}, - {file = "PyYAML-6.0-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:a80a78046a72361de73f8f395f1f1e49f956c6be882eed58505a15f3e430962b"}, - {file = "PyYAML-6.0-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:f84fbc98b019fef2ee9a1cb3ce93e3187a6df0b2538a651bfb890254ba9f90b5"}, - {file = "PyYAML-6.0-cp310-cp310-win32.whl", hash = "sha256:2cd5df3de48857ed0544b34e2d40e9fac445930039f3cfe4bcc592a1f836d513"}, - {file = "PyYAML-6.0-cp310-cp310-win_amd64.whl", hash = "sha256:daf496c58a8c52083df09b80c860005194014c3698698d1a57cbcfa182142a3a"}, - {file = "PyYAML-6.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:d4b0ba9512519522b118090257be113b9468d804b19d63c71dbcf4a48fa32358"}, - {file = "PyYAML-6.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:81957921f441d50af23654aa6c5e5eaf9b06aba7f0a19c18a538dc7ef291c5a1"}, - {file = "PyYAML-6.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:afa17f5bc4d1b10afd4466fd3a44dc0e245382deca5b3c353d8b757f9e3ecb8d"}, - {file = "PyYAML-6.0-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:dbad0e9d368bb989f4515da330b88a057617d16b6a8245084f1b05400f24609f"}, - {file = "PyYAML-6.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:432557aa2c09802be39460360ddffd48156e30721f5e8d917f01d31694216782"}, - {file = "PyYAML-6.0-cp311-cp311-win32.whl", hash = "sha256:bfaef573a63ba8923503d27530362590ff4f576c626d86a9fed95822a8255fd7"}, - {file = "PyYAML-6.0-cp311-cp311-win_amd64.whl", hash = "sha256:01b45c0191e6d66c470b6cf1b9531a771a83c1c4208272ead47a3ae4f2f603bf"}, - {file = "PyYAML-6.0-cp36-cp36m-macosx_10_9_x86_64.whl", hash = "sha256:897b80890765f037df3403d22bab41627ca8811ae55e9a722fd0392850ec4d86"}, - {file = "PyYAML-6.0-cp36-cp36m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:50602afada6d6cbfad699b0c7bb50d5ccffa7e46a3d738092afddc1f9758427f"}, - {file = "PyYAML-6.0-cp36-cp36m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:48c346915c114f5fdb3ead70312bd042a953a8ce5c7106d5bfb1a5254e47da92"}, - {file = "PyYAML-6.0-cp36-cp36m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:98c4d36e99714e55cfbaaee6dd5badbc9a1ec339ebfc3b1f52e293aee6bb71a4"}, - {file = "PyYAML-6.0-cp36-cp36m-win32.whl", hash = "sha256:0283c35a6a9fbf047493e3a0ce8d79ef5030852c51e9d911a27badfde0605293"}, - {file = "PyYAML-6.0-cp36-cp36m-win_amd64.whl", hash = "sha256:07751360502caac1c067a8132d150cf3d61339af5691fe9e87803040dbc5db57"}, - {file = "PyYAML-6.0-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:819b3830a1543db06c4d4b865e70ded25be52a2e0631ccd2f6a47a2822f2fd7c"}, - {file = "PyYAML-6.0-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:473f9edb243cb1935ab5a084eb238d842fb8f404ed2193a915d1784b5a6b5fc0"}, - {file = "PyYAML-6.0-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:0ce82d761c532fe4ec3f87fc45688bdd3a4c1dc5e0b4a19814b9009a29baefd4"}, - {file = "PyYAML-6.0-cp37-cp37m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:231710d57adfd809ef5d34183b8ed1eeae3f76459c18fb4a0b373ad56bedcdd9"}, - {file = "PyYAML-6.0-cp37-cp37m-win32.whl", hash = "sha256:c5687b8d43cf58545ade1fe3e055f70eac7a5a1a0bf42824308d868289a95737"}, - {file = "PyYAML-6.0-cp37-cp37m-win_amd64.whl", hash = "sha256:d15a181d1ecd0d4270dc32edb46f7cb7733c7c508857278d3d378d14d606db2d"}, - {file = "PyYAML-6.0-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:0b4624f379dab24d3725ffde76559cff63d9ec94e1736b556dacdfebe5ab6d4b"}, - {file = "PyYAML-6.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:213c60cd50106436cc818accf5baa1aba61c0189ff610f64f4a3e8c6726218ba"}, - {file = "PyYAML-6.0-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:9fa600030013c4de8165339db93d182b9431076eb98eb40ee068700c9c813e34"}, - {file = "PyYAML-6.0-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:277a0ef2981ca40581a47093e9e2d13b3f1fbbeffae064c1d21bfceba2030287"}, - {file = "PyYAML-6.0-cp38-cp38-win32.whl", hash = "sha256:d4eccecf9adf6fbcc6861a38015c2a64f38b9d94838ac1810a9023a0609e1b78"}, - {file = "PyYAML-6.0-cp38-cp38-win_amd64.whl", hash = "sha256:1e4747bc279b4f613a09eb64bba2ba602d8a6664c6ce6396a4d0cd413a50ce07"}, - {file = "PyYAML-6.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:055d937d65826939cb044fc8c9b08889e8c743fdc6a32b33e2390f66013e449b"}, - {file = "PyYAML-6.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:e61ceaab6f49fb8bdfaa0f92c4b57bcfbea54c09277b1b4f7ac376bfb7a7c174"}, - {file = "PyYAML-6.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d67d839ede4ed1b28a4e8909735fc992a923cdb84e618544973d7dfc71540803"}, - {file = "PyYAML-6.0-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:cba8c411ef271aa037d7357a2bc8f9ee8b58b9965831d9e51baf703280dc73d3"}, - {file = "PyYAML-6.0-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:40527857252b61eacd1d9af500c3337ba8deb8fc298940291486c465c8b46ec0"}, - {file = "PyYAML-6.0-cp39-cp39-win32.whl", hash = "sha256:b5b9eccad747aabaaffbc6064800670f0c297e52c12754eb1d976c57e4f74dcb"}, - {file = "PyYAML-6.0-cp39-cp39-win_amd64.whl", hash = "sha256:b3d267842bf12586ba6c734f89d1f5b871df0273157918b0ccefa29deb05c21c"}, - {file = "PyYAML-6.0.tar.gz", hash = "sha256:68fb519c14306fec9720a2a5b45bc9f0c8d1b9c72adf45c37baedfcd949c35a2"}, + {file = "PyYAML-6.0.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:d858aa552c999bc8a8d57426ed01e40bef403cd8ccdd0fc5f6f04a00414cac2a"}, + {file = "PyYAML-6.0.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:fd66fc5d0da6d9815ba2cebeb4205f95818ff4b79c3ebe268e75d961704af52f"}, + {file = "PyYAML-6.0.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:69b023b2b4daa7548bcfbd4aa3da05b3a74b772db9e23b982788168117739938"}, + {file = "PyYAML-6.0.1-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:81e0b275a9ecc9c0c0c07b4b90ba548307583c125f54d5b6946cfee6360c733d"}, + {file = "PyYAML-6.0.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ba336e390cd8e4d1739f42dfe9bb83a3cc2e80f567d8805e11b46f4a943f5515"}, + {file = "PyYAML-6.0.1-cp310-cp310-win32.whl", hash = "sha256:bd4af7373a854424dabd882decdc5579653d7868b8fb26dc7d0e99f823aa5924"}, + {file = "PyYAML-6.0.1-cp310-cp310-win_amd64.whl", hash = "sha256:fd1592b3fdf65fff2ad0004b5e363300ef59ced41c2e6b3a99d4089fa8c5435d"}, + {file = "PyYAML-6.0.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:6965a7bc3cf88e5a1c3bd2e0b5c22f8d677dc88a455344035f03399034eb3007"}, + {file = "PyYAML-6.0.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:f003ed9ad21d6a4713f0a9b5a7a0a79e08dd0f221aff4525a2be4c346ee60aab"}, + {file = "PyYAML-6.0.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:42f8152b8dbc4fe7d96729ec2b99c7097d656dc1213a3229ca5383f973a5ed6d"}, + {file = "PyYAML-6.0.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:062582fca9fabdd2c8b54a3ef1c978d786e0f6b3a1510e0ac93ef59e0ddae2bc"}, + {file = "PyYAML-6.0.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d2b04aac4d386b172d5b9692e2d2da8de7bfb6c387fa4f801fbf6fb2e6ba4673"}, + {file = "PyYAML-6.0.1-cp311-cp311-win32.whl", hash = "sha256:1635fd110e8d85d55237ab316b5b011de701ea0f29d07611174a1b42f1444741"}, + {file = "PyYAML-6.0.1-cp311-cp311-win_amd64.whl", hash = "sha256:bf07ee2fef7014951eeb99f56f39c9bb4af143d8aa3c21b1677805985307da34"}, + {file = "PyYAML-6.0.1-cp36-cp36m-macosx_10_9_x86_64.whl", hash = "sha256:50550eb667afee136e9a77d6dc71ae76a44df8b3e51e41b77f6de2932bfe0f47"}, + {file = "PyYAML-6.0.1-cp36-cp36m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1fe35611261b29bd1de0070f0b2f47cb6ff71fa6595c077e42bd0c419fa27b98"}, + {file = "PyYAML-6.0.1-cp36-cp36m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:704219a11b772aea0d8ecd7058d0082713c3562b4e271b849ad7dc4a5c90c13c"}, + {file = "PyYAML-6.0.1-cp36-cp36m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:afd7e57eddb1a54f0f1a974bc4391af8bcce0b444685d936840f125cf046d5bd"}, + {file = "PyYAML-6.0.1-cp36-cp36m-win32.whl", hash = "sha256:fca0e3a251908a499833aa292323f32437106001d436eca0e6e7833256674585"}, + {file = "PyYAML-6.0.1-cp36-cp36m-win_amd64.whl", hash = "sha256:f22ac1c3cac4dbc50079e965eba2c1058622631e526bd9afd45fedd49ba781fa"}, + {file = "PyYAML-6.0.1-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:b1275ad35a5d18c62a7220633c913e1b42d44b46ee12554e5fd39c70a243d6a3"}, + {file = "PyYAML-6.0.1-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:18aeb1bf9a78867dc38b259769503436b7c72f7a1f1f4c93ff9a17de54319b27"}, + {file = "PyYAML-6.0.1-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:596106435fa6ad000c2991a98fa58eeb8656ef2325d7e158344fb33864ed87e3"}, + {file = "PyYAML-6.0.1-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:baa90d3f661d43131ca170712d903e6295d1f7a0f595074f151c0aed377c9b9c"}, + {file = "PyYAML-6.0.1-cp37-cp37m-win32.whl", hash = "sha256:9046c58c4395dff28dd494285c82ba00b546adfc7ef001486fbf0324bc174fba"}, + {file = "PyYAML-6.0.1-cp37-cp37m-win_amd64.whl", hash = "sha256:4fb147e7a67ef577a588a0e2c17b6db51dda102c71de36f8549b6816a96e1867"}, + {file = "PyYAML-6.0.1-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:1d4c7e777c441b20e32f52bd377e0c409713e8bb1386e1099c2415f26e479595"}, + {file = "PyYAML-6.0.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a0cd17c15d3bb3fa06978b4e8958dcdc6e0174ccea823003a106c7d4d7899ac5"}, + {file = "PyYAML-6.0.1-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:28c119d996beec18c05208a8bd78cbe4007878c6dd15091efb73a30e90539696"}, + {file = "PyYAML-6.0.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7e07cbde391ba96ab58e532ff4803f79c4129397514e1413a7dc761ccd755735"}, + {file = "PyYAML-6.0.1-cp38-cp38-win32.whl", hash = "sha256:184c5108a2aca3c5b3d3bf9395d50893a7ab82a38004c8f61c258d4428e80206"}, + {file = "PyYAML-6.0.1-cp38-cp38-win_amd64.whl", hash = "sha256:1e2722cc9fbb45d9b87631ac70924c11d3a401b2d7f410cc0e3bbf249f2dca62"}, + {file = "PyYAML-6.0.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:9eb6caa9a297fc2c2fb8862bc5370d0303ddba53ba97e71f08023b6cd73d16a8"}, + {file = "PyYAML-6.0.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:c8098ddcc2a85b61647b2590f825f3db38891662cfc2fc776415143f599bb859"}, + {file = "PyYAML-6.0.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5773183b6446b2c99bb77e77595dd486303b4faab2b086e7b17bc6bef28865f6"}, + {file = "PyYAML-6.0.1-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:b786eecbdf8499b9ca1d697215862083bd6d2a99965554781d0d8d1ad31e13a0"}, + {file = "PyYAML-6.0.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bc1bf2925a1ecd43da378f4db9e4f799775d6367bdb94671027b73b393a7c42c"}, + {file = "PyYAML-6.0.1-cp39-cp39-win32.whl", hash = "sha256:faca3bdcf85b2fc05d06ff3fbc1f83e1391b3e724afa3feba7d13eeab355484c"}, + {file = "PyYAML-6.0.1-cp39-cp39-win_amd64.whl", hash = "sha256:510c9deebc5c0225e8c96813043e62b680ba2f9c50a08d3724c7f28a747d1486"}, + {file = "PyYAML-6.0.1.tar.gz", hash = "sha256:bfdf460b1736c775f2ba9f6a92bca30bc2095067b8a9d77876d1fad6cc3b4a43"}, ] [[package]] name = "ray" -version = "2.4.0" +version = "2.6.1" description = "Ray provides a simple, universal API for building distributed applications." -category = "main" optional = true python-versions = "*" files = [ - {file = "ray-2.4.0-cp310-cp310-macosx_10_15_universal2.whl", hash = "sha256:4250581dfc4cd2e502c1f1876a8ef3d16e8695b565b98aacf2ef2cf3b77ae0b4"}, - {file = "ray-2.4.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:26d5187b7f82ca4100e3e7c6115b76728822db34c31a596e70a2954a83eed5bc"}, - {file = "ray-2.4.0-cp310-cp310-manylinux2014_aarch64.whl", hash = "sha256:36cb9a71399972022d27f228f7f80b482e8db3528b1b74b5ace0c58f29964989"}, - {file = "ray-2.4.0-cp310-cp310-manylinux2014_x86_64.whl", hash = "sha256:5ed5a29795b122e9e2b832d5224ab9b1cc235beab700d2a413b23c63b3d3c80c"}, - {file = "ray-2.4.0-cp310-cp310-win_amd64.whl", hash = "sha256:e9942c516e55e12e9bab7ea8f6eae9c875bd044fd448e1fe9e8153df06a7c24f"}, - {file = "ray-2.4.0-cp311-cp311-manylinux2014_aarch64.whl", hash = "sha256:ee7463c235e0dbb6e0c0c54dd8ae25973222d90a5ac6b5c4a820e647daef1560"}, - {file = "ray-2.4.0-cp311-cp311-manylinux2014_x86_64.whl", hash = "sha256:d316861298f6e996f4841e4160ed38dc289f81cf0ffe9874dc14ef7e4e5a9190"}, - {file = "ray-2.4.0-cp36-cp36m-macosx_10_15_x86_64.whl", hash = "sha256:c31e86d3832b967818da18d69beffc4e85114d8eed31302251b57a8e43ac8a4e"}, - {file = "ray-2.4.0-cp36-cp36m-manylinux2014_aarch64.whl", hash = "sha256:fb22b3a7f64777b830b1dc0d81b1815ea138e7a01e569dfbc3149ec8d440589c"}, - {file = "ray-2.4.0-cp36-cp36m-manylinux2014_x86_64.whl", hash = "sha256:e2f201fb3cfad35555f241d4e1f1442bbfe15a6442ea8bb5b6826abc14109cd1"}, - {file = "ray-2.4.0-cp37-cp37m-macosx_10_15_x86_64.whl", hash = "sha256:6d63cb511731aa92cd01620891eeeec74884c53f0c41c576bb0f0b4386b35698"}, - {file = "ray-2.4.0-cp37-cp37m-manylinux2014_aarch64.whl", hash = "sha256:511e39bc99ae2835b711be5281980cb442b4f1d798cd695579098ce3724b2bac"}, - {file = "ray-2.4.0-cp37-cp37m-manylinux2014_x86_64.whl", hash = "sha256:fa7bf30f6e187274066801323c2ca6c46953644df95caf6344b471e2270883ae"}, - {file = "ray-2.4.0-cp37-cp37m-win_amd64.whl", hash = "sha256:2c34c871c780b7ae1d153a6f5e052a33c5e6ffe4aa2a90d4129a31ad6e76d243"}, - {file = "ray-2.4.0-cp38-cp38-macosx_10_15_x86_64.whl", hash = "sha256:afde23bc5552597db32c2ce4b47a6c867d3f8ddd4ea4c2479658d0c8e8a802a4"}, - {file = "ray-2.4.0-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:425932492f7b1b0ecbb92cf45305f712f2fdf590b4da1461f11adc67c5cf0f82"}, - {file = "ray-2.4.0-cp38-cp38-manylinux2014_aarch64.whl", hash = "sha256:cb39b9c54c37ed9780b75ef012dfbe44008d168d57dfee3fd8f37bd3ea84eb4f"}, - {file = "ray-2.4.0-cp38-cp38-manylinux2014_x86_64.whl", hash = "sha256:96f4f8dfe4acc88c19734034a27718519c79fb1df274d9ae02f9405520564ba4"}, - {file = "ray-2.4.0-cp38-cp38-win_amd64.whl", hash = "sha256:d09606e3cecd5352f769cde31c5e1148f5bdca45ba2142fbb4fb2f18f157e0cd"}, - {file = "ray-2.4.0-cp39-cp39-macosx_10_15_x86_64.whl", hash = "sha256:b19f7dca7d570713691243e834f8948634b48357e114d6e99813efd79e1c3be4"}, - {file = "ray-2.4.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:dce50e34e0cfff143b1af196abd3a630132512eec0d822e3c6d6a93d3873dbaf"}, - {file = "ray-2.4.0-cp39-cp39-manylinux2014_aarch64.whl", hash = "sha256:1f68ff29f7a0eee90556b78aff198819af3158322b3b38f7f98e31f416aa0d31"}, - {file = "ray-2.4.0-cp39-cp39-manylinux2014_x86_64.whl", hash = "sha256:4889b457363a3cfa52088b3572b864ebb391806371bc59b2bb047e44f999bb32"}, - {file = "ray-2.4.0-cp39-cp39-win_amd64.whl", hash = "sha256:a869186ceeabc85e20e145ad5df9813bd93d32c9a2c5d5856fb97ef9a2bc7c85"}, + {file = "ray-2.6.1-cp310-cp310-macosx_10_15_x86_64.whl", hash = "sha256:aede30300f153637cc17d2c0c00bb471ffeafb206b4bbdaa0bd3bca5a31dbfec"}, + {file = "ray-2.6.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:59de7031aad513c80d4b88c27254b7fafa03b8bca8508864c10d793ef8e8f4c3"}, + {file = "ray-2.6.1-cp310-cp310-manylinux2014_aarch64.whl", hash = "sha256:4018fbea1e4c41bf969bf8cb03f802f5efd9ae45c850b4285e7481815e63243b"}, + {file = "ray-2.6.1-cp310-cp310-manylinux2014_x86_64.whl", hash = "sha256:c9b5aabf5f41fe05028e4f3a271dc89ca7cd9c210f48a4ed815b852210ebb5a8"}, + {file = "ray-2.6.1-cp310-cp310-win_amd64.whl", hash = "sha256:d9e4fb536e34a423bed6d3f362657b837beb1f027cf60e3dcea5f7cd2d1cfc68"}, + {file = "ray-2.6.1-cp311-cp311-macosx_10_15_x86_64.whl", hash = "sha256:188e3d9d2ecc5f7fb7c93f85332c5a45343ee187c2398735ad66a86a14143d11"}, + {file = "ray-2.6.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:283f75129ea93a32c1d5878ad735375d18d1bb53211adb0cd65c5b88917d438c"}, + {file = "ray-2.6.1-cp311-cp311-manylinux2014_aarch64.whl", hash = "sha256:6877abfe0fdda1983d3c70821cda1201a06c713546e9ed612bd0b51c6fa6a5d6"}, + {file = "ray-2.6.1-cp311-cp311-manylinux2014_x86_64.whl", hash = "sha256:7b0286cd05d9107a2d978c716a7447c09ffd382971e5b2b388602d56f6b1c662"}, + {file = "ray-2.6.1-cp311-cp311-win_amd64.whl", hash = "sha256:548ab80e8563fee93d71a6ed76743bdbebb1f6d6227acbd78f96419a1455a55f"}, + {file = "ray-2.6.1-cp37-cp37m-manylinux2014_aarch64.whl", hash = "sha256:3cde0a94569b98caf6f5ecc813d1df07e910f84937d609a3a9ebc96d456444a6"}, + {file = "ray-2.6.1-cp37-cp37m-manylinux2014_x86_64.whl", hash = "sha256:f8da192448a53c37a19316bc8db66dbe7ee6a7f5f682adb375886b7889beb1e8"}, + {file = "ray-2.6.1-cp37-cp37m-win_amd64.whl", hash = "sha256:13249bcdec92aa6be6f912f9b5218e23a5128c78c2b8a239f3eb06bd7852440a"}, + {file = "ray-2.6.1-cp38-cp38-macosx_10_15_x86_64.whl", hash = "sha256:c78df9a2d2ee4c52284f3ebdb8688b50e2e38ff747fdb77444e88a05deb43dd2"}, + {file = "ray-2.6.1-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:1cc027e31bfab5e902e7eddf3806654aafb88dd3ba73f49c8d2bb940b367216b"}, + {file = "ray-2.6.1-cp38-cp38-manylinux2014_aarch64.whl", hash = "sha256:5b5765a3e5f43493a1d21b549d115546f5f2f94b2fc460faaba9eb9d9f38f23a"}, + {file = "ray-2.6.1-cp38-cp38-manylinux2014_x86_64.whl", hash = "sha256:043b6ca8ae530ff54d2b30e192880fbecc6a7713100c48531e41aaec1c389c0a"}, + {file = "ray-2.6.1-cp38-cp38-win_amd64.whl", hash = "sha256:6052a9f5959fcb3c160c71ecd7bb5f131791bce144ab3ead31b353f4387c09a0"}, + {file = "ray-2.6.1-cp39-cp39-macosx_10_15_x86_64.whl", hash = "sha256:3952027e6d100449f8efcb65e2056070ac87c45d7be6e1d2d6072994af4a8270"}, + {file = "ray-2.6.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:1c4231f0964fd48926a87ce092dbc2d73b297451499d6ff79442cb47883a35c4"}, + {file = "ray-2.6.1-cp39-cp39-manylinux2014_aarch64.whl", hash = "sha256:783b1624070516d1f782c2230cbbbbb821932f78006c7362ea9d95aa0696b8d7"}, + {file = "ray-2.6.1-cp39-cp39-manylinux2014_x86_64.whl", hash = "sha256:7708cedbeed8e37e468740b75aa941b2a3c80d2cb8791081e0b0ea159617a912"}, + {file = "ray-2.6.1-cp39-cp39-win_amd64.whl", hash = "sha256:23daadba08f28ecf18b419ede64e8c864ac846e961ec3c0516dc49e8da687e6a"}, ] [package.dependencies] aiosignal = "*" -attrs = "*" click = ">=7.0" filelock = "*" frozenlist = "*" grpcio = [ - {version = ">=1.32.0,<=1.51.3", markers = "python_version < \"3.10\" and sys_platform != \"darwin\""}, - {version = ">=1.32.0,<=1.49.1", markers = "python_version < \"3.10\" and sys_platform == \"darwin\""}, - {version = ">=1.42.0,<=1.51.3", markers = "python_version >= \"3.10\" and sys_platform != \"darwin\""}, - {version = ">=1.42.0,<=1.49.1", markers = "python_version >= \"3.10\" and sys_platform == \"darwin\""}, + {version = ">=1.32.0", markers = "python_version < \"3.10\""}, + {version = ">=1.42.0", markers = "python_version >= \"3.10\""}, ] jsonschema = "*" msgpack = ">=1.0.0,<2.0.0" @@ -2314,30 +2306,43 @@ numpy = [ {version = ">=1.16", markers = "python_version < \"3.9\""}, {version = ">=1.19.3", markers = "python_version >= \"3.9\""}, ] -packaging = {version = "*", markers = "python_version >= \"3.10\""} +packaging = "*" protobuf = ">=3.15.3,<3.19.5 || >3.19.5" pyyaml = "*" requests = "*" -virtualenv = ">=20.0.24,<20.21.1" [package.extras] -air = ["aiohttp (>=3.7)", "aiohttp-cors", "aiorwlock", "colorful", "fastapi", "fsspec", "gpustat (>=1.0.0)", "numpy (>=1.20)", "opencensus", "pandas", "pandas (>=1.3)", "prometheus-client (>=0.7.1)", "py-spy (>=0.2.0)", "pyarrow (>=6.0.1)", "pydantic", "requests", "smart-open", "starlette", "tabulate", "tensorboardX (>=1.9)", "uvicorn"] -all = ["aiohttp (>=3.7)", "aiohttp-cors", "aiorwlock", "colorful", "dm-tree", "fastapi", "fsspec", "gpustat (>=1.0.0)", "gymnasium (==0.26.3)", "kubernetes", "lz4", "numpy (>=1.20)", "opencensus", "opentelemetry-api", "opentelemetry-exporter-otlp", "opentelemetry-sdk", "pandas", "pandas (>=1.3)", "prometheus-client (>=0.7.1)", "py-spy (>=0.2.0)", "pyarrow (>=6.0.1)", "pydantic", "pyyaml", "ray-cpp (==2.4.0)", "requests", "rich", "scikit-image", "scipy", "smart-open", "starlette", "tabulate", "tensorboardX (>=1.9)", "typer", "urllib3", "uvicorn"] -cpp = ["ray-cpp (==2.4.0)"] +air = ["aiohttp (>=3.7)", "aiohttp-cors", "aiorwlock", "colorful", "fastapi", "fsspec", "gpustat (>=1.0.0)", "numpy (>=1.20)", "opencensus", "pandas", "pandas (>=1.3)", "prometheus-client (>=0.7.1)", "py-spy (>=0.2.0)", "pyarrow (>=6.0.1)", "pydantic (<2)", "requests", "smart-open", "starlette", "tensorboardX (>=1.9)", "uvicorn", "virtualenv (>=20.0.24,<20.21.1)"] +all = ["aiohttp (>=3.7)", "aiohttp-cors", "aiorwlock", "colorful", "dm-tree", "fastapi", "fsspec", "gpustat (>=1.0.0)", "grpcio (!=1.56.0)", "gymnasium (==0.26.3)", "lz4", "numpy (>=1.20)", "opencensus", "opentelemetry-api", "opentelemetry-exporter-otlp", "opentelemetry-sdk", "pandas", "pandas (>=1.3)", "prometheus-client (>=0.7.1)", "py-spy (>=0.2.0)", "pyarrow (>=6.0.1)", "pydantic (<2)", "pyyaml", "ray-cpp (==2.6.1)", "requests", "rich", "scikit-image", "scipy", "smart-open", "starlette", "tensorboardX (>=1.9)", "typer", "uvicorn", "virtualenv (>=20.0.24,<20.21.1)"] +client = ["grpcio (!=1.56.0)"] +cpp = ["ray-cpp (==2.6.1)"] data = ["fsspec", "numpy (>=1.20)", "pandas (>=1.3)", "pyarrow (>=6.0.1)"] -default = ["aiohttp (>=3.7)", "aiohttp-cors", "colorful", "gpustat (>=1.0.0)", "opencensus", "prometheus-client (>=0.7.1)", "py-spy (>=0.2.0)", "pydantic", "requests", "smart-open"] -k8s = ["kubernetes", "urllib3"] +default = ["aiohttp (>=3.7)", "aiohttp-cors", "colorful", "gpustat (>=1.0.0)", "opencensus", "prometheus-client (>=0.7.1)", "py-spy (>=0.2.0)", "pydantic (<2)", "requests", "smart-open", "virtualenv (>=20.0.24,<20.21.1)"] observability = ["opentelemetry-api", "opentelemetry-exporter-otlp", "opentelemetry-sdk"] -rllib = ["dm-tree", "gymnasium (==0.26.3)", "lz4", "pandas", "pyyaml", "requests", "rich", "scikit-image", "scipy", "tabulate", "tensorboardX (>=1.9)", "typer"] -serve = ["aiohttp (>=3.7)", "aiohttp-cors", "aiorwlock", "colorful", "fastapi", "gpustat (>=1.0.0)", "opencensus", "prometheus-client (>=0.7.1)", "py-spy (>=0.2.0)", "pydantic", "requests", "smart-open", "starlette", "uvicorn"] -train = ["pandas", "requests", "tabulate", "tensorboardX (>=1.9)"] -tune = ["pandas", "requests", "tabulate", "tensorboardX (>=1.9)"] +rllib = ["dm-tree", "gymnasium (==0.26.3)", "lz4", "pandas", "pyarrow (>=6.0.1)", "pyyaml", "requests", "rich", "scikit-image", "scipy", "tensorboardX (>=1.9)", "typer"] +serve = ["aiohttp (>=3.7)", "aiohttp-cors", "aiorwlock", "colorful", "fastapi", "gpustat (>=1.0.0)", "opencensus", "prometheus-client (>=0.7.1)", "py-spy (>=0.2.0)", "pydantic (<2)", "requests", "smart-open", "starlette", "uvicorn", "virtualenv (>=20.0.24,<20.21.1)"] +train = ["pandas", "pyarrow (>=6.0.1)", "requests", "tensorboardX (>=1.9)"] +tune = ["pandas", "pyarrow (>=6.0.1)", "requests", "tensorboardX (>=1.9)"] + +[[package]] +name = "referencing" +version = "0.30.0" +description = "JSON Referencing + Python" +optional = true +python-versions = ">=3.8" +files = [ + {file = "referencing-0.30.0-py3-none-any.whl", hash = "sha256:c257b08a399b6c2f5a3510a50d28ab5dbc7bbde049bcaf954d43c446f83ab548"}, + {file = "referencing-0.30.0.tar.gz", hash = "sha256:47237742e990457f7512c7d27486394a9aadaf876cbfaa4be65b27b4f4d47c6b"}, +] + +[package.dependencies] +attrs = ">=22.2.0" +rpds-py = ">=0.7.0" [[package]] name = "requests" version = "2.31.0" description = "Python HTTP for Humans." -category = "main" optional = false python-versions = ">=3.7" files = [ @@ -2357,14 +2362,13 @@ use-chardet-on-py3 = ["chardet (>=3.0.2,<6)"] [[package]] name = "requests-mock" -version = "1.10.0" +version = "1.11.0" description = "Mock out responses from the requests package" -category = "dev" optional = false python-versions = "*" files = [ - {file = "requests-mock-1.10.0.tar.gz", hash = "sha256:59c9c32419a9fb1ae83ec242d98e889c45bd7d7a65d48375cc243ec08441658b"}, - {file = "requests_mock-1.10.0-py2.py3-none-any.whl", hash = "sha256:2fdbb637ad17ee15c06f33d31169e71bf9fe2bdb7bc9da26185be0dd8d842699"}, + {file = "requests-mock-1.11.0.tar.gz", hash = "sha256:ef10b572b489a5f28e09b708697208c4a3b2b89ef80a9f01584340ea357ec3c4"}, + {file = "requests_mock-1.11.0-py2.py3-none-any.whl", hash = "sha256:f7fae383f228633f6bececebdab236c478ace2284d6292c6e7e2867b9ab74d15"}, ] [package.dependencies] @@ -2373,13 +2377,12 @@ six = "*" [package.extras] fixture = ["fixtures"] -test = ["fixtures", "mock", "purl", "pytest", "requests-futures", "sphinx", "testrepository (>=0.0.18)", "testtools"] +test = ["fixtures", "mock", "purl", "pytest", "requests-futures", "sphinx", "testtools"] [[package]] name = "responses" version = "0.23.1" description = "A utility library for mocking out the `requests` Python library." -category = "dev" optional = false python-versions = ">=3.7" files = [ @@ -2398,40 +2401,144 @@ tests = ["coverage (>=6.0.0)", "flake8", "mypy", "pytest (>=7.0.0)", "pytest-asy [[package]] name = "rich" -version = "13.4.1" +version = "13.5.0" description = "Render rich text, tables, progress bars, syntax highlighting, markdown and more to the terminal" -category = "main" optional = false python-versions = ">=3.7.0" files = [ - {file = "rich-13.4.1-py3-none-any.whl", hash = "sha256:d204aadb50b936bf6b1a695385429d192bc1fdaf3e8b907e8e26f4c4e4b5bf75"}, - {file = "rich-13.4.1.tar.gz", hash = "sha256:76f6b65ea7e5c5d924ba80e322231d7cb5b5981aa60bfc1e694f1bc097fe6fe1"}, + {file = "rich-13.5.0-py3-none-any.whl", hash = "sha256:996670a7618ccce27c55ba6fc0142e6e343773e11d34c96566a17b71b0e6f179"}, + {file = "rich-13.5.0.tar.gz", hash = "sha256:62c81e88dc078d2372858660e3d5566746870133e51321f852ccc20af5c7e7b2"}, ] [package.dependencies] -markdown-it-py = ">=2.2.0,<3.0.0" +markdown-it-py = ">=2.2.0" pygments = ">=2.13.0,<3.0.0" typing-extensions = {version = ">=4.0.0,<5.0", markers = "python_version < \"3.9\""} [package.extras] jupyter = ["ipywidgets (>=7.5.1,<9)"] +[[package]] +name = "rpds-py" +version = "0.9.2" +description = "Python bindings to Rust's persistent data structures (rpds)" +optional = true +python-versions = ">=3.8" +files = [ + {file = "rpds_py-0.9.2-cp310-cp310-macosx_10_7_x86_64.whl", hash = "sha256:ab6919a09c055c9b092798ce18c6c4adf49d24d4d9e43a92b257e3f2548231e7"}, + {file = "rpds_py-0.9.2-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:d55777a80f78dd09410bd84ff8c95ee05519f41113b2df90a69622f5540c4f8b"}, + {file = "rpds_py-0.9.2-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a216b26e5af0a8e265d4efd65d3bcec5fba6b26909014effe20cd302fd1138fa"}, + {file = "rpds_py-0.9.2-cp310-cp310-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:29cd8bfb2d716366a035913ced99188a79b623a3512292963d84d3e06e63b496"}, + {file = "rpds_py-0.9.2-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:44659b1f326214950a8204a248ca6199535e73a694be8d3e0e869f820767f12f"}, + {file = "rpds_py-0.9.2-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:745f5a43fdd7d6d25a53ab1a99979e7f8ea419dfefebcab0a5a1e9095490ee5e"}, + {file = "rpds_py-0.9.2-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a987578ac5214f18b99d1f2a3851cba5b09f4a689818a106c23dbad0dfeb760f"}, + {file = "rpds_py-0.9.2-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:bf4151acb541b6e895354f6ff9ac06995ad9e4175cbc6d30aaed08856558201f"}, + {file = "rpds_py-0.9.2-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:03421628f0dc10a4119d714a17f646e2837126a25ac7a256bdf7c3943400f67f"}, + {file = "rpds_py-0.9.2-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:13b602dc3e8dff3063734f02dcf05111e887f301fdda74151a93dbbc249930fe"}, + {file = "rpds_py-0.9.2-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:fae5cb554b604b3f9e2c608241b5d8d303e410d7dfb6d397c335f983495ce7f6"}, + {file = "rpds_py-0.9.2-cp310-none-win32.whl", hash = "sha256:47c5f58a8e0c2c920cc7783113df2fc4ff12bf3a411d985012f145e9242a2764"}, + {file = "rpds_py-0.9.2-cp310-none-win_amd64.whl", hash = "sha256:4ea6b73c22d8182dff91155af018b11aac9ff7eca085750455c5990cb1cfae6e"}, + {file = "rpds_py-0.9.2-cp311-cp311-macosx_10_7_x86_64.whl", hash = "sha256:e564d2238512c5ef5e9d79338ab77f1cbbda6c2d541ad41b2af445fb200385e3"}, + {file = "rpds_py-0.9.2-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:f411330a6376fb50e5b7a3e66894e4a39e60ca2e17dce258d53768fea06a37bd"}, + {file = "rpds_py-0.9.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0e7521f5af0233e89939ad626b15278c71b69dc1dfccaa7b97bd4cdf96536bb7"}, + {file = "rpds_py-0.9.2-cp311-cp311-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:8d3335c03100a073883857e91db9f2e0ef8a1cf42dc0369cbb9151c149dbbc1b"}, + {file = "rpds_py-0.9.2-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:d25b1c1096ef0447355f7293fbe9ad740f7c47ae032c2884113f8e87660d8f6e"}, + {file = "rpds_py-0.9.2-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:6a5d3fbd02efd9cf6a8ffc2f17b53a33542f6b154e88dd7b42ef4a4c0700fdad"}, + {file = "rpds_py-0.9.2-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c5934e2833afeaf36bd1eadb57256239785f5af0220ed8d21c2896ec4d3a765f"}, + {file = "rpds_py-0.9.2-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:095b460e117685867d45548fbd8598a8d9999227e9061ee7f012d9d264e6048d"}, + {file = "rpds_py-0.9.2-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:91378d9f4151adc223d584489591dbb79f78814c0734a7c3bfa9c9e09978121c"}, + {file = "rpds_py-0.9.2-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:24a81c177379300220e907e9b864107614b144f6c2a15ed5c3450e19cf536fae"}, + {file = "rpds_py-0.9.2-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:de0b6eceb46141984671802d412568d22c6bacc9b230174f9e55fc72ef4f57de"}, + {file = "rpds_py-0.9.2-cp311-none-win32.whl", hash = "sha256:700375326ed641f3d9d32060a91513ad668bcb7e2cffb18415c399acb25de2ab"}, + {file = "rpds_py-0.9.2-cp311-none-win_amd64.whl", hash = "sha256:0766babfcf941db8607bdaf82569ec38107dbb03c7f0b72604a0b346b6eb3298"}, + {file = "rpds_py-0.9.2-cp312-cp312-macosx_10_7_x86_64.whl", hash = "sha256:b1440c291db3f98a914e1afd9d6541e8fc60b4c3aab1a9008d03da4651e67386"}, + {file = "rpds_py-0.9.2-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:0f2996fbac8e0b77fd67102becb9229986396e051f33dbceada3debaacc7033f"}, + {file = "rpds_py-0.9.2-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:9f30d205755566a25f2ae0382944fcae2f350500ae4df4e795efa9e850821d82"}, + {file = "rpds_py-0.9.2-cp312-cp312-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:159fba751a1e6b1c69244e23ba6c28f879a8758a3e992ed056d86d74a194a0f3"}, + {file = "rpds_py-0.9.2-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:a1f044792e1adcea82468a72310c66a7f08728d72a244730d14880cd1dabe36b"}, + {file = "rpds_py-0.9.2-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:9251eb8aa82e6cf88510530b29eef4fac825a2b709baf5b94a6094894f252387"}, + {file = "rpds_py-0.9.2-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:01899794b654e616c8625b194ddd1e5b51ef5b60ed61baa7a2d9c2ad7b2a4238"}, + {file = "rpds_py-0.9.2-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:b0c43f8ae8f6be1d605b0465671124aa8d6a0e40f1fb81dcea28b7e3d87ca1e1"}, + {file = "rpds_py-0.9.2-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:207f57c402d1f8712618f737356e4b6f35253b6d20a324d9a47cb9f38ee43a6b"}, + {file = "rpds_py-0.9.2-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:b52e7c5ae35b00566d244ffefba0f46bb6bec749a50412acf42b1c3f402e2c90"}, + {file = "rpds_py-0.9.2-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:978fa96dbb005d599ec4fd9ed301b1cc45f1a8f7982d4793faf20b404b56677d"}, + {file = "rpds_py-0.9.2-cp38-cp38-macosx_10_7_x86_64.whl", hash = "sha256:6aa8326a4a608e1c28da191edd7c924dff445251b94653988efb059b16577a4d"}, + {file = "rpds_py-0.9.2-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:aad51239bee6bff6823bbbdc8ad85136c6125542bbc609e035ab98ca1e32a192"}, + {file = "rpds_py-0.9.2-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4bd4dc3602370679c2dfb818d9c97b1137d4dd412230cfecd3c66a1bf388a196"}, + {file = "rpds_py-0.9.2-cp38-cp38-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:dd9da77c6ec1f258387957b754f0df60766ac23ed698b61941ba9acccd3284d1"}, + {file = "rpds_py-0.9.2-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:190ca6f55042ea4649ed19c9093a9be9d63cd8a97880106747d7147f88a49d18"}, + {file = "rpds_py-0.9.2-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:876bf9ed62323bc7dcfc261dbc5572c996ef26fe6406b0ff985cbcf460fc8a4c"}, + {file = "rpds_py-0.9.2-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fa2818759aba55df50592ecbc95ebcdc99917fa7b55cc6796235b04193eb3c55"}, + {file = "rpds_py-0.9.2-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:9ea4d00850ef1e917815e59b078ecb338f6a8efda23369677c54a5825dbebb55"}, + {file = "rpds_py-0.9.2-cp38-cp38-musllinux_1_2_aarch64.whl", hash = "sha256:5855c85eb8b8a968a74dc7fb014c9166a05e7e7a8377fb91d78512900aadd13d"}, + {file = "rpds_py-0.9.2-cp38-cp38-musllinux_1_2_i686.whl", hash = "sha256:14c408e9d1a80dcb45c05a5149e5961aadb912fff42ca1dd9b68c0044904eb32"}, + {file = "rpds_py-0.9.2-cp38-cp38-musllinux_1_2_x86_64.whl", hash = "sha256:65a0583c43d9f22cb2130c7b110e695fff834fd5e832a776a107197e59a1898e"}, + {file = "rpds_py-0.9.2-cp38-none-win32.whl", hash = "sha256:71f2f7715935a61fa3e4ae91d91b67e571aeb5cb5d10331ab681256bda2ad920"}, + {file = "rpds_py-0.9.2-cp38-none-win_amd64.whl", hash = "sha256:674c704605092e3ebbbd13687b09c9f78c362a4bc710343efe37a91457123044"}, + {file = "rpds_py-0.9.2-cp39-cp39-macosx_10_7_x86_64.whl", hash = "sha256:07e2c54bef6838fa44c48dfbc8234e8e2466d851124b551fc4e07a1cfeb37260"}, + {file = "rpds_py-0.9.2-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:f7fdf55283ad38c33e35e2855565361f4bf0abd02470b8ab28d499c663bc5d7c"}, + {file = "rpds_py-0.9.2-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:890ba852c16ace6ed9f90e8670f2c1c178d96510a21b06d2fa12d8783a905193"}, + {file = "rpds_py-0.9.2-cp39-cp39-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:50025635ba8b629a86d9d5474e650da304cb46bbb4d18690532dd79341467846"}, + {file = "rpds_py-0.9.2-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:517cbf6e67ae3623c5127206489d69eb2bdb27239a3c3cc559350ef52a3bbf0b"}, + {file = "rpds_py-0.9.2-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:0836d71ca19071090d524739420a61580f3f894618d10b666cf3d9a1688355b1"}, + {file = "rpds_py-0.9.2-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9c439fd54b2b9053717cca3de9583be6584b384d88d045f97d409f0ca867d80f"}, + {file = "rpds_py-0.9.2-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:f68996a3b3dc9335037f82754f9cdbe3a95db42bde571d8c3be26cc6245f2324"}, + {file = "rpds_py-0.9.2-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:7d68dc8acded354c972116f59b5eb2e5864432948e098c19fe6994926d8e15c3"}, + {file = "rpds_py-0.9.2-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:f963c6b1218b96db85fc37a9f0851eaf8b9040aa46dec112611697a7023da535"}, + {file = "rpds_py-0.9.2-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:5a46859d7f947061b4010e554ccd1791467d1b1759f2dc2ec9055fa239f1bc26"}, + {file = "rpds_py-0.9.2-cp39-none-win32.whl", hash = "sha256:e07e5dbf8a83c66783a9fe2d4566968ea8c161199680e8ad38d53e075df5f0d0"}, + {file = "rpds_py-0.9.2-cp39-none-win_amd64.whl", hash = "sha256:682726178138ea45a0766907957b60f3a1bf3acdf212436be9733f28b6c5af3c"}, + {file = "rpds_py-0.9.2-pp310-pypy310_pp73-macosx_10_7_x86_64.whl", hash = "sha256:196cb208825a8b9c8fc360dc0f87993b8b260038615230242bf18ec84447c08d"}, + {file = "rpds_py-0.9.2-pp310-pypy310_pp73-macosx_11_0_arm64.whl", hash = "sha256:c7671d45530fcb6d5e22fd40c97e1e1e01965fc298cbda523bb640f3d923b387"}, + {file = "rpds_py-0.9.2-pp310-pypy310_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:83b32f0940adec65099f3b1c215ef7f1d025d13ff947975a055989cb7fd019a4"}, + {file = "rpds_py-0.9.2-pp310-pypy310_pp73-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:7f67da97f5b9eac838b6980fc6da268622e91f8960e083a34533ca710bec8611"}, + {file = "rpds_py-0.9.2-pp310-pypy310_pp73-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:03975db5f103997904c37e804e5f340c8fdabbb5883f26ee50a255d664eed58c"}, + {file = "rpds_py-0.9.2-pp310-pypy310_pp73-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:987b06d1cdb28f88a42e4fb8a87f094e43f3c435ed8e486533aea0bf2e53d931"}, + {file = "rpds_py-0.9.2-pp310-pypy310_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c861a7e4aef15ff91233751619ce3a3d2b9e5877e0fcd76f9ea4f6847183aa16"}, + {file = "rpds_py-0.9.2-pp310-pypy310_pp73-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:02938432352359805b6da099c9c95c8a0547fe4b274ce8f1a91677401bb9a45f"}, + {file = "rpds_py-0.9.2-pp310-pypy310_pp73-musllinux_1_2_aarch64.whl", hash = "sha256:ef1f08f2a924837e112cba2953e15aacfccbbfcd773b4b9b4723f8f2ddded08e"}, + {file = "rpds_py-0.9.2-pp310-pypy310_pp73-musllinux_1_2_i686.whl", hash = "sha256:35da5cc5cb37c04c4ee03128ad59b8c3941a1e5cd398d78c37f716f32a9b7f67"}, + {file = "rpds_py-0.9.2-pp310-pypy310_pp73-musllinux_1_2_x86_64.whl", hash = "sha256:141acb9d4ccc04e704e5992d35472f78c35af047fa0cfae2923835d153f091be"}, + {file = "rpds_py-0.9.2-pp38-pypy38_pp73-macosx_10_7_x86_64.whl", hash = "sha256:79f594919d2c1a0cc17d1988a6adaf9a2f000d2e1048f71f298b056b1018e872"}, + {file = "rpds_py-0.9.2-pp38-pypy38_pp73-macosx_11_0_arm64.whl", hash = "sha256:a06418fe1155e72e16dddc68bb3780ae44cebb2912fbd8bb6ff9161de56e1798"}, + {file = "rpds_py-0.9.2-pp38-pypy38_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8b2eb034c94b0b96d5eddb290b7b5198460e2d5d0c421751713953a9c4e47d10"}, + {file = "rpds_py-0.9.2-pp38-pypy38_pp73-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:8b08605d248b974eb02f40bdcd1a35d3924c83a2a5e8f5d0fa5af852c4d960af"}, + {file = "rpds_py-0.9.2-pp38-pypy38_pp73-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:a0805911caedfe2736935250be5008b261f10a729a303f676d3d5fea6900c96a"}, + {file = "rpds_py-0.9.2-pp38-pypy38_pp73-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:ab2299e3f92aa5417d5e16bb45bb4586171c1327568f638e8453c9f8d9e0f020"}, + {file = "rpds_py-0.9.2-pp38-pypy38_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8c8d7594e38cf98d8a7df25b440f684b510cf4627fe038c297a87496d10a174f"}, + {file = "rpds_py-0.9.2-pp38-pypy38_pp73-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:8b9ec12ad5f0a4625db34db7e0005be2632c1013b253a4a60e8302ad4d462afd"}, + {file = "rpds_py-0.9.2-pp38-pypy38_pp73-musllinux_1_2_aarch64.whl", hash = "sha256:1fcdee18fea97238ed17ab6478c66b2095e4ae7177e35fb71fbe561a27adf620"}, + {file = "rpds_py-0.9.2-pp38-pypy38_pp73-musllinux_1_2_i686.whl", hash = "sha256:933a7d5cd4b84f959aedeb84f2030f0a01d63ae6cf256629af3081cf3e3426e8"}, + {file = "rpds_py-0.9.2-pp38-pypy38_pp73-musllinux_1_2_x86_64.whl", hash = "sha256:686ba516e02db6d6f8c279d1641f7067ebb5dc58b1d0536c4aaebb7bf01cdc5d"}, + {file = "rpds_py-0.9.2-pp39-pypy39_pp73-macosx_10_7_x86_64.whl", hash = "sha256:0173c0444bec0a3d7d848eaeca2d8bd32a1b43f3d3fde6617aac3731fa4be05f"}, + {file = "rpds_py-0.9.2-pp39-pypy39_pp73-macosx_11_0_arm64.whl", hash = "sha256:d576c3ef8c7b2d560e301eb33891d1944d965a4d7a2eacb6332eee8a71827db6"}, + {file = "rpds_py-0.9.2-pp39-pypy39_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ed89861ee8c8c47d6beb742a602f912b1bb64f598b1e2f3d758948721d44d468"}, + {file = "rpds_py-0.9.2-pp39-pypy39_pp73-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:1054a08e818f8e18910f1bee731583fe8f899b0a0a5044c6e680ceea34f93876"}, + {file = "rpds_py-0.9.2-pp39-pypy39_pp73-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:99e7c4bb27ff1aab90dcc3e9d37ee5af0231ed98d99cb6f5250de28889a3d502"}, + {file = "rpds_py-0.9.2-pp39-pypy39_pp73-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:c545d9d14d47be716495076b659db179206e3fd997769bc01e2d550eeb685596"}, + {file = "rpds_py-0.9.2-pp39-pypy39_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9039a11bca3c41be5a58282ed81ae422fa680409022b996032a43badef2a3752"}, + {file = "rpds_py-0.9.2-pp39-pypy39_pp73-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:fb39aca7a64ad0c9490adfa719dbeeb87d13be137ca189d2564e596f8ba32c07"}, + {file = "rpds_py-0.9.2-pp39-pypy39_pp73-musllinux_1_2_aarch64.whl", hash = "sha256:2d8b3b3a2ce0eaa00c5bbbb60b6713e94e7e0becab7b3db6c5c77f979e8ed1f1"}, + {file = "rpds_py-0.9.2-pp39-pypy39_pp73-musllinux_1_2_i686.whl", hash = "sha256:99b1c16f732b3a9971406fbfe18468592c5a3529585a45a35adbc1389a529a03"}, + {file = "rpds_py-0.9.2-pp39-pypy39_pp73-musllinux_1_2_x86_64.whl", hash = "sha256:c27ee01a6c3223025f4badd533bea5e87c988cb0ba2811b690395dfe16088cfe"}, + {file = "rpds_py-0.9.2.tar.gz", hash = "sha256:8d70e8f14900f2657c249ea4def963bed86a29b81f81f5b76b5a9215680de945"}, +] + [[package]] name = "s3fs" -version = "2023.5.0" +version = "2023.6.0" description = "Convenient Filesystem interface over S3" -category = "main" optional = true python-versions = ">= 3.8" files = [ - {file = "s3fs-2023.5.0-py3-none-any.whl", hash = "sha256:0d82c4fa43d1214117f56b239c3e03c9a2886f41c31000c1c967ac6030d20362"}, - {file = "s3fs-2023.5.0.tar.gz", hash = "sha256:106b5d9a1000e6af413f918156ba4b96789ac832b7e08c99d186eb08164e6981"}, + {file = "s3fs-2023.6.0-py3-none-any.whl", hash = "sha256:d1a0a423d0d2e17fb2a193d9531935dc3f45ba742693448a461b6b34f6a92a24"}, + {file = "s3fs-2023.6.0.tar.gz", hash = "sha256:63fd8ddf05eb722de784b7b503196107f2a518061298cf005a8a4715b4d49117"}, ] [package.dependencies] aiobotocore = ">=2.5.0,<2.6.0" aiohttp = "<4.0.0a0 || >4.0.0a0,<4.0.0a1 || >4.0.0a1" -fsspec = "2023.5.0" +fsspec = "2023.6.0" [package.extras] awscli = ["aiobotocore[awscli] (>=2.5.0,<2.6.0)"] @@ -2441,7 +2548,6 @@ boto3 = ["aiobotocore[boto3] (>=2.5.0,<2.6.0)"] name = "s3transfer" version = "0.6.1" description = "An Amazon S3 Transfer Manager" -category = "main" optional = false python-versions = ">= 3.7" files = [ @@ -2457,14 +2563,13 @@ crt = ["botocore[crt] (>=1.20.29,<2.0a.0)"] [[package]] name = "setuptools" -version = "67.8.0" +version = "68.0.0" description = "Easily download, build, install, upgrade, and uninstall Python packages" -category = "dev" optional = false python-versions = ">=3.7" files = [ - {file = "setuptools-67.8.0-py3-none-any.whl", hash = "sha256:5df61bf30bb10c6f756eb19e7c9f3b473051f48db77fddbe06ff2ca307df9a6f"}, - {file = "setuptools-67.8.0.tar.gz", hash = "sha256:62642358adc77ffa87233bc4d2354c4b2682d214048f500964dbe760ccedf102"}, + {file = "setuptools-68.0.0-py3-none-any.whl", hash = "sha256:11e52c67415a381d10d6b462ced9cfb97066179f0e871399e006c4ab101fc85f"}, + {file = "setuptools-68.0.0.tar.gz", hash = "sha256:baf1fdb41c6da4cd2eae722e135500da913332ab3f2f5c7d33af9b492acb5235"}, ] [package.extras] @@ -2476,7 +2581,6 @@ testing-integration = ["build[virtualenv]", "filelock (>=3.4.0)", "jaraco.envs ( name = "six" version = "1.16.0" description = "Python 2 and 3 compatibility utilities" -category = "main" optional = false python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*" files = [ @@ -2484,11 +2588,99 @@ files = [ {file = "six-1.16.0.tar.gz", hash = "sha256:1e61c37477a1626458e36f7b1d82aa5c9b094fa4802892072e49de9c60c4c926"}, ] +[[package]] +name = "sortedcontainers" +version = "2.4.0" +description = "Sorted Containers -- Sorted List, Sorted Dict, Sorted Set" +optional = false +python-versions = "*" +files = [ + {file = "sortedcontainers-2.4.0-py2.py3-none-any.whl", hash = "sha256:a163dcaede0f1c021485e957a39245190e74249897e2ae4b2aa38595db237ee0"}, + {file = "sortedcontainers-2.4.0.tar.gz", hash = "sha256:25caa5a06cc30b6b83d11423433f65d1f9d76c4c6a0c90e3379eaa43b9bfdb88"}, +] + +[[package]] +name = "sqlalchemy" +version = "2.0.19" +description = "Database Abstraction Library" +optional = true +python-versions = ">=3.7" +files = [ + {file = "SQLAlchemy-2.0.19-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:9deaae357edc2091a9ed5d25e9ee8bba98bcfae454b3911adeaf159c2e9ca9e3"}, + {file = "SQLAlchemy-2.0.19-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:0bf0fd65b50a330261ec7fe3d091dfc1c577483c96a9fa1e4323e932961aa1b5"}, + {file = "SQLAlchemy-2.0.19-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1d90ccc15ba1baa345796a8fb1965223ca7ded2d235ccbef80a47b85cea2d71a"}, + {file = "SQLAlchemy-2.0.19-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:cb4e688f6784427e5f9479d1a13617f573de8f7d4aa713ba82813bcd16e259d1"}, + {file = "SQLAlchemy-2.0.19-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:584f66e5e1979a7a00f4935015840be627e31ca29ad13f49a6e51e97a3fb8cae"}, + {file = "SQLAlchemy-2.0.19-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:2c69ce70047b801d2aba3e5ff3cba32014558966109fecab0c39d16c18510f15"}, + {file = "SQLAlchemy-2.0.19-cp310-cp310-win32.whl", hash = "sha256:96f0463573469579d32ad0c91929548d78314ef95c210a8115346271beeeaaa2"}, + {file = "SQLAlchemy-2.0.19-cp310-cp310-win_amd64.whl", hash = "sha256:22bafb1da60c24514c141a7ff852b52f9f573fb933b1e6b5263f0daa28ce6db9"}, + {file = "SQLAlchemy-2.0.19-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:d6894708eeb81f6d8193e996257223b6bb4041cb05a17cd5cf373ed836ef87a2"}, + {file = "SQLAlchemy-2.0.19-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:d8f2afd1aafded7362b397581772c670f20ea84d0a780b93a1a1529da7c3d369"}, + {file = "SQLAlchemy-2.0.19-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b15afbf5aa76f2241184c1d3b61af1a72ba31ce4161013d7cb5c4c2fca04fd6e"}, + {file = "SQLAlchemy-2.0.19-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8fc05b59142445a4efb9c1fd75c334b431d35c304b0e33f4fa0ff1ea4890f92e"}, + {file = "SQLAlchemy-2.0.19-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:5831138f0cc06b43edf5f99541c64adf0ab0d41f9a4471fd63b54ae18399e4de"}, + {file = "SQLAlchemy-2.0.19-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:3afa8a21a9046917b3a12ffe016ba7ebe7a55a6fc0c7d950beb303c735c3c3ad"}, + {file = "SQLAlchemy-2.0.19-cp311-cp311-win32.whl", hash = "sha256:c896d4e6ab2eba2afa1d56be3d0b936c56d4666e789bfc59d6ae76e9fcf46145"}, + {file = "SQLAlchemy-2.0.19-cp311-cp311-win_amd64.whl", hash = "sha256:024d2f67fb3ec697555e48caeb7147cfe2c08065a4f1a52d93c3d44fc8e6ad1c"}, + {file = "SQLAlchemy-2.0.19-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:89bc2b374ebee1a02fd2eae6fd0570b5ad897ee514e0f84c5c137c942772aa0c"}, + {file = "SQLAlchemy-2.0.19-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:dd4d410a76c3762511ae075d50f379ae09551d92525aa5bb307f8343bf7c2c12"}, + {file = "SQLAlchemy-2.0.19-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f469f15068cd8351826df4080ffe4cc6377c5bf7d29b5a07b0e717dddb4c7ea2"}, + {file = "SQLAlchemy-2.0.19-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:cda283700c984e699e8ef0fcc5c61f00c9d14b6f65a4f2767c97242513fcdd84"}, + {file = "SQLAlchemy-2.0.19-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:43699eb3f80920cc39a380c159ae21c8a8924fe071bccb68fc509e099420b148"}, + {file = "SQLAlchemy-2.0.19-cp37-cp37m-win32.whl", hash = "sha256:61ada5831db36d897e28eb95f0f81814525e0d7927fb51145526c4e63174920b"}, + {file = "SQLAlchemy-2.0.19-cp37-cp37m-win_amd64.whl", hash = "sha256:57d100a421d9ab4874f51285c059003292433c648df6abe6c9c904e5bd5b0828"}, + {file = "SQLAlchemy-2.0.19-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:16a310f5bc75a5b2ce7cb656d0e76eb13440b8354f927ff15cbaddd2523ee2d1"}, + {file = "SQLAlchemy-2.0.19-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:cf7b5e3856cbf1876da4e9d9715546fa26b6e0ba1a682d5ed2fc3ca4c7c3ec5b"}, + {file = "SQLAlchemy-2.0.19-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:2e7b69d9ced4b53310a87117824b23c509c6fc1f692aa7272d47561347e133b6"}, + {file = "SQLAlchemy-2.0.19-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8f9eb4575bfa5afc4b066528302bf12083da3175f71b64a43a7c0badda2be365"}, + {file = "SQLAlchemy-2.0.19-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:6b54d1ad7a162857bb7c8ef689049c7cd9eae2f38864fc096d62ae10bc100c7d"}, + {file = "SQLAlchemy-2.0.19-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:5d6afc41ca0ecf373366fd8e10aee2797128d3ae45eb8467b19da4899bcd1ee0"}, + {file = "SQLAlchemy-2.0.19-cp38-cp38-win32.whl", hash = "sha256:430614f18443b58ceb9dedec323ecddc0abb2b34e79d03503b5a7579cd73a531"}, + {file = "SQLAlchemy-2.0.19-cp38-cp38-win_amd64.whl", hash = "sha256:eb60699de43ba1a1f77363f563bb2c652f7748127ba3a774f7cf2c7804aa0d3d"}, + {file = "SQLAlchemy-2.0.19-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:a752b7a9aceb0ba173955d4f780c64ee15a1a991f1c52d307d6215c6c73b3a4c"}, + {file = "SQLAlchemy-2.0.19-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:7351c05db355da112e056a7b731253cbeffab9dfdb3be1e895368513c7d70106"}, + {file = "SQLAlchemy-2.0.19-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:fa51ce4aea583b0c6b426f4b0563d3535c1c75986c4373a0987d84d22376585b"}, + {file = "SQLAlchemy-2.0.19-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ae7473a67cd82a41decfea58c0eac581209a0aa30f8bc9190926fbf628bb17f7"}, + {file = "SQLAlchemy-2.0.19-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:851a37898a8a39783aab603c7348eb5b20d83c76a14766a43f56e6ad422d1ec8"}, + {file = "SQLAlchemy-2.0.19-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:539010665c90e60c4a1650afe4ab49ca100c74e6aef882466f1de6471d414be7"}, + {file = "SQLAlchemy-2.0.19-cp39-cp39-win32.whl", hash = "sha256:f82c310ddf97b04e1392c33cf9a70909e0ae10a7e2ddc1d64495e3abdc5d19fb"}, + {file = "SQLAlchemy-2.0.19-cp39-cp39-win_amd64.whl", hash = "sha256:8e712cfd2e07b801bc6b60fdf64853bc2bd0af33ca8fa46166a23fe11ce0dbb0"}, + {file = "SQLAlchemy-2.0.19-py3-none-any.whl", hash = "sha256:314145c1389b021a9ad5aa3a18bac6f5d939f9087d7fc5443be28cba19d2c972"}, + {file = "SQLAlchemy-2.0.19.tar.gz", hash = "sha256:77a14fa20264af73ddcdb1e2b9c5a829b8cc6b8304d0f093271980e36c200a3f"}, +] + +[package.dependencies] +greenlet = {version = "!=0.4.17", markers = "platform_machine == \"win32\" or platform_machine == \"WIN32\" or platform_machine == \"AMD64\" or platform_machine == \"amd64\" or platform_machine == \"x86_64\" or platform_machine == \"ppc64le\" or platform_machine == \"aarch64\""} +typing-extensions = ">=4.2.0" + +[package.extras] +aiomysql = ["aiomysql", "greenlet (!=0.4.17)"] +aiosqlite = ["aiosqlite", "greenlet (!=0.4.17)", "typing-extensions (!=3.10.0.1)"] +asyncio = ["greenlet (!=0.4.17)"] +asyncmy = ["asyncmy (>=0.2.3,!=0.2.4,!=0.2.6)", "greenlet (!=0.4.17)"] +mariadb-connector = ["mariadb (>=1.0.1,!=1.1.2,!=1.1.5)"] +mssql = ["pyodbc"] +mssql-pymssql = ["pymssql"] +mssql-pyodbc = ["pyodbc"] +mypy = ["mypy (>=0.910)"] +mysql = ["mysqlclient (>=1.4.0)"] +mysql-connector = ["mysql-connector-python"] +oracle = ["cx-oracle (>=7)"] +oracle-oracledb = ["oracledb (>=1.0.1)"] +postgresql = ["psycopg2 (>=2.7)"] +postgresql-asyncpg = ["asyncpg", "greenlet (!=0.4.17)"] +postgresql-pg8000 = ["pg8000 (>=1.29.1)"] +postgresql-psycopg = ["psycopg (>=3.0.7)"] +postgresql-psycopg2binary = ["psycopg2-binary"] +postgresql-psycopg2cffi = ["psycopg2cffi"] +postgresql-psycopgbinary = ["psycopg[binary] (>=3.0.7)"] +pymysql = ["pymysql"] +sqlcipher = ["sqlcipher3-binary"] + [[package]] name = "strictyaml" version = "1.7.3" description = "Strict, typed YAML parser" -category = "main" optional = false python-versions = ">=3.7.0" files = [ @@ -2503,7 +2695,6 @@ python-dateutil = ">=2.6.0" name = "thrift" version = "0.16.0" description = "Python bindings for the Apache Thrift RPC system" -category = "main" optional = true python-versions = "*" files = [ @@ -2522,7 +2713,6 @@ twisted = ["twisted"] name = "tomli" version = "2.0.1" description = "A lil' TOML parser" -category = "dev" optional = false python-versions = ">=3.7" files = [ @@ -2532,33 +2722,30 @@ files = [ [[package]] name = "types-pyyaml" -version = "6.0.12.10" +version = "6.0.12.11" description = "Typing stubs for PyYAML" -category = "dev" optional = false python-versions = "*" files = [ - {file = "types-PyYAML-6.0.12.10.tar.gz", hash = "sha256:ebab3d0700b946553724ae6ca636ea932c1b0868701d4af121630e78d695fc97"}, - {file = "types_PyYAML-6.0.12.10-py3-none-any.whl", hash = "sha256:662fa444963eff9b68120d70cda1af5a5f2aa57900003c2006d7626450eaae5f"}, + {file = "types-PyYAML-6.0.12.11.tar.gz", hash = "sha256:7d340b19ca28cddfdba438ee638cd4084bde213e501a3978738543e27094775b"}, + {file = "types_PyYAML-6.0.12.11-py3-none-any.whl", hash = "sha256:a461508f3096d1d5810ec5ab95d7eeecb651f3a15b71959999988942063bf01d"}, ] [[package]] name = "typing-extensions" -version = "4.6.3" +version = "4.7.1" description = "Backported and Experimental Type Hints for Python 3.7+" -category = "main" optional = false python-versions = ">=3.7" files = [ - {file = "typing_extensions-4.6.3-py3-none-any.whl", hash = "sha256:88a4153d8505aabbb4e13aacb7c486c2b4a33ca3b3f807914a9b4c844c471c26"}, - {file = "typing_extensions-4.6.3.tar.gz", hash = "sha256:d91d5919357fe7f681a9f2b5b4cb2a5f1ef0a1e9f59c4d8ff0d3491e05c0ffd5"}, + {file = "typing_extensions-4.7.1-py3-none-any.whl", hash = "sha256:440d5dd3af93b060174bf433bccd69b0babc3b15b1a8dca43789fd7f61514b36"}, + {file = "typing_extensions-4.7.1.tar.gz", hash = "sha256:b75ddc264f0ba5615db7ba217daeb99701ad295353c45f9e95963337ceeeffb2"}, ] [[package]] name = "tzdata" version = "2023.3" description = "Provider of IANA time zone data" -category = "main" optional = true python-versions = ">=2" files = [ @@ -2570,7 +2757,6 @@ files = [ name = "urllib3" version = "1.26.16" description = "HTTP library with thread-safe connection pooling, file post, and more." -category = "main" optional = false python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*, !=3.4.*, !=3.5.*" files = [ @@ -2585,35 +2771,33 @@ socks = ["PySocks (>=1.5.6,!=1.5.7,<2.0)"] [[package]] name = "virtualenv" -version = "20.21.0" +version = "20.24.1" description = "Virtual Python Environment builder" -category = "main" optional = false python-versions = ">=3.7" files = [ - {file = "virtualenv-20.21.0-py3-none-any.whl", hash = "sha256:31712f8f2a17bd06234fa97fdf19609e789dd4e3e4bf108c3da71d710651adbc"}, - {file = "virtualenv-20.21.0.tar.gz", hash = "sha256:f50e3e60f990a0757c9b68333c9fdaa72d7188caa417f96af9e52407831a3b68"}, + {file = "virtualenv-20.24.1-py3-none-any.whl", hash = "sha256:01aacf8decd346cf9a865ae85c0cdc7f64c8caa07ff0d8b1dfc1733d10677442"}, + {file = "virtualenv-20.24.1.tar.gz", hash = "sha256:2ef6a237c31629da6442b0bcaa3999748108c7166318d1f55cc9f8d7294e97bd"}, ] [package.dependencies] distlib = ">=0.3.6,<1" -filelock = ">=3.4.1,<4" -platformdirs = ">=2.4,<4" +filelock = ">=3.12,<4" +platformdirs = ">=3.5.1,<4" [package.extras] -docs = ["furo (>=2022.12.7)", "proselint (>=0.13)", "sphinx (>=6.1.3)", "sphinx-argparse (>=0.4)", "sphinxcontrib-towncrier (>=0.2.1a0)", "towncrier (>=22.12)"] -test = ["covdefaults (>=2.2.2)", "coverage (>=7.1)", "coverage-enable-subprocess (>=1)", "flaky (>=3.7)", "packaging (>=23)", "pytest (>=7.2.1)", "pytest-env (>=0.8.1)", "pytest-freezegun (>=0.4.2)", "pytest-mock (>=3.10)", "pytest-randomly (>=3.12)", "pytest-timeout (>=2.1)"] +docs = ["furo (>=2023.5.20)", "proselint (>=0.13)", "sphinx (>=7.0.1)", "sphinx-argparse (>=0.4)", "sphinxcontrib-towncrier (>=0.2.1a0)", "towncrier (>=23.6)"] +test = ["covdefaults (>=2.3)", "coverage (>=7.2.7)", "coverage-enable-subprocess (>=1)", "flaky (>=3.7)", "packaging (>=23.1)", "pytest (>=7.3.1)", "pytest-env (>=0.8.1)", "pytest-freezer (>=0.4.6)", "pytest-mock (>=3.10)", "pytest-randomly (>=3.12)", "pytest-timeout (>=2.1)", "setuptools (>=67.8)", "time-machine (>=2.9)"] [[package]] name = "werkzeug" -version = "2.3.4" +version = "2.3.6" description = "The comprehensive WSGI web application library." -category = "dev" optional = false python-versions = ">=3.8" files = [ - {file = "Werkzeug-2.3.4-py3-none-any.whl", hash = "sha256:48e5e61472fee0ddee27ebad085614ebedb7af41e88f687aaf881afb723a162f"}, - {file = "Werkzeug-2.3.4.tar.gz", hash = "sha256:1d5a58e0377d1fe39d061a5de4469e414e78ccb1e1e59c0f5ad6fa1c36c52b76"}, + {file = "Werkzeug-2.3.6-py3-none-any.whl", hash = "sha256:935539fa1413afbb9195b24880778422ed620c0fc09670945185cce4d91a8890"}, + {file = "Werkzeug-2.3.6.tar.gz", hash = "sha256:98c774df2f91b05550078891dee5f0eb0cb797a522c757a2452b9cee5b202330"}, ] [package.dependencies] @@ -2626,7 +2810,6 @@ watchdog = ["watchdog (>=2.3)"] name = "wrapt" version = "1.15.0" description = "Module for decorators, wrappers and monkey patching." -category = "main" optional = true python-versions = "!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*,!=3.4.*,>=2.7" files = [ @@ -2711,7 +2894,6 @@ files = [ name = "xmltodict" version = "0.13.0" description = "Makes working with XML feel like you are working with JSON" -category = "dev" optional = false python-versions = ">=3.4" files = [ @@ -2723,7 +2905,6 @@ files = [ name = "yarl" version = "1.9.2" description = "Yet another URL library" -category = "main" optional = true python-versions = ">=3.7" files = [ @@ -2809,25 +2990,23 @@ multidict = ">=4.0" [[package]] name = "zipp" -version = "3.15.0" +version = "3.16.2" description = "Backport of pathlib-compatible object wrapper for zip files" -category = "main" optional = false -python-versions = ">=3.7" +python-versions = ">=3.8" files = [ - {file = "zipp-3.15.0-py3-none-any.whl", hash = "sha256:48904fc76a60e542af151aded95726c1a5c34ed43ab4134b597665c86d7ad556"}, - {file = "zipp-3.15.0.tar.gz", hash = "sha256:112929ad649da941c23de50f356a2b5570c954b65150642bccdd66bf194d224b"}, + {file = "zipp-3.16.2-py3-none-any.whl", hash = "sha256:679e51dd4403591b2d6838a48de3d283f3d188412a9782faadf845f298736ba0"}, + {file = "zipp-3.16.2.tar.gz", hash = "sha256:ebc15946aa78bd63458992fc81ec3b6f7b1e92d51c35e6de1c3804e73b799147"}, ] [package.extras] -docs = ["furo", "jaraco.packaging (>=9)", "jaraco.tidelift (>=1.4)", "rst.linker (>=1.9)", "sphinx (>=3.5)", "sphinx-lint"] -testing = ["big-O", "flake8 (<5)", "jaraco.functools", "jaraco.itertools", "more-itertools", "pytest (>=6)", "pytest-black (>=0.3.7)", "pytest-checkdocs (>=2.4)", "pytest-cov", "pytest-enabler (>=1.3)", "pytest-flake8", "pytest-mypy (>=0.9.1)"] +docs = ["furo", "jaraco.packaging (>=9.3)", "jaraco.tidelift (>=1.4)", "rst.linker (>=1.9)", "sphinx (>=3.5)", "sphinx-lint"] +testing = ["big-O", "jaraco.functools", "jaraco.itertools", "more-itertools", "pytest (>=6)", "pytest-black (>=0.3.7)", "pytest-checkdocs (>=2.4)", "pytest-cov", "pytest-enabler (>=2.2)", "pytest-ignore-flaky", "pytest-mypy (>=0.9.1)", "pytest-ruff"] [[package]] name = "zstandard" version = "0.21.0" description = "Zstandard bindings for Python" -category = "main" optional = false python-versions = ">=3.7" files = [ @@ -2890,12 +3069,13 @@ glue = ["boto3"] hive = ["thrift"] pandas = ["pandas", "pyarrow"] pyarrow = ["pyarrow"] -ray = ["ray", "pyarrow", "pandas"] +ray = ["pandas", "pyarrow", "ray"] s3fs = ["s3fs"] snappy = ["python-snappy"] +sql-postgres = ["psycopg2-binary", "sqlalchemy"] zstandard = ["zstandard"] [metadata] lock-version = "2.0" python-versions = "^3.8" -content-hash = "e11ad53a6e619deaf308393a4df3d3b3ba4ed142c90fda6dda0c7b04c86bf4cb" +content-hash = "5d2066d7196f6defe9f350a0e20f46e54753b3b2a2ad8de0e655e6f0901d8fea" diff --git a/python/pyiceberg/__init__.py b/python/pyiceberg/__init__.py index 12badee19175..e725b34eaeed 100644 --- a/python/pyiceberg/__init__.py +++ b/python/pyiceberg/__init__.py @@ -15,4 +15,4 @@ # specific language governing permissions and limitations # under the License. -__version__ = "0.3.0" +__version__ = "0.4.0" diff --git a/python/pyiceberg/avro/__init__.py b/python/pyiceberg/avro/__init__.py index 13a83393a912..d7d8b55ef913 100644 --- a/python/pyiceberg/avro/__init__.py +++ b/python/pyiceberg/avro/__init__.py @@ -14,3 +14,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +import struct + +STRUCT_FLOAT = struct.Struct(" None: - """Incrementally compute CRC-32 from bytes and compare to a checksum + """Incrementally compute CRC-32 from bytes and compare to a checksum. Args: bytes_ (bytes): The bytes to check against `checksum` diff --git a/python/pyiceberg/avro/decoder.py b/python/pyiceberg/avro/decoder.py index a65a7160055a..35a1651192c3 100644 --- a/python/pyiceberg/avro/decoder.py +++ b/python/pyiceberg/avro/decoder.py @@ -15,69 +15,50 @@ # specific language governing permissions and limitations # under the License. import decimal -import struct +from abc import ABC, abstractmethod from datetime import datetime, time from io import SEEK_CUR -from typing import List +from typing import Dict, List from uuid import UUID +from pyiceberg.avro import STRUCT_DOUBLE, STRUCT_FLOAT from pyiceberg.io import InputStream from pyiceberg.utils.datetime import micros_to_time, micros_to_timestamp, micros_to_timestamptz from pyiceberg.utils.decimal import unscaled_to_decimal -STRUCT_FLOAT = struct.Struct("h") # big-endian signed short -STRUCT_SIGNED_INT = struct.Struct(">i") # big-endian signed int -STRUCT_SIGNED_LONG = struct.Struct(">q") # big-endian signed long - -class BinaryDecoder: +class BinaryDecoder(ABC): """Read leaf values.""" - _input_stream: InputStream - + @abstractmethod def __init__(self, input_stream: InputStream) -> None: - """ - reader is a Python object on which we can call read, seek, and tell. - """ - self._input_stream = input_stream + """Create the decoder.""" - def read(self, n: int) -> bytes: - """ - Read n bytes. - """ - if n < 0: - raise ValueError(f"Requested {n} bytes to read, expected positive integer.") - data: List[bytes] = [] - - n_remaining = n - while n_remaining > 0: - data_read = self._input_stream.read(n_remaining) - read_len = len(data_read) - if read_len == n: - # If we read everything, we return directly - # otherwise we'll continue to fetch the rest - return data_read - elif read_len <= 0: - raise EOFError(f"EOF: read {read_len} bytes") - data.append(data_read) - n_remaining -= read_len + @abstractmethod + def tell(self) -> int: + """Return the current position.""" - return b"".join(data) + @abstractmethod + def read(self, n: int) -> bytes: + """Read n bytes.""" + @abstractmethod def skip(self, n: int) -> None: - self._input_stream.seek(n, SEEK_CUR) + """Skip n bytes.""" def read_boolean(self) -> bool: - """ - a boolean is written as a single byte + """Reads a value from the stream as a boolean. + + A boolean is written as a single byte whose value is either 0 (false) or 1 (true). """ return ord(self.read(1)) == 1 def read_int(self) -> int: - """int/long values are written using variable-length, zigzag coding.""" + """Reads an int/long value. + + int/long values are written using variable-length, zigzag coding. + """ b = ord(self.read(1)) n = b & 0x7F shift = 7 @@ -88,8 +69,28 @@ def read_int(self) -> int: datum = (n >> 1) ^ -(n & 1) return datum + def read_ints(self, n: int, dest: List[int]) -> None: + """Reads a list of integers.""" + for _ in range(n): + dest.append(self.read_int()) + + def read_int_int_dict(self, n: int, dest: Dict[int, int]) -> None: + """Reads a dictionary of integers for keys and values into a destination dictionary.""" + for _ in range(n): + k = self.read_int() + v = self.read_int() + dest[k] = v + + def read_int_bytes_dict(self, n: int, dest: Dict[int, bytes]) -> None: + """Reads a dictionary of integers for keys and bytes for values into a destination dictionary.""" + for _ in range(n): + k = self.read_int() + v = self.read_bytes() + dest[k] = v + def read_float(self) -> float: - """ + """Reads a value from the stream as a float. + A float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java's floatToIntBits and then encoded in little-endian format. @@ -97,7 +98,8 @@ def read_float(self) -> float: return float(STRUCT_FLOAT.unpack(self.read(4))[0]) def read_double(self) -> float: - """ + """Reads a value from the stream as a double. + A double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java's doubleToLongBits and then encoded in little-endian format. @@ -105,7 +107,8 @@ def read_double(self) -> float: return float(STRUCT_DOUBLE.unpack(self.read(8))[0]) def read_decimal_from_bytes(self, precision: int, scale: int) -> decimal.Decimal: - """ + """Reads a value from the stream as a decimal. + Decimal bytes are decoded as signed short, int or long depending on the size of bytes. """ @@ -113,7 +116,8 @@ def read_decimal_from_bytes(self, precision: int, scale: int) -> decimal.Decimal return self.read_decimal_from_fixed(precision, scale, size) def read_decimal_from_fixed(self, _: int, scale: int, size: int) -> decimal.Decimal: - """ + """Reads a value from the stream as a decimal. + Decimal is encoded as fixed. Fixed instances are encoded using the number of bytes declared in the schema. """ @@ -122,51 +126,54 @@ def read_decimal_from_fixed(self, _: int, scale: int, size: int) -> decimal.Deci return unscaled_to_decimal(unscaled_datum, scale) def read_bytes(self) -> bytes: - """ - Bytes are encoded as a long followed by that many bytes of data. - """ + """Bytes are encoded as a long followed by that many bytes of data.""" num_bytes = self.read_int() return self.read(num_bytes) if num_bytes > 0 else b"" def read_utf8(self) -> str: - """ + """Reads a utf-8 encoded string from the stream. + A string is encoded as a long followed by that many bytes of UTF-8 encoded character data. """ return self.read_bytes().decode("utf-8") def read_uuid_from_fixed(self) -> UUID: - """Reads a UUID as a fixed[16]""" + """Reads a UUID as a fixed[16].""" return UUID(bytes=self.read(16)) def read_time_millis(self) -> time: - """ - int is decoded as python time object which represents + """Reads a milliseconds granularity time from the stream. + + Int is decoded as python time object which represents the number of milliseconds after midnight, 00:00:00.000. """ millis = self.read_int() return micros_to_time(millis * 1000) def read_time_micros(self) -> time: - """ - long is decoded as python time object which represents + """Reads a microseconds granularity time from the stream. + + Long is decoded as python time object which represents the number of microseconds after midnight, 00:00:00.000000. """ return micros_to_time(self.read_int()) def read_timestamp_micros(self) -> datetime: - """ - long is decoded as python datetime object which represents + """Reads a microsecond granularity timestamp from the stream. + + Long is decoded as python datetime object which represents the number of microseconds from the unix epoch, 1 January 1970. """ return micros_to_timestamp(self.read_int()) def read_timestamptz_micros(self) -> datetime: - """ - long is decoded as python datetime object which represents + """Reads a microsecond granularity timestamptz from the stream. + + Long is decoded as python datetime object which represents the number of microseconds from the unix epoch, 1 January 1970. - Adjusted to UTC + Adjusted to UTC. """ return micros_to_timestamptz(self.read_int()) @@ -189,3 +196,136 @@ def skip_bytes(self) -> None: def skip_utf8(self) -> None: self.skip_bytes() + + +class StreamingBinaryDecoder(BinaryDecoder): + """Read leaf values.""" + + __slots__ = "_input_stream" + _input_stream: InputStream + + def __init__(self, input_stream: InputStream) -> None: + """Reader is a Python object on which we can call read, seek, and tell.""" + super().__init__(input_stream) + self._input_stream = input_stream + + def tell(self) -> int: + """Return the current stream position.""" + return self._input_stream.tell() + + def read(self, n: int) -> bytes: + """Read n bytes.""" + if n < 0: + raise ValueError(f"Requested {n} bytes to read, expected positive integer.") + data: List[bytes] = [] + + n_remaining = n + while n_remaining > 0: + data_read = self._input_stream.read(n_remaining) + read_len = len(data_read) + if read_len == n: + # If we read everything, we return directly + # otherwise we'll continue to fetch the rest + return data_read + elif read_len <= 0: + raise EOFError(f"EOF: read {read_len} bytes") + data.append(data_read) + n_remaining -= read_len + + return b"".join(data) + + def skip(self, n: int) -> None: + self._input_stream.seek(n, SEEK_CUR) + + +class InMemoryBinaryDecoder(BinaryDecoder): + """Implement a BinaryDecoder that reads from an in-memory buffer. + + This may be more efficient if the entire block is already in memory + as it does not need to interact with the I/O subsystem. + """ + + __slots__ = ["_contents", "_position", "_size"] + _contents: bytes + _position: int + _size: int + + def __init__(self, input_stream: InputStream) -> None: + """Reader is a Python object on which we can call read, seek, and tell.""" + super().__init__(input_stream) + self._contents = input_stream.read() + self._size = len(self._contents) + self._position = 0 + + def tell(self) -> int: + """Return the current stream position.""" + return self._position + + def read(self, n: int) -> bytes: + """Read n bytes.""" + if n < 0: + raise ValueError(f"Requested {n} bytes to read, expected positive integer.") + if self._position + n > self._size: + raise EOFError(f"EOF: read {n} bytes") + r = self._contents[self._position : self._position + n] + self._position += n + return r + + def skip(self, n: int) -> None: + self._position += n + + def read_boolean(self) -> bool: + """Reads a value from the stream as a boolean. + + A boolean is written as a single byte + whose value is either 0 (false) or 1 (true). + """ + r = self._contents[self._position] + self._position += 1 + return r != 0 + + def read_int(self) -> int: + """Reads a value from the stream as an integer. + + int/long values are written using variable-length, zigzag coding. + """ + if self._position == self._size: + raise EOFError("EOF: read 1 byte") + b = self._contents[self._position] + self._position += 1 + n = b & 0x7F + shift = 7 + while b & 0x80: + b = self._contents[self._position] + self._position += 1 + n |= (b & 0x7F) << shift + shift += 7 + return (n >> 1) ^ -(n & 1) + + def read_int_bytes_dict(self, n: int, dest: Dict[int, bytes]) -> None: + """Reads a dictionary of integers for keys and bytes for values into a destination dict.""" + for _ in range(n): + k = self.read_int() + + byte_length = self.read_int() + if byte_length <= 0: + dest[k] = b"" + else: + dest[k] = self._contents[self._position : self._position + byte_length] + self._position += byte_length + + def read_bytes(self) -> bytes: + """Bytes are encoded as a long followed by that many bytes of data.""" + num_bytes = self.read_int() + if num_bytes <= 0: + return b"" + r = self._contents[self._position : self._position + num_bytes] + self._position += num_bytes + return r + + def skip_int(self) -> None: + b = self._contents[self._position] + self._position += 1 + while b & 0x80: + b = self._contents[self._position] + self._position += 1 diff --git a/python/pyiceberg/avro/encoder.py b/python/pyiceberg/avro/encoder.py new file mode 100644 index 000000000000..cf6d60123357 --- /dev/null +++ b/python/pyiceberg/avro/encoder.py @@ -0,0 +1,175 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +import decimal +import struct +from datetime import date, datetime, time + +from pyiceberg.avro import STRUCT_DOUBLE, STRUCT_FLOAT +from pyiceberg.io import OutputStream +from pyiceberg.utils.datetime import date_to_days, datetime_to_micros, time_object_to_micros + + +class BinaryEncoder: + """Write leaf values.""" + + _output_stream: OutputStream + + def __init__(self, output_stream: OutputStream) -> None: + self._output_stream = output_stream + + def write(self, b: bytes) -> None: + self._output_stream.write(b) + + def write_boolean(self, boolean: bool) -> None: + """A boolean is written as a single byte whose value is either 0 (false) or 1 (true). + + Args: + boolean: The boolean to write. + """ + self.write(bytearray([bool(boolean)])) + + def write_int(self, integer: int) -> None: + """Integer and long values are written using variable-length zig-zag coding.""" + datum = (integer << 1) ^ (integer >> 63) + while (datum & ~0x7F) != 0: + self.write(bytearray([(datum & 0x7F) | 0x80])) + datum >>= 7 + self.write(bytearray([datum])) + + def write_float(self, f: float) -> None: + """A float is written as 4 bytes.""" + self.write(STRUCT_FLOAT.pack(f)) + + def write_double(self, f: float) -> None: + """A double is written as 8 bytes.""" + self.write(STRUCT_DOUBLE.pack(f)) + + def write_decimal_bytes(self, datum: decimal.Decimal) -> None: + """ + Decimal in bytes are encoded as long. + + Since size of packed value in bytes for signed long is 8, 8 bytes are written. + """ + sign, digits, _ = datum.as_tuple() + + unscaled_datum = 0 + for digit in digits: + unscaled_datum = (unscaled_datum * 10) + digit + + bits_req = unscaled_datum.bit_length() + 1 + if sign: + unscaled_datum = (1 << bits_req) - unscaled_datum + + bytes_req = bits_req // 8 + padding_bits = ~((1 << bits_req) - 1) if sign else 0 + packed_bits = padding_bits | unscaled_datum + + bytes_req += 1 if (bytes_req << 3) < bits_req else 0 + self.write_int(bytes_req) + for index in range(bytes_req - 1, -1, -1): + bits_to_write = packed_bits >> (8 * index) + self.write(bytearray([bits_to_write & 0xFF])) + + def write_decimal_fixed(self, datum: decimal.Decimal, size: int) -> None: + """Decimal in fixed are encoded as size of fixed bytes.""" + sign, digits, _ = datum.as_tuple() + + unscaled_datum = 0 + for digit in digits: + unscaled_datum = (unscaled_datum * 10) + digit + + bits_req = unscaled_datum.bit_length() + 1 + size_in_bits = size * 8 + offset_bits = size_in_bits - bits_req + + mask = 2**size_in_bits - 1 + bit = 1 + for _ in range(bits_req): + mask ^= bit + bit <<= 1 + + if bits_req < 8: + bytes_req = 1 + else: + bytes_req = bits_req // 8 + if bits_req % 8 != 0: + bytes_req += 1 + if sign: + unscaled_datum = (1 << bits_req) - unscaled_datum + unscaled_datum = mask | unscaled_datum + for index in range(size - 1, -1, -1): + bits_to_write = unscaled_datum >> (8 * index) + self.write(bytearray([bits_to_write & 0xFF])) + else: + for _ in range(offset_bits // 8): + self.write(b"\x00") + for index in range(bytes_req - 1, -1, -1): + bits_to_write = unscaled_datum >> (8 * index) + self.write(bytearray([bits_to_write & 0xFF])) + + def write_bytes(self, b: bytes) -> None: + """Bytes are encoded as a long followed by that many bytes of data.""" + self.write_int(len(b)) + self.write(struct.pack(f"{len(b)}s", b)) + + def write_bytes_fixed(self, b: bytes) -> None: + """Writes fixed number of bytes.""" + self.write(struct.pack(f"{len(b)}s", b)) + + def write_utf8(self, s: str) -> None: + """A string is encoded as a long followed by that many bytes of UTF-8 encoded character data.""" + self.write_bytes(s.encode("utf-8")) + + def write_date_int(self, d: date) -> None: + """ + Encode python date object as int. + + It stores the number of days from the unix epoch, 1 January 1970 (ISO calendar). + """ + self.write_int(date_to_days(d)) + + def write_time_millis_int(self, dt: time) -> None: + """ + Encode python time object as int. + + It stores the number of milliseconds from midnight, 00:00:00.000 + """ + self.write_int(int(time_object_to_micros(dt) / 1000)) + + def write_time_micros_long(self, dt: time) -> None: + """ + Encode python time object as long. + + It stores the number of microseconds from midnight, 00:00:00.000000 + """ + self.write_int(time_object_to_micros(dt)) + + def write_timestamp_millis_long(self, dt: datetime) -> None: + """ + Encode python datetime object as long. + + It stores the number of milliseconds from midnight of unix epoch, 1 January 1970. + """ + self.write_int(int(datetime_to_micros(dt) / 1000)) + + def write_timestamp_micros_long(self, dt: datetime) -> None: + """ + Encode python datetime object as long. + + It stores the number of microseconds from midnight of unix epoch, 1 January 1970. + """ + self.write_int(datetime_to_micros(dt)) diff --git a/python/pyiceberg/avro/file.py b/python/pyiceberg/avro/file.py index 3968cdc42e98..f780a8a30cb1 100644 --- a/python/pyiceberg/avro/file.py +++ b/python/pyiceberg/avro/file.py @@ -15,12 +15,12 @@ # specific language governing permissions and limitations # under the License. # pylint: disable=W0621 -""" -Avro reader for reading Avro files -""" +"""Avro reader for reading Avro files.""" from __future__ import annotations +import io import json +import os from dataclasses import dataclass from enum import Enum from types import TracebackType @@ -28,17 +28,19 @@ Callable, Dict, Generic, + List, Optional, Type, TypeVar, ) from pyiceberg.avro.codecs import KNOWN_CODECS, Codec -from pyiceberg.avro.decoder import BinaryDecoder +from pyiceberg.avro.decoder import BinaryDecoder, InMemoryBinaryDecoder +from pyiceberg.avro.encoder import BinaryEncoder from pyiceberg.avro.reader import Reader -from pyiceberg.avro.resolver import construct_reader, resolve -from pyiceberg.io import InputFile, InputStream -from pyiceberg.io.memory import MemoryInputStream +from pyiceberg.avro.resolver import construct_reader, construct_writer, resolve +from pyiceberg.avro.writer import Writer +from pyiceberg.io import InputFile, OutputFile, OutputStream from pyiceberg.schema import Schema from pyiceberg.typedef import EMPTY_DICT, Record, StructProtocol from pyiceberg.types import ( @@ -70,6 +72,7 @@ class AvroFileHeader(Record): + __slots__ = ("magic", "meta", "sync") magic: bytes meta: Dict[str, str] sync: bytes @@ -78,7 +81,7 @@ def compression_codec(self) -> Optional[Type[Codec]]: """Get the file's compression codec algorithm from the file's metadata. In the case of a null codec, we return a None indicating that we - don't need to compress/decompress + don't need to compress/decompress. """ codec_name = self.meta.get(_CODEC_KEY, "null") if codec_name not in KNOWN_CODECS: @@ -106,12 +109,14 @@ class Block(Generic[D]): position: int = 0 def __iter__(self) -> Block[D]: + """Returns an iterator for the Block class.""" return self def has_next(self) -> bool: return self.position < self.block_records def __next__(self) -> D: + """Returns the next item when iterating over the Block class.""" if self.has_next(): self.position += 1 return self.reader.read(self.block_decoder) @@ -119,17 +124,27 @@ def __next__(self) -> D: class AvroFile(Generic[D]): + __slots__ = ( + "input_file", + "read_schema", + "read_types", + "read_enums", + "header", + "schema", + "reader", + "decoder", + "block", + ) input_file: InputFile read_schema: Optional[Schema] read_types: Dict[int, Callable[..., StructProtocol]] read_enums: Dict[int, Callable[..., Enum]] - input_stream: InputStream header: AvroFileHeader schema: Schema reader: Reader decoder: BinaryDecoder - block: Optional[Block[D]] = None + block: Optional[Block[D]] def __init__( self, @@ -142,17 +157,16 @@ def __init__( self.read_schema = read_schema self.read_types = read_types self.read_enums = read_enums + self.block = None def __enter__(self) -> AvroFile[D]: - """ - Opens the file and reads the header and generates - a reader tree to start reading the payload + """Generates a reader tree for the payload within an avro file. Returns: - A generator returning the AvroStructs + A generator returning the AvroStructs. """ - self.input_stream = self.input_file.open(seekable=False) - self.decoder = BinaryDecoder(self.input_stream) + with self.input_file.open() as f: + self.decoder = InMemoryBinaryDecoder(io.BytesIO(f.read())) self.header = self._read_header() self.schema = self.header.get_schema() if not self.read_schema: @@ -165,9 +179,10 @@ def __enter__(self) -> AvroFile[D]: def __exit__( self, exctype: Optional[Type[BaseException]], excinst: Optional[BaseException], exctb: Optional[TracebackType] ) -> None: - self.input_stream.close() + """Performs cleanup when exiting the scope of a 'with' statement.""" def __iter__(self) -> AvroFile[D]: + """Returns an iterator for the AvroFile class.""" return self def _read_block(self) -> int: @@ -184,11 +199,12 @@ def _read_block(self) -> int: block_bytes = codec.decompress(block_bytes) self.block = Block( - reader=self.reader, block_records=block_records, block_decoder=BinaryDecoder(MemoryInputStream(block_bytes)) + reader=self.reader, block_records=block_records, block_decoder=InMemoryBinaryDecoder(io.BytesIO(block_bytes)) ) return block_records def __next__(self) -> D: + """Returns the next item when iterating over the AvroFile class.""" if self.block and self.block.has_next(): return next(self.block) @@ -203,3 +219,60 @@ def __next__(self) -> D: def _read_header(self) -> AvroFileHeader: return construct_reader(META_SCHEMA, {-1: AvroFileHeader}).read(self.decoder) + + +class AvroOutputFile(Generic[D]): + output_file: OutputFile + output_stream: OutputStream + schema: Schema + schema_name: str + encoder: BinaryEncoder + sync_bytes: bytes + writer: Writer + + def __init__(self, output_file: OutputFile, schema: Schema, schema_name: str, metadata: Dict[str, str] = EMPTY_DICT) -> None: + self.output_file = output_file + self.schema = schema + self.schema_name = schema_name + self.sync_bytes = os.urandom(SYNC_SIZE) + self.writer = construct_writer(self.schema) + self.metadata = metadata + + def __enter__(self) -> AvroOutputFile[D]: + """ + Opens the file and writes the header. + + Returns: + The file object to write records to + """ + self.output_stream = self.output_file.create(overwrite=True) + self.encoder = BinaryEncoder(self.output_stream) + + self._write_header() + self.writer = construct_writer(self.schema) + + return self + + def __exit__( + self, exctype: Optional[Type[BaseException]], excinst: Optional[BaseException], exctb: Optional[TracebackType] + ) -> None: + """Performs cleanup when exiting the scope of a 'with' statement.""" + self.output_stream.close() + + def _write_header(self) -> None: + json_schema = json.dumps(AvroSchemaConversion().iceberg_to_avro(self.schema, schema_name=self.schema_name)) + meta = {**self.metadata, _SCHEMA_KEY: json_schema, _CODEC_KEY: "null"} + header = AvroFileHeader(magic=MAGIC, meta=meta, sync=self.sync_bytes) + construct_writer(META_SCHEMA).write(self.encoder, header) + + def write_block(self, objects: List[D]) -> None: + in_memory = io.BytesIO() + block_content_encoder = BinaryEncoder(output_stream=in_memory) + for obj in objects: + self.writer.write(block_content_encoder, obj) + block_content = in_memory.getvalue() + + self.encoder.write_int(len(objects)) + self.encoder.write_int(len(block_content)) + self.encoder.write(block_content) + self.encoder.write_bytes_fixed(self.sync_bytes) diff --git a/python/pyiceberg/avro/reader.py b/python/pyiceberg/avro/reader.py index 2a94d0c069e0..ba9456e892d0 100644 --- a/python/pyiceberg/avro/reader.py +++ b/python/pyiceberg/avro/reader.py @@ -15,13 +15,13 @@ # specific language governing permissions and limitations # under the License. """ -Classes for building the Reader tree +Classes for building the Reader tree. Constructing a reader tree from the schema makes it easy to decouple the reader implementation from the schema. The reader tree can be changed in such a way that the -read schema is different, while respecting the read schema +read schema is different, while respecting the read schema. """ from __future__ import annotations @@ -47,7 +47,7 @@ def _skip_map_array(decoder: BinaryDecoder, skip_entry: Callable[[], None]) -> None: - """Skips over an array or map + """Skips over an array or map. Both the array and map are encoded similar, and we can re-use the logic of skipping in an efficient way. @@ -65,10 +65,10 @@ def _skip_map_array(decoder: BinaryDecoder, skip_entry: Callable[[], None]) -> N Args: decoder: - The decoder that reads the types from the underlying data + The decoder that reads the types from the underlying data. skip_entry: Function to skip over the underlying data, element in case of an array, and the - key/value in the case of a map + key/value in the case of a map. """ block_count = decoder.read_int() while block_count != 0: @@ -92,6 +92,7 @@ def skip(self, decoder: BinaryDecoder) -> None: ... def __repr__(self) -> str: + """Returns the string representation of the Reader class.""" return f"{self.__class__.__name__}()" @@ -104,6 +105,7 @@ def skip(self, decoder: BinaryDecoder) -> None: class DefaultReader(Reader): + __slots__ = ("default_value",) default_value: Any def __init__(self, default_value: Any) -> None: @@ -125,7 +127,7 @@ def skip(self, decoder: BinaryDecoder) -> None: class IntegerReader(Reader): - """Longs and ints are encoded the same way, and there is no long in Python""" + """Longs and ints are encoded the same way, and there is no long in Python.""" def read(self, decoder: BinaryDecoder) -> int: return decoder.read_int() @@ -209,9 +211,11 @@ def skip(self, decoder: BinaryDecoder) -> None: decoder.skip(len(self)) def __len__(self) -> int: + """Returns the length of an instance of the FixedReader class.""" return self._len def __repr__(self) -> str: + """Returns the string representation of the FixedReader class.""" return f"FixedReader({self._len})" @@ -235,6 +239,7 @@ def skip(self, decoder: BinaryDecoder) -> None: decoder.skip_bytes() def __repr__(self) -> str: + """Returns the string representation of the DecimalReader class.""" return f"DecimalReader({self.precision}, {self.scale})" @@ -262,9 +267,11 @@ def skip(self, decoder: BinaryDecoder) -> None: class StructReader(Reader): + __slots__ = ("field_readers", "create_struct", "struct", "_create_with_keyword", "_field_reader_functions", "_hash") field_readers: Tuple[Tuple[Optional[int], Reader], ...] create_struct: Callable[..., StructProtocol] struct: StructType + field_reader_functions = Tuple[Tuple[Optional[str], int, Optional[Callable[[BinaryDecoder], Any]]], ...] def __init__( self, @@ -276,24 +283,38 @@ def __init__( self.create_struct = create_struct self.struct = struct - def read(self, decoder: BinaryDecoder) -> StructProtocol: try: # Try initializing the struct, first with the struct keyword argument - struct = self.create_struct(struct=self.struct) + created_struct = self.create_struct(struct=self.struct) + self._create_with_keyword = True except TypeError as e: if "'struct' is an invalid keyword argument for" in str(e): - struct = self.create_struct() + created_struct = self.create_struct() + self._create_with_keyword = False else: raise ValueError(f"Unable to initialize struct: {self.create_struct}") from e - if not isinstance(struct, StructProtocol): + if not isinstance(created_struct, StructProtocol): raise ValueError(f"Incompatible with StructProtocol: {self.create_struct}") - for pos, field in self.field_readers: + reading_callbacks: List[Tuple[Optional[int], Callable[[BinaryDecoder], Any]]] = [] + for pos, field in field_readers: + if pos is not None: + reading_callbacks.append((pos, field.read)) + else: + reading_callbacks.append((None, field.skip)) + + self._field_reader_functions = tuple(reading_callbacks) + self._hash = hash(self._field_reader_functions) + + def read(self, decoder: BinaryDecoder) -> StructProtocol: + struct = self.create_struct(struct=self.struct) if self._create_with_keyword else self.create_struct() + + for pos, field_reader in self._field_reader_functions: if pos is not None: - struct[pos] = field.read(decoder) # later: pass reuse in here + struct[pos] = field_reader(decoder) # later: pass reuse in here else: - field.skip(decoder) + field_reader(decoder) return struct @@ -302,6 +323,7 @@ def skip(self, decoder: BinaryDecoder) -> None: field.skip(decoder) def __eq__(self, other: Any) -> bool: + """Returns the equality of two instances of the StructReader class.""" return ( self.field_readers == other.field_readers and self.create_struct == other.create_struct if isinstance(other, StructReader) @@ -309,49 +331,93 @@ def __eq__(self, other: Any) -> bool: ) def __repr__(self) -> str: + """Returns the string representation of the StructReader class.""" return f"StructReader(({','.join(repr(field) for field in self.field_readers)}), {repr(self.create_struct)})" def __hash__(self) -> int: - return hash(self.field_readers) + """Returns a hashed representation of the StructReader class.""" + return self._hash -@dataclass(frozen=True) +@dataclass(frozen=False, init=False) class ListReader(Reader): + __slots__ = ("element", "_is_int_list", "_hash") element: Reader + def __init__(self, element: Reader) -> None: + super().__init__() + self.element = element + self._hash = hash(self.element) + self._is_int_list = isinstance(self.element, IntegerReader) + def read(self, decoder: BinaryDecoder) -> List[Any]: - read_items = [] + read_items: List[Any] = [] block_count = decoder.read_int() while block_count != 0: if block_count < 0: block_count = -block_count _ = decoder.read_int() - for _ in range(block_count): - read_items.append(self.element.read(decoder)) + if self._is_int_list: + decoder.read_ints(block_count, read_items) + else: + for _ in range(block_count): + read_items.append(self.element.read(decoder)) block_count = decoder.read_int() return read_items def skip(self, decoder: BinaryDecoder) -> None: _skip_map_array(decoder, lambda: self.element.skip(decoder)) + def __hash__(self) -> int: + """Returns a hashed representation of the ListReader class.""" + return self._hash -@dataclass(frozen=True) + +@dataclass(frozen=False, init=False) class MapReader(Reader): + __slots__ = ("key", "value", "_is_int_int", "_is_int_bytes", "_key_reader", "_value_reader", "_hash") key: Reader value: Reader + def __init__(self, key: Reader, value: Reader) -> None: + super().__init__() + self.key = key + self.value = value + if isinstance(self.key, IntegerReader): + self._is_int_int = isinstance(self.value, IntegerReader) + self._is_int_bytes = isinstance(self.value, BinaryReader) + else: + self._is_int_int = False + self._is_int_bytes = False + self._key_reader = self.key.read + self._value_reader = self.value.read + self._hash = hash((self.key, self.value)) + def read(self, decoder: BinaryDecoder) -> Dict[Any, Any]: - read_items = {} + read_items: dict[Any, Any] = {} + block_count = decoder.read_int() - while block_count != 0: - if block_count < 0: - block_count = -block_count - # We ignore the block size for now - _ = decoder.read_int() - for _ in range(block_count): - key = self.key.read(decoder) - read_items[key] = self.value.read(decoder) - block_count = decoder.read_int() + if self._is_int_int or self._is_int_bytes: + while block_count != 0: + if block_count < 0: + block_count = -block_count + # We ignore the block size for now + _ = decoder.read_int() + if self._is_int_int: + decoder.read_int_int_dict(block_count, read_items) + else: + decoder.read_int_bytes_dict(block_count, read_items) + block_count = decoder.read_int() + else: + while block_count != 0: + if block_count < 0: + block_count = -block_count + # We ignore the block size for now + _ = decoder.read_int() + for _ in range(block_count): + key = self._key_reader(decoder) + read_items[key] = self._value_reader(decoder) + block_count = decoder.read_int() return read_items @@ -361,3 +427,7 @@ def skip() -> None: self.value.skip(decoder) _skip_map_array(decoder, skip) + + def __hash__(self) -> int: + """Returns a hashed representation of the MapReader class.""" + return self._hash diff --git a/python/pyiceberg/avro/resolver.py b/python/pyiceberg/avro/resolver.py index e72430c644f0..efd99936c572 100644 --- a/python/pyiceberg/avro/resolver.py +++ b/python/pyiceberg/avro/resolver.py @@ -48,12 +48,34 @@ TimestamptzReader, UUIDReader, ) +from pyiceberg.avro.writer import ( + BinaryWriter, + BooleanWriter, + DateWriter, + DecimalWriter, + DoubleWriter, + FixedWriter, + FloatWriter, + IntegerWriter, + ListWriter, + MapWriter, + OptionWriter, + StringWriter, + StructWriter, + TimestamptzWriter, + TimestampWriter, + TimeWriter, + UUIDWriter, + Writer, +) from pyiceberg.exceptions import ResolveError from pyiceberg.schema import ( PartnerAccessor, PrimitiveWithPartnerVisitor, Schema, + SchemaVisitorPerPrimitiveType, promote, + visit, visit_with_partner, ) from pyiceberg.typedef import EMPTY_DICT, Record, StructProtocol @@ -86,33 +108,106 @@ def construct_reader( file_schema: Union[Schema, IcebergType], read_types: Dict[int, Callable[..., StructProtocol]] = EMPTY_DICT ) -> Reader: - """Constructs a reader from a file schema + """Constructs a reader from a file schema. Args: - file_schema (Schema | IcebergType): The schema of the Avro file + file_schema (Schema | IcebergType): The schema of the Avro file. Raises: - NotImplementedError: If attempting to resolve an unrecognized object type + NotImplementedError: If attempting to resolve an unrecognized object type. """ return resolve(file_schema, file_schema, read_types) +def construct_writer(file_schema: Union[Schema, IcebergType]) -> Writer: + """Constructs a writer from a file schema. + + Args: + file_schema (Schema | IcebergType): The schema of the Avro file. + + Raises: + NotImplementedError: If attempting to resolve an unrecognized object type. + """ + return visit(file_schema, ConstructWriter()) + + +class ConstructWriter(SchemaVisitorPerPrimitiveType[Writer]): + """Constructs a writer tree from an Iceberg schema.""" + + def schema(self, schema: Schema, struct_result: Writer) -> Writer: + return struct_result + + def struct(self, struct: StructType, field_results: List[Writer]) -> Writer: + return StructWriter(tuple(field_results)) + + def field(self, field: NestedField, field_result: Writer) -> Writer: + return field_result if field.required else OptionWriter(field_result) + + def list(self, list_type: ListType, element_result: Writer) -> Writer: + return ListWriter(element_result) + + def map(self, map_type: MapType, key_result: Writer, value_result: Writer) -> Writer: + return MapWriter(key_result, value_result) + + def visit_fixed(self, fixed_type: FixedType) -> Writer: + return FixedWriter(len(fixed_type)) + + def visit_decimal(self, decimal_type: DecimalType) -> Writer: + return DecimalWriter(decimal_type.precision, decimal_type.scale) + + def visit_boolean(self, boolean_type: BooleanType) -> Writer: + return BooleanWriter() + + def visit_integer(self, integer_type: IntegerType) -> Writer: + return IntegerWriter() + + def visit_long(self, long_type: LongType) -> Writer: + return IntegerWriter() + + def visit_float(self, float_type: FloatType) -> Writer: + return FloatWriter() + + def visit_double(self, double_type: DoubleType) -> Writer: + return DoubleWriter() + + def visit_date(self, date_type: DateType) -> Writer: + return DateWriter() + + def visit_time(self, time_type: TimeType) -> Writer: + return TimeWriter() + + def visit_timestamp(self, timestamp_type: TimestampType) -> Writer: + return TimestampWriter() + + def visit_timestamptz(self, timestamptz_type: TimestamptzType) -> Writer: + return TimestamptzWriter() + + def visit_string(self, string_type: StringType) -> Writer: + return StringWriter() + + def visit_uuid(self, uuid_type: UUIDType) -> Writer: + return UUIDWriter() + + def visit_binary(self, binary_type: BinaryType) -> Writer: + return BinaryWriter() + + def resolve( file_schema: Union[Schema, IcebergType], read_schema: Union[Schema, IcebergType], read_types: Dict[int, Callable[..., StructProtocol]] = EMPTY_DICT, read_enums: Dict[int, Callable[..., Enum]] = EMPTY_DICT, ) -> Reader: - """Resolves the file and read schema to produce a reader + """Resolves the file and read schema to produce a reader. Args: - file_schema (Schema | IcebergType): The schema of the Avro file - read_schema (Schema | IcebergType): The requested read schema which is equal, subset or superset of the file schema - read_types (Dict[int, Callable[..., StructProtocol]]): A dict of types to use for struct data - read_enums (Dict[int, Callable[..., Enum]]): A dict of fields that have to be converted to an enum + file_schema (Schema | IcebergType): The schema of the Avro file. + read_schema (Schema | IcebergType): The requested read schema which is equal, subset or superset of the file schema. + read_types (Dict[int, Callable[..., StructProtocol]]): A dict of types to use for struct data. + read_enums (Dict[int, Callable[..., Enum]]): A dict of fields that have to be converted to an enum. Raises: - NotImplementedError: If attempting to resolve an unrecognized object type + NotImplementedError: If attempting to resolve an unrecognized object type. """ return visit_with_partner( file_schema, read_schema, SchemaResolver(read_types, read_enums), SchemaPartnerAccessor() @@ -120,7 +215,9 @@ def resolve( class EnumReader(Reader): - """An Enum reader to wrap primitive values into an Enum""" + """An Enum reader to wrap primitive values into an Enum.""" + + __slots__ = ("enum", "reader") enum: Callable[..., Enum] reader: Reader @@ -137,6 +234,7 @@ def skip(self, decoder: BinaryDecoder) -> None: class SchemaResolver(PrimitiveWithPartnerVisitor[IcebergType, Reader]): + __slots__ = ("read_types", "read_enums", "context") read_types: Dict[int, Callable[..., StructProtocol]] read_enums: Dict[int, Callable[..., Enum]] context: List[int] @@ -249,7 +347,7 @@ def visit_time(self, time_type: TimeType, partner: Optional[IcebergType]) -> Rea def visit_timestamp(self, timestamp_type: TimestampType, partner: Optional[IcebergType]) -> Reader: return TimestampReader() - def visit_timestampz(self, timestamptz_type: TimestamptzType, partner: Optional[IcebergType]) -> Reader: + def visit_timestamptz(self, timestamptz_type: TimestamptzType, partner: Optional[IcebergType]) -> Reader: return TimestamptzReader() def visit_string(self, string_type: StringType, partner: Optional[IcebergType]) -> Reader: diff --git a/python/pyiceberg/avro/writer.py b/python/pyiceberg/avro/writer.py new file mode 100644 index 000000000000..8ac44036e1ce --- /dev/null +++ b/python/pyiceberg/avro/writer.py @@ -0,0 +1,206 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +""" +Classes for building the Writer tree. + +Constructing a writer tree from the schema makes it easy +to decouple the writing implementation from the schema. +""" +from __future__ import annotations + +from abc import abstractmethod +from dataclasses import dataclass +from dataclasses import field as dataclassfield +from datetime import datetime, time +from typing import ( + Any, + Dict, + List, + Tuple, +) +from uuid import UUID + +from pyiceberg.avro.encoder import BinaryEncoder +from pyiceberg.types import StructType +from pyiceberg.utils.singleton import Singleton + + +class Writer(Singleton): + @abstractmethod + def write(self, encoder: BinaryEncoder, val: Any) -> Any: + ... + + def __repr__(self) -> str: + """Returns string representation of this object.""" + return f"{self.__class__.__name__}()" + + +class NoneWriter(Writer): + def write(self, _: BinaryEncoder, __: Any) -> None: + pass + + +class BooleanWriter(Writer): + def write(self, encoder: BinaryEncoder, val: bool) -> None: + encoder.write_boolean(val) + + +class IntegerWriter(Writer): + """Longs and ints are encoded the same way, and there is no long in Python.""" + + def write(self, encoder: BinaryEncoder, val: int) -> None: + encoder.write_int(val) + + +class FloatWriter(Writer): + def write(self, encoder: BinaryEncoder, val: float) -> None: + encoder.write_float(val) + + +class DoubleWriter(Writer): + def write(self, encoder: BinaryEncoder, val: float) -> None: + encoder.write_double(val) + + +class DateWriter(Writer): + def write(self, encoder: BinaryEncoder, val: Any) -> None: + encoder.write_date_int(val) + + +class TimeWriter(Writer): + def write(self, encoder: BinaryEncoder, val: time) -> None: + encoder.write_time_micros_long(val) + + +class TimestampWriter(Writer): + def write(self, encoder: BinaryEncoder, val: datetime) -> None: + encoder.write_timestamp_micros_long(val) + + +class TimestamptzWriter(Writer): + def write(self, encoder: BinaryEncoder, val: datetime) -> None: + encoder.write_timestamp_micros_long(val) + + +class StringWriter(Writer): + def write(self, encoder: BinaryEncoder, val: Any) -> None: + encoder.write_utf8(val) + + +class UUIDWriter(Writer): + def write(self, encoder: BinaryEncoder, val: UUID) -> None: + uuid_bytes = val.bytes + + if len(uuid_bytes) != 16: + raise ValueError(f"Expected UUID to be 16 bytes, got: {len(uuid_bytes)}") + + encoder.write_bytes_fixed(uuid_bytes) + + +@dataclass(frozen=True) +class FixedWriter(Writer): + _len: int = dataclassfield() + + def write(self, encoder: BinaryEncoder, val: bytes) -> None: + encoder.write(val) + + def __len__(self) -> int: + """Returns the length of this object.""" + return self._len + + def __repr__(self) -> str: + """Returns string representation of this object.""" + return f"FixedWriter({self._len})" + + +class BinaryWriter(Writer): + """Variable byte length writer.""" + + def write(self, encoder: BinaryEncoder, val: Any) -> None: + encoder.write_bytes(val) + + +@dataclass(frozen=True) +class DecimalWriter(Writer): + precision: int = dataclassfield() + scale: int = dataclassfield() + + def write(self, encoder: BinaryEncoder, val: Any) -> None: + return encoder.write_decimal_bytes(val) + + def __repr__(self) -> str: + """Returns string representation of this object.""" + return f"DecimalWriter({self.precision}, {self.scale})" + + +@dataclass(frozen=True) +class OptionWriter(Writer): + option: Writer = dataclassfield() + + def write(self, encoder: BinaryEncoder, val: Any) -> None: + if val is not None: + encoder.write_int(1) + self.option.write(encoder, val) + else: + encoder.write_int(0) + + +@dataclass(frozen=True) +class StructWriter(Writer): + field_writers: Tuple[Writer, ...] = dataclassfield() + + def write(self, encoder: BinaryEncoder, val: StructType) -> None: + for writer, value in zip(self.field_writers, val.record_fields()): + writer.write(encoder, value) + + def __eq__(self, other: Any) -> bool: + """Implements the equality operator for this object.""" + return self.field_writers == other.field_writers if isinstance(other, StructWriter) else False + + def __repr__(self) -> str: + """Returns string representation of this object.""" + return f"StructWriter({','.join(repr(field) for field in self.field_writers)})" + + def __hash__(self) -> int: + """Returns the hash of the writer as hash of this object.""" + return hash(self.field_writers) + + +@dataclass(frozen=True) +class ListWriter(Writer): + element_writer: Writer + + def write(self, encoder: BinaryEncoder, val: List[Any]) -> None: + encoder.write_int(len(val)) + for v in val: + self.element_writer.write(encoder, v) + if len(val) > 0: + encoder.write_int(0) + + +@dataclass(frozen=True) +class MapWriter(Writer): + key_writer: Writer + value_writer: Writer + + def write(self, encoder: BinaryEncoder, val: Dict[Any, Any]) -> None: + encoder.write_int(len(val)) + for k, v in val.items(): + self.key_writer.write(encoder, k) + self.value_writer.write(encoder, v) + if len(val) > 0: + encoder.write_int(0) diff --git a/python/pyiceberg/catalog/__init__.py b/python/pyiceberg/catalog/__init__.py index cd71d685b9a0..bc42e5ab0987 100644 --- a/python/pyiceberg/catalog/__init__.py +++ b/python/pyiceberg/catalog/__init__.py @@ -40,7 +40,12 @@ from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec from pyiceberg.schema import Schema from pyiceberg.serializers import ToOutputFile -from pyiceberg.table import Table, TableMetadata +from pyiceberg.table import ( + CommitTableRequest, + CommitTableResponse, + Table, + TableMetadata, +) from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder from pyiceberg.typedef import ( EMPTY_DICT, @@ -75,6 +80,7 @@ class CatalogType(Enum): HIVE = "hive" GLUE = "glue" DYNAMODB = "dynamodb" + SQL = "sql" def load_rest(name: str, conf: Properties) -> Catalog: @@ -110,26 +116,36 @@ def load_dynamodb(name: str, conf: Properties) -> Catalog: raise NotInstalledError("AWS DynamoDB support not installed: pip install 'pyiceberg[dynamodb]'") from exc +def load_sql(name: str, conf: Properties) -> Catalog: + try: + from pyiceberg.catalog.sql import SqlCatalog + + return SqlCatalog(name, **conf) + except ImportError as exc: + raise NotInstalledError("SQLAlchemy support not installed: pip install 'pyiceberg[sql-postgres]'") from exc + + AVAILABLE_CATALOGS: dict[CatalogType, Callable[[str, Properties], Catalog]] = { CatalogType.REST: load_rest, CatalogType.HIVE: load_hive, CatalogType.GLUE: load_glue, CatalogType.DYNAMODB: load_dynamodb, + CatalogType.SQL: load_sql, } def infer_catalog_type(name: str, catalog_properties: RecursiveDict) -> Optional[CatalogType]: - """Tries to infer the type based on the dict + """Tries to infer the type based on the dict. Args: - name: Name of the catalog - catalog_properties: Catalog properties + name: Name of the catalog. + catalog_properties: Catalog properties. Returns: - The inferred type based on the provided properties + The inferred type based on the provided properties. Raises: - ValueError: Raises a ValueError in case properties are missing, or the wrong type + ValueError: Raises a ValueError in case properties are missing, or the wrong type. """ if uri := catalog_properties.get("uri"): if isinstance(uri, str): @@ -137,6 +153,8 @@ def infer_catalog_type(name: str, catalog_properties: RecursiveDict) -> Optional return CatalogType.REST elif uri.startswith("thrift"): return CatalogType.HIVE + elif uri.startswith("postgresql"): + return CatalogType.SQL else: raise ValueError(f"Could not infer the catalog type from the uri: {uri}") else: @@ -147,22 +165,21 @@ def infer_catalog_type(name: str, catalog_properties: RecursiveDict) -> Optional def load_catalog(name: Optional[str], **properties: Optional[str]) -> Catalog: - """Load the catalog based on the properties + """Load the catalog based on the properties. - Will look up the properties from the config, based on the name + Will look up the properties from the config, based on the name. Args: - name: The name of the catalog - properties: The properties that are used next to the configuration + name: The name of the catalog. + properties: The properties that are used next to the configuration. Returns: - An initialized Catalog + An initialized Catalog. Raises: ValueError: Raises a ValueError in case properties are missing or malformed, - or if it could not determine the catalog based on the properties + or if it could not determine the catalog based on the properties. """ - if name is None: name = _ENV_CONFIG.get_default_catalog_name() @@ -187,12 +204,12 @@ def load_catalog(name: Optional[str], **properties: Optional[str]) -> Catalog: def delete_files(io: FileIO, files_to_delete: Set[str], file_type: str) -> None: """Helper to delete files. - Log warnings if failing to delete any file + Log warnings if failing to delete any file. Args: - io: The FileIO used to delete the object - files_to_delete: A set of file paths to be deleted - file_type: The type of the file + io: The FileIO used to delete the object. + files_to_delete: A set of file paths to be deleted. + file_type: The type of the file. """ for file in files_to_delete: try: @@ -204,15 +221,15 @@ def delete_files(io: FileIO, files_to_delete: Set[str], file_type: str) -> None: def delete_data_files(io: FileIO, manifests_to_delete: List[ManifestFile]) -> None: """Helper to delete data files linked to given manifests. - Log warnings if failing to delete any file + Log warnings if failing to delete any file. Args: - io: The FileIO used to delete the object - manifests_to_delete: A list of manifest contains paths of data files to be deleted + io: The FileIO used to delete the object. + manifests_to_delete: A list of manifest contains paths of data files to be deleted. """ deleted_files: dict[str, bool] = {} for manifest_file in manifests_to_delete: - for entry in manifest_file.fetch_manifest_entry(io): + for entry in manifest_file.fetch_manifest_entry(io, discard_deleted=False): path = entry.data_file.file_path if not deleted_files.get(path, False): try: @@ -239,8 +256,8 @@ class Catalog(ABC): or tuple of strings. Attributes: - name (str | None): Name of the catalog - properties (Properties): Catalog properties + name (str): Name of the catalog. + properties (Properties): Catalog properties. """ name: str @@ -263,7 +280,7 @@ def create_table( sort_order: SortOrder = UNSORTED_SORT_ORDER, properties: Properties = EMPTY_DICT, ) -> Table: - """Create a table + """Create a table. Args: identifier (str | Identifier): Table identifier. @@ -274,27 +291,27 @@ def create_table( properties (Properties): Table properties that can be a string based dictionary. Returns: - Table: the created table instance + Table: the created table instance. Raises: - TableAlreadyExistsError: If a table with the name already exists + TableAlreadyExistsError: If a table with the name already exists. """ @abstractmethod def load_table(self, identifier: Union[str, Identifier]) -> Table: """Loads the table's metadata and returns the table instance. - You can also use this method to check for table existence using 'try catalog.table() except NoSuchTableError' + You can also use this method to check for table existence using 'try catalog.table() except NoSuchTableError'. Note: This method doesn't scan data stored in the table. Args: identifier (str | Identifier): Table identifier. Returns: - Table: the table instance with its metadata + Table: the table instance with its metadata. Raises: - NoSuchTableError: If a table with the name does not exist + NoSuchTableError: If a table with the name does not exist. """ @abstractmethod @@ -305,22 +322,36 @@ def drop_table(self, identifier: Union[str, Identifier]) -> None: identifier (str | Identifier): Table identifier. Raises: - NoSuchTableError: If a table with the name does not exist + NoSuchTableError: If a table with the name does not exist. """ @abstractmethod def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table: - """Rename a fully classified table name + """Rename a fully classified table name. Args: from_identifier (str | Identifier): Existing table identifier. to_identifier (str | Identifier): New table identifier. Returns: - Table: the updated table instance with its metadata + Table: the updated table instance with its metadata. + + Raises: + NoSuchTableError: If a table with the name does not exist. + """ + + @abstractmethod + def _commit_table(self, table_request: CommitTableRequest) -> CommitTableResponse: + """Updates one or more tables. + + Args: + table_request (CommitTableRequest): The table requests to be carried out. + + Returns: + CommitTableResponse: The updated metadata. Raises: - NoSuchTableError: If a table with the name does not exist + NoSuchTableError: If a table with the given identifier does not exist. """ @abstractmethod @@ -328,11 +359,11 @@ def create_namespace(self, namespace: Union[str, Identifier], properties: Proper """Create a namespace in the catalog. Args: - namespace (str | Identifier): Namespace identifier - properties (Properties): A string dictionary of properties for the given namespace + namespace (str | Identifier): Namespace identifier. + properties (Properties): A string dictionary of properties for the given namespace. Raises: - NamespaceAlreadyExistsError: If a namespace with the given name already exists + NamespaceAlreadyExistsError: If a namespace with the given name already exists. """ @abstractmethod @@ -340,11 +371,11 @@ def drop_namespace(self, namespace: Union[str, Identifier]) -> None: """Drop a namespace. Args: - namespace (str | Identifier): Namespace identifier + namespace (str | Identifier): Namespace identifier. Raises: - NoSuchNamespaceError: If a namespace with the given name does not exist - NamespaceNotEmptyError: If the namespace is not empty + NoSuchNamespaceError: If a namespace with the given name does not exist. + NamespaceNotEmptyError: If the namespace is not empty. """ @abstractmethod @@ -360,7 +391,7 @@ def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]: List[Identifier]: list of table identifiers. Raises: - NoSuchNamespaceError: If a namespace with the given name does not exist + NoSuchNamespaceError: If a namespace with the given name does not exist. """ @abstractmethod @@ -371,10 +402,10 @@ def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identi namespace (str | Identifier): Namespace identifier to search. Returns: - List[Identifier]: a List of namespace identifiers + List[Identifier]: a List of namespace identifiers. Raises: - NoSuchNamespaceError: If a namespace with the given name does not exist + NoSuchNamespaceError: If a namespace with the given name does not exist. """ @abstractmethod @@ -382,28 +413,28 @@ def load_namespace_properties(self, namespace: Union[str, Identifier]) -> Proper """Get properties for a namespace. Args: - namespace (str | Identifier): Namespace identifier + namespace (str | Identifier): Namespace identifier. Returns: - Properties: Properties for the given namespace + Properties: Properties for the given namespace. Raises: - NoSuchNamespaceError: If a namespace with the given name does not exist + NoSuchNamespaceError: If a namespace with the given name does not exist. """ @abstractmethod def update_namespace_properties( - self, namespace: Union[str, Identifier], removals: set[str] | None = None, updates: Properties = EMPTY_DICT + self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Properties = EMPTY_DICT ) -> PropertiesUpdateSummary: """Removes provided property keys and updates properties for a namespace. Args: - namespace (str | Identifier): Namespace identifier + namespace (str | Identifier): Namespace identifier. removals (Set[str]): Set of property keys that need to be removed. Optional Argument. updates (Properties): Properties to be updated for the given namespace. Raises: - NoSuchNamespaceError: If a namespace with the given name does not exist + NoSuchNamespaceError: If a namespace with the given name does not exist. ValueError: If removals and updates have overlapping keys. """ @@ -414,34 +445,34 @@ def identifier_to_tuple(identifier: Union[str, Identifier]) -> Identifier: If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is. Args: - identifier (str | Identifier: an identifier, either a string or tuple of strings + identifier (str | Identifier: an identifier, either a string or tuple of strings. Returns: - Identifier: a tuple of strings + Identifier: a tuple of strings. """ return identifier if isinstance(identifier, tuple) else tuple(str.split(identifier, ".")) @staticmethod def table_name_from(identifier: Union[str, Identifier]) -> str: - """Extracts table name from a table identifier + """Extracts table name from a table identifier. Args: - identifier (str | Identifier: a table identifier + identifier (str | Identifier: a table identifier. Returns: - str: Table name + str: Table name. """ return Catalog.identifier_to_tuple(identifier)[-1] @staticmethod def namespace_from(identifier: Union[str, Identifier]) -> Identifier: - """Extracts table namespace from a table identifier + """Extracts table namespace from a table identifier. Args: - identifier (str | Identifier: a table identifier + identifier (Union[str, Identifier]): a table identifier. Returns: - Identifier: Namespace identifier + Identifier: Namespace identifier. """ return Catalog.identifier_to_tuple(identifier)[:-1] @@ -493,13 +524,13 @@ def identifier_to_database_and_table( def purge_table(self, identifier: Union[str, Identifier]) -> None: """Drop a table and purge all data and metadata files. - Note: This method only logs warning rather than raise exception when encountering file deletion failure + Note: This method only logs warning rather than raise exception when encountering file deletion failure. Args: identifier (str | Identifier): Table identifier. Raises: - NoSuchTableError: If a table with the name does not exist, or the identifier is invalid + NoSuchTableError: If a table with the name does not exist, or the identifier is invalid. """ table = self.load_table(identifier) self.drop_table(identifier) diff --git a/python/pyiceberg/catalog/dynamodb.py b/python/pyiceberg/catalog/dynamodb.py index 50f3b07571f4..d11ffb5d3941 100644 --- a/python/pyiceberg/catalog/dynamodb.py +++ b/python/pyiceberg/catalog/dynamodb.py @@ -52,7 +52,7 @@ from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec from pyiceberg.schema import Schema from pyiceberg.serializers import FromInputFile -from pyiceberg.table import Table +from pyiceberg.table import CommitTableRequest, CommitTableResponse, Table from pyiceberg.table.metadata import new_table_metadata from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder from pyiceberg.typedef import EMPTY_DICT @@ -126,7 +126,7 @@ def create_table( properties: Properties = EMPTY_DICT, ) -> Table: """ - Create an Iceberg table + Create an Iceberg table. Args: identifier: Table identifier. @@ -137,11 +137,11 @@ def create_table( properties: Table properties that can be a string based dictionary. Returns: - Table: the created table instance + Table: the created table instance. Raises: - AlreadyExistsError: If a table with the name already exists - ValueError: If the identifier is invalid, or no path is given to store metadata + AlreadyExistsError: If a table with the name already exists. + ValueError: If the identifier is invalid, or no path is given to store metadata. """ database_name, table_name = self.identifier_to_database_and_table(identifier) @@ -168,21 +168,35 @@ def create_table( return self.load_table(identifier=identifier) + def _commit_table(self, table_request: CommitTableRequest) -> CommitTableResponse: + """Updates the table. + + Args: + table_request (CommitTableRequest): The table requests to be carried out. + + Returns: + CommitTableResponse: The updated metadata. + + Raises: + NoSuchTableError: If a table with the given identifier does not exist. + """ + raise NotImplementedError + def load_table(self, identifier: Union[str, Identifier]) -> Table: """ Loads the table's metadata and returns the table instance. - You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError' + You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'. Note: This method doesn't scan data stored in the table. Args: identifier: Table identifier. Returns: - Table: the table instance with its metadata + Table: the table instance with its metadata. Raises: - NoSuchTableError: If a table with the name does not exist, or the identifier is invalid + NoSuchTableError: If a table with the name does not exist, or the identifier is invalid. """ database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError) dynamo_table_item = self._get_iceberg_table_item(database_name=database_name, table_name=table_name) @@ -195,7 +209,7 @@ def drop_table(self, identifier: Union[str, Identifier]) -> None: identifier: Table identifier. Raises: - NoSuchTableError: If a table with the name does not exist, or the identifier is invalid + NoSuchTableError: If a table with the name does not exist, or the identifier is invalid. """ database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError) @@ -209,23 +223,23 @@ def drop_table(self, identifier: Union[str, Identifier]) -> None: raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table: - """Rename a fully classified table name + """Rename a fully classified table name. - This method can only rename Iceberg tables in AWS Glue + This method can only rename Iceberg tables in AWS Glue. Args: from_identifier: Existing table identifier. to_identifier: New table identifier. Returns: - Table: the updated table instance with its metadata + Table: the updated table instance with its metadata. Raises: - ValueError: When from table identifier is invalid - NoSuchTableError: When a table with the name does not exist - NoSuchIcebergTableError: When from table is not a valid iceberg table - NoSuchPropertyException: When from table miss some required properties - NoSuchNamespaceError: When the destination namespace doesn't exist + ValueError: When from table identifier is invalid. + NoSuchTableError: When a table with the name does not exist. + NoSuchIcebergTableError: When from table is not a valid iceberg table. + NoSuchPropertyException: When from table miss some required properties. + NoSuchNamespaceError: When the destination namespace doesn't exist. """ from_database_name, from_table_name = self.identifier_to_database_and_table(from_identifier, NoSuchTableError) to_database_name, to_table_name = self.identifier_to_database_and_table(to_identifier) @@ -278,12 +292,12 @@ def create_namespace(self, namespace: Union[str, Identifier], properties: Proper """Create a namespace in the catalog. Args: - namespace: Namespace identifier - properties: A string dictionary of properties for the given namespace + namespace: Namespace identifier. + properties: A string dictionary of properties for the given namespace. Raises: - ValueError: If the identifier is invalid - AlreadyExistsError: If a namespace with the given name already exists + ValueError: If the identifier is invalid. + AlreadyExistsError: If a namespace with the given name already exists. """ database_name = self.identifier_to_database(namespace) @@ -298,14 +312,14 @@ def create_namespace(self, namespace: Union[str, Identifier], properties: Proper def drop_namespace(self, namespace: Union[str, Identifier]) -> None: """Drop a namespace. - A Glue namespace can only be dropped if it is empty + A Glue namespace can only be dropped if it is empty. Args: - namespace: Namespace identifier + namespace: Namespace identifier. Raises: - NoSuchNamespaceError: If a namespace with the given name does not exist, or the identifier is invalid - NamespaceNotEmptyError: If the namespace is not empty + NoSuchNamespaceError: If a namespace with the given name does not exist, or the identifier is invalid. + NamespaceNotEmptyError: If the namespace is not empty. """ database_name = self.identifier_to_database(namespace, NoSuchNamespaceError) table_identifiers = self.list_tables(namespace=database_name) @@ -323,7 +337,7 @@ def drop_namespace(self, namespace: Union[str, Identifier]) -> None: raise NoSuchNamespaceError(f"Database does not exist: {database_name}") from e def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]: - """List tables under the given namespace in the catalog (including non-Iceberg tables) + """List tables under the given namespace in the catalog (including non-Iceberg tables). Args: namespace (str | Identifier): Namespace identifier to search. @@ -331,7 +345,6 @@ def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]: Returns: List[Identifier]: list of table identifiers. """ - database_name = self.identifier_to_database(namespace, NoSuchNamespaceError) paginator = self.dynamodb.get_paginator("query") @@ -368,14 +381,13 @@ def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]: return table_identifiers def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identifier]: - """ - List top-level namespaces from the catalog. + """List top-level namespaces from the catalog. + We do not support hierarchical namespace. Returns: - List[Identifier]: a List of namespace identifiers + List[Identifier]: a List of namespace identifiers. """ - # Hierarchical namespace is not supported. Return an empty list if namespace: return [] @@ -415,13 +427,13 @@ def load_namespace_properties(self, namespace: Union[str, Identifier]) -> Proper Get properties for a namespace. Args: - namespace: Namespace identifier + namespace: Namespace identifier. Returns: - Properties: Properties for the given namespace + Properties: Properties for the given namespace. Raises: - NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid + NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid. """ database_name = self.identifier_to_database(namespace, NoSuchNamespaceError) namespace_item = self._get_iceberg_namespace_item(database_name=database_name) @@ -440,10 +452,9 @@ def update_namespace_properties( updates: Properties to be updated for the given namespace. Raises: - NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid + NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid. ValueError: If removals and updates have overlapping keys. """ - database_name = self.identifier_to_database(namespace, NoSuchNamespaceError) namespace_item = self._get_iceberg_namespace_item(database_name=database_name) namespace_dict = _convert_dynamo_item_to_regular_dict(namespace_item) @@ -580,6 +591,7 @@ def _convert_dynamo_table_item_to_iceberg_table(self, dynamo_table_item: Dict[st metadata=metadata, metadata_location=metadata_location, io=self._load_file_io(metadata.properties, metadata_location), + catalog=self, ) @@ -718,8 +730,9 @@ def _get_namespace_properties(namespace_dict: Dict[str, str]) -> Properties: def _convert_dynamo_item_to_regular_dict(dynamo_json: Dict[str, Any]) -> Dict[str, str]: - """ - Converts a dynamo json to a regular json. Example of a dynamo json: + """Converts a dynamo json to a regular json. + + Example of a dynamo json: { "AlbumTitle": { "S": "Songs About Life", @@ -741,7 +754,6 @@ def _convert_dynamo_item_to_regular_dict(dynamo_json: Dict[str, Any]) -> Dict[st Only "S" and "N" data types are supported since those are the only ones that Iceberg is utilizing. """ - regular_json = {} for column_name, val_dict in dynamo_json.items(): keys = list(val_dict.keys()) diff --git a/python/pyiceberg/catalog/glue.py b/python/pyiceberg/catalog/glue.py index bb83eababb1a..13ee7f81d640 100644 --- a/python/pyiceberg/catalog/glue.py +++ b/python/pyiceberg/catalog/glue.py @@ -51,13 +51,16 @@ from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec from pyiceberg.schema import Schema from pyiceberg.serializers import FromInputFile -from pyiceberg.table import Table +from pyiceberg.table import CommitTableRequest, CommitTableResponse, Table from pyiceberg.table.metadata import new_table_metadata from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder from pyiceberg.typedef import EMPTY_DICT +BOTO_SESSION_CONFIG_KEYS = ["aws_secret_key_id", "aws_secret_access_key", "aws_session_token", "region_name", "profile_name"] + GLUE_CLIENT = "glue" + PROP_GLUE_TABLE = "Table" PROP_GLUE_TABLE_TYPE = "TableType" PROP_GLUE_TABLE_DESCRIPTION = "Description" @@ -131,7 +134,10 @@ def _construct_database_input(database_name: str, properties: Properties) -> Dic class GlueCatalog(Catalog): def __init__(self, name: str, **properties: str): super().__init__(name, **properties) - self.glue = boto3.client(GLUE_CLIENT) + + session_config = {k: v for k, v in properties.items() if k in BOTO_SESSION_CONFIG_KEYS} + session = boto3.Session(**session_config) + self.glue = session.client(GLUE_CLIENT) def _convert_glue_to_iceberg(self, glue_table: Dict[str, Any]) -> Table: properties: Properties = glue_table.get(PROP_GLUE_TABLE_PARAMETERS, {}) @@ -160,10 +166,11 @@ def _convert_glue_to_iceberg(self, glue_table: Dict[str, Any]) -> Table: file = io.new_input(metadata_location) metadata = FromInputFile.table_metadata(file) return Table( - identifier=(glue_table[PROP_GLUE_TABLE_DATABASE_NAME], glue_table[PROP_GLUE_TABLE_NAME]), + identifier=(self.name, glue_table[PROP_GLUE_TABLE_DATABASE_NAME], glue_table[PROP_GLUE_TABLE_NAME]), metadata=metadata, metadata_location=metadata_location, io=self._load_file_io(metadata.properties, metadata_location), + catalog=self, ) def _create_glue_table(self, database_name: str, table_name: str, table_input: Dict[str, Any]) -> None: @@ -184,7 +191,7 @@ def create_table( properties: Properties = EMPTY_DICT, ) -> Table: """ - Create an Iceberg table + Create an Iceberg table. Args: identifier: Table identifier. @@ -195,11 +202,11 @@ def create_table( properties: Table properties that can be a string based dictionary. Returns: - Table: the created table instance + Table: the created table instance. Raises: - AlreadyExistsError: If a table with the name already exists - ValueError: If the identifier is invalid, or no path is given to store metadata + AlreadyExistsError: If a table with the name already exists. + ValueError: If the identifier is invalid, or no path is given to store metadata. """ database_name, table_name = self.identifier_to_database_and_table(identifier) @@ -218,20 +225,34 @@ def create_table( return self.load_table(identifier=identifier) + def _commit_table(self, table_request: CommitTableRequest) -> CommitTableResponse: + """Updates the table. + + Args: + table_request (CommitTableRequest): The table requests to be carried out. + + Returns: + CommitTableResponse: The updated metadata. + + Raises: + NoSuchTableError: If a table with the given identifier does not exist. + """ + raise NotImplementedError + def load_table(self, identifier: Union[str, Identifier]) -> Table: """Loads the table's metadata and returns the table instance. - You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError' + You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'. Note: This method doesn't scan data stored in the table. Args: identifier: Table identifier. Returns: - Table: the table instance with its metadata + Table: the table instance with its metadata. Raises: - NoSuchTableError: If a table with the name does not exist, or the identifier is invalid + NoSuchTableError: If a table with the name does not exist, or the identifier is invalid. """ database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError) try: @@ -248,7 +269,7 @@ def drop_table(self, identifier: Union[str, Identifier]) -> None: identifier: Table identifier. Raises: - NoSuchTableError: If a table with the name does not exist, or the identifier is invalid + NoSuchTableError: If a table with the name does not exist, or the identifier is invalid. """ database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError) try: @@ -257,23 +278,23 @@ def drop_table(self, identifier: Union[str, Identifier]) -> None: raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table: - """Rename a fully classified table name + """Rename a fully classified table name. - This method can only rename Iceberg tables in AWS Glue + This method can only rename Iceberg tables in AWS Glue. Args: from_identifier: Existing table identifier. to_identifier: New table identifier. Returns: - Table: the updated table instance with its metadata + Table: the updated table instance with its metadata. Raises: - ValueError: When from table identifier is invalid - NoSuchTableError: When a table with the name does not exist - NoSuchIcebergTableError: When from table is not a valid iceberg table - NoSuchPropertyException: When from table miss some required properties - NoSuchNamespaceError: When the destination namespace doesn't exist + ValueError: When from table identifier is invalid. + NoSuchTableError: When a table with the name does not exist. + NoSuchIcebergTableError: When from table is not a valid iceberg table. + NoSuchPropertyException: When from table miss some required properties. + NoSuchNamespaceError: When the destination namespace doesn't exist. """ from_database_name, from_table_name = self.identifier_to_database_and_table(from_identifier, NoSuchTableError) to_database_name, to_table_name = self.identifier_to_database_and_table(to_identifier) @@ -320,12 +341,12 @@ def create_namespace(self, namespace: Union[str, Identifier], properties: Proper """Create a namespace in the catalog. Args: - namespace: Namespace identifier - properties: A string dictionary of properties for the given namespace + namespace: Namespace identifier. + properties: A string dictionary of properties for the given namespace. Raises: - ValueError: If the identifier is invalid - AlreadyExistsError: If a namespace with the given name already exists + ValueError: If the identifier is invalid. + AlreadyExistsError: If a namespace with the given name already exists. """ database_name = self.identifier_to_database(namespace) try: @@ -336,14 +357,14 @@ def create_namespace(self, namespace: Union[str, Identifier], properties: Proper def drop_namespace(self, namespace: Union[str, Identifier]) -> None: """Drop a namespace. - A Glue namespace can only be dropped if it is empty + A Glue namespace can only be dropped if it is empty. Args: - namespace: Namespace identifier + namespace: Namespace identifier. Raises: - NoSuchNamespaceError: If a namespace with the given name does not exist, or the identifier is invalid - NamespaceNotEmptyError: If the namespace is not empty + NoSuchNamespaceError: If a namespace with the given name does not exist, or the identifier is invalid. + NamespaceNotEmptyError: If the namespace is not empty. """ database_name = self.identifier_to_database(namespace, NoSuchNamespaceError) try: @@ -357,7 +378,7 @@ def drop_namespace(self, namespace: Union[str, Identifier]) -> None: self.glue.delete_database(Name=database_name) def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]: - """List tables under the given namespace in the catalog (including non-Iceberg tables) + """List tables under the given namespace in the catalog (including non-Iceberg tables). Args: namespace (str | Identifier): Namespace identifier to search. @@ -366,9 +387,8 @@ def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]: List[Identifier]: list of table identifiers. Raises: - NoSuchNamespaceError: If a namespace with the given name does not exist, or the identifier is invalid + NoSuchNamespaceError: If a namespace with the given name does not exist, or the identifier is invalid. """ - database_name = self.identifier_to_database(namespace, NoSuchNamespaceError) table_list = [] try: @@ -387,7 +407,7 @@ def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identi """List namespaces from the given namespace. If not given, list top-level namespaces from the catalog. Returns: - List[Identifier]: a List of namespace identifiers + List[Identifier]: a List of namespace identifiers. """ # Hierarchical namespace is not supported. Return an empty list if namespace: @@ -407,13 +427,13 @@ def load_namespace_properties(self, namespace: Union[str, Identifier]) -> Proper """Get properties for a namespace. Args: - namespace: Namespace identifier + namespace: Namespace identifier. Returns: - Properties: Properties for the given namespace + Properties: Properties for the given namespace. Raises: - NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid + NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid. """ database_name = self.identifier_to_database(namespace, NoSuchNamespaceError) try: @@ -424,10 +444,8 @@ def load_namespace_properties(self, namespace: Union[str, Identifier]) -> Proper raise NoSuchNamespaceError(f"Invalid input for namespace {database_name}") from e database = database_response[PROP_GLUE_DATABASE] - if PROP_GLUE_DATABASE_PARAMETERS not in database: - return {} - properties = dict(database[PROP_GLUE_DATABASE_PARAMETERS]) + properties = dict(database.get(PROP_GLUE_DATABASE_PARAMETERS, {})) if database_location := database.get(PROP_GLUE_DATABASE_LOCATION): properties[LOCATION] = database_location if database_description := database.get(PROP_GLUE_DATABASE_DESCRIPTION): @@ -441,15 +459,14 @@ def update_namespace_properties( """Removes provided property keys and updates properties for a namespace. Args: - namespace: Namespace identifier + namespace: Namespace identifier. removals: Set of property keys that need to be removed. Optional Argument. updates: Properties to be updated for the given namespace. Raises: - NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid + NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid. ValueError: If removals and updates have overlapping keys. """ - current_properties = self.load_namespace_properties(namespace=namespace) properties_update_summary, updated_properties = self._get_updated_props_and_update_summary( current_properties=current_properties, removals=removals, updates=updates diff --git a/python/pyiceberg/catalog/hive.py b/python/pyiceberg/catalog/hive.py index 3baf55864dbc..08655676de49 100644 --- a/python/pyiceberg/catalog/hive.py +++ b/python/pyiceberg/catalog/hive.py @@ -66,7 +66,7 @@ from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec from pyiceberg.schema import Schema, SchemaVisitor, visit from pyiceberg.serializers import FromInputFile -from pyiceberg.table import Table +from pyiceberg.table import CommitTableRequest, CommitTableResponse, Table from pyiceberg.table.metadata import new_table_metadata from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder from pyiceberg.typedef import EMPTY_DICT @@ -112,7 +112,7 @@ class _HiveClient: - """Helper class to nicely open and close the transport""" + """Helper class to nicely open and close the transport.""" _transport: TTransport _client: Client @@ -239,10 +239,11 @@ def _convert_hive_into_iceberg(self, table: HiveTable, io: FileIO) -> Table: file = io.new_input(metadata_location) metadata = FromInputFile.table_metadata(file) return Table( - identifier=(table.dbName, table.tableName), + identifier=(self.name, table.dbName, table.tableName), metadata=metadata, metadata_location=metadata_location, io=self._load_file_io(metadata.properties, metadata_location), + catalog=self, ) def create_table( @@ -254,7 +255,7 @@ def create_table( sort_order: SortOrder = UNSORTED_SORT_ORDER, properties: Properties = EMPTY_DICT, ) -> Table: - """Create a table + """Create a table. Args: identifier: Table identifier. @@ -265,11 +266,11 @@ def create_table( properties: Table properties that can be a string based dictionary. Returns: - Table: the created table instance + Table: the created table instance. Raises: - AlreadyExistsError: If a table with the name already exists - ValueError: If the identifier is invalid + AlreadyExistsError: If a table with the name already exists. + ValueError: If the identifier is invalid. """ database_name, table_name = self.identifier_to_database_and_table(identifier) current_time_millis = int(time.time() * 1000) @@ -302,20 +303,34 @@ def create_table( return self._convert_hive_into_iceberg(hive_table, io) + def _commit_table(self, table_request: CommitTableRequest) -> CommitTableResponse: + """Updates the table. + + Args: + table_request (CommitTableRequest): The table requests to be carried out. + + Returns: + CommitTableResponse: The updated metadata. + + Raises: + NoSuchTableError: If a table with the given identifier does not exist. + """ + raise NotImplementedError + def load_table(self, identifier: Union[str, Identifier]) -> Table: """Loads the table's metadata and returns the table instance. - You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError' + You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'. Note: This method doesn't scan data stored in the table. Args: identifier: Table identifier. Returns: - Table: the table instance with its metadata + Table: the table instance with its metadata. Raises: - NoSuchTableError: If a table with the name does not exist, or the identifier is invalid + NoSuchTableError: If a table with the name does not exist, or the identifier is invalid. """ database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError) try: @@ -334,7 +349,7 @@ def drop_table(self, identifier: Union[str, Identifier]) -> None: identifier: Table identifier. Raises: - NoSuchTableError: If a table with the name does not exist, or the identifier is invalid + NoSuchTableError: If a table with the name does not exist, or the identifier is invalid. """ database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError) try: @@ -349,19 +364,19 @@ def purge_table(self, identifier: Union[str, Identifier]) -> None: raise NotImplementedError("Not yet implemented") def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table: - """Rename a fully classified table name + """Rename a fully classified table name. Args: from_identifier: Existing table identifier. to_identifier: New table identifier. Returns: - Table: the updated table instance with its metadata + Table: the updated table instance with its metadata. Raises: - ValueError: When from table identifier is invalid - NoSuchTableError: When a table with the name does not exist - NoSuchNamespaceError: When the destination namespace doesn't exist + ValueError: When from table identifier is invalid. + NoSuchTableError: When a table with the name does not exist. + NoSuchNamespaceError: When the destination namespace doesn't exist. """ from_database_name, from_table_name = self.identifier_to_database_and_table(from_identifier, NoSuchTableError) to_database_name, to_table_name = self.identifier_to_database_and_table(to_identifier) @@ -381,12 +396,12 @@ def create_namespace(self, namespace: Union[str, Identifier], properties: Proper """Create a namespace in the catalog. Args: - namespace: Namespace identifier - properties: A string dictionary of properties for the given namespace + namespace: Namespace identifier. + properties: A string dictionary of properties for the given namespace. Raises: - ValueError: If the identifier is invalid - AlreadyExistsError: If a namespace with the given name already exists + ValueError: If the identifier is invalid. + AlreadyExistsError: If a namespace with the given name already exists. """ database_name = self.identifier_to_database(namespace) hive_database = HiveDatabase(name=database_name, parameters=properties) @@ -401,11 +416,11 @@ def drop_namespace(self, namespace: Union[str, Identifier]) -> None: """Drop a namespace. Args: - namespace: Namespace identifier + namespace: Namespace identifier. Raises: - NoSuchNamespaceError: If a namespace with the given name does not exist, or the identifier is invalid - NamespaceNotEmptyError: If the namespace is not empty + NoSuchNamespaceError: If a namespace with the given name does not exist, or the identifier is invalid. + NamespaceNotEmptyError: If the namespace is not empty. """ database_name = self.identifier_to_database(namespace, NoSuchNamespaceError) try: @@ -417,9 +432,9 @@ def drop_namespace(self, namespace: Union[str, Identifier]) -> None: raise NoSuchNamespaceError(f"Database does not exists: {database_name}") from e def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]: - """List tables under the given namespace in the catalog (including non-Iceberg tables) + """List tables under the given namespace in the catalog (including non-Iceberg tables). - When the database doesn't exist, it will just return an empty list + When the database doesn't exist, it will just return an empty list. Args: namespace: Database to list. @@ -428,7 +443,7 @@ def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]: List[Identifier]: list of table identifiers. Raises: - NoSuchNamespaceError: If a namespace with the given name does not exist, or the identifier is invalid + NoSuchNamespaceError: If a namespace with the given name does not exist, or the identifier is invalid. """ database_name = self.identifier_to_database(namespace, NoSuchNamespaceError) with self._client as open_client: @@ -438,7 +453,7 @@ def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identi """List namespaces from the given namespace. If not given, list top-level namespaces from the catalog. Returns: - List[Identifier]: a List of namespace identifiers + List[Identifier]: a List of namespace identifiers. """ # Hierarchical namespace is not supported. Return an empty list if namespace: @@ -451,13 +466,13 @@ def load_namespace_properties(self, namespace: Union[str, Identifier]) -> Proper """Get properties for a namespace. Args: - namespace: Namespace identifier + namespace: Namespace identifier. Returns: - Properties: Properties for the given namespace + Properties: Properties for the given namespace. Raises: - NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid + NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid. """ database_name = self.identifier_to_database(namespace, NoSuchNamespaceError) try: @@ -477,7 +492,7 @@ def update_namespace_properties( """Removes provided property keys and updates properties for a namespace. Args: - namespace: Namespace identifier + namespace: Namespace identifier. removals: Set of property keys that need to be removed. Optional Argument. updates: Properties to be updated for the given namespace. @@ -485,7 +500,6 @@ def update_namespace_properties( NoSuchNamespaceError: If a namespace with the given name does not exist ValueError: If removals and updates have overlapping keys. """ - self._check_for_overlap(updates=updates, removals=removals) database_name = self.identifier_to_database(namespace, NoSuchNamespaceError) with self._client as open_client: diff --git a/python/pyiceberg/catalog/noop.py b/python/pyiceberg/catalog/noop.py new file mode 100644 index 000000000000..bb93772aa759 --- /dev/null +++ b/python/pyiceberg/catalog/noop.py @@ -0,0 +1,79 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +from typing import ( + List, + Optional, + Set, + Union, +) + +from pyiceberg.catalog import Catalog, PropertiesUpdateSummary +from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec +from pyiceberg.schema import Schema +from pyiceberg.table import ( + CommitTableRequest, + CommitTableResponse, + SortOrder, + Table, +) +from pyiceberg.table.sorting import UNSORTED_SORT_ORDER +from pyiceberg.typedef import EMPTY_DICT, Identifier, Properties + + +class NoopCatalog(Catalog): + def create_table( + self, + identifier: Union[str, Identifier], + schema: Schema, + location: Optional[str] = None, + partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC, + sort_order: SortOrder = UNSORTED_SORT_ORDER, + properties: Properties = EMPTY_DICT, + ) -> Table: + raise NotImplementedError + + def load_table(self, identifier: Union[str, Identifier]) -> Table: + raise NotImplementedError + + def drop_table(self, identifier: Union[str, Identifier]) -> None: + raise NotImplementedError + + def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table: + raise NotImplementedError + + def _commit_table(self, table_request: CommitTableRequest) -> CommitTableResponse: + raise NotImplementedError + + def create_namespace(self, namespace: Union[str, Identifier], properties: Properties = EMPTY_DICT) -> None: + raise NotImplementedError + + def drop_namespace(self, namespace: Union[str, Identifier]) -> None: + raise NotImplementedError + + def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]: + raise NotImplementedError + + def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identifier]: + raise NotImplementedError + + def load_namespace_properties(self, namespace: Union[str, Identifier]) -> Properties: + raise NotImplementedError + + def update_namespace_properties( + self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Properties = EMPTY_DICT + ) -> PropertiesUpdateSummary: + raise NotImplementedError diff --git a/python/pyiceberg/catalog/rest.py b/python/pyiceberg/catalog/rest.py index e24e23910b2a..56b8d9316ebf 100644 --- a/python/pyiceberg/catalog/rest.py +++ b/python/pyiceberg/catalog/rest.py @@ -22,6 +22,7 @@ Literal, Optional, Set, + Tuple, Type, Union, ) @@ -42,6 +43,8 @@ from pyiceberg.exceptions import ( AuthorizationExpiredError, BadRequestError, + CommitFailedException, + CommitStateUnknownException, ForbiddenError, NamespaceAlreadyExistsError, NoSuchNamespaceError, @@ -55,7 +58,12 @@ ) from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec from pyiceberg.schema import Schema -from pyiceberg.table import Table, TableMetadata +from pyiceberg.table import ( + CommitTableRequest, + CommitTableResponse, + Table, + TableMetadata, +) from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder from pyiceberg.typedef import EMPTY_DICT, IcebergBaseModel @@ -68,7 +76,7 @@ class Endpoints: create_namespace: str = "namespaces" load_namespace_metadata: str = "namespaces/{namespace}" drop_namespace: str = "namespaces/{namespace}" - update_properties: str = "namespaces/{namespace}/properties" + update_namespace_properties: str = "namespaces/{namespace}/properties" list_tables: str = "namespaces/{namespace}/tables" create_table: str = "namespaces/{namespace}/tables" load_table: str = "namespaces/{namespace}/tables/{table}" @@ -178,13 +186,13 @@ class RestCatalog(Catalog): _session: Session def __init__(self, name: str, **properties: str): - """Rest Catalog + """Rest Catalog. You either need to provide a client_id and client_secret, or an already valid token. Args: - name: Name to identify the catalog - properties: Properties that are passed along to the configuration + name: Name to identify the catalog. + properties: Properties that are passed along to the configuration. """ super().__init__(name, **properties) self.uri = properties[URI] @@ -192,7 +200,7 @@ def __init__(self, name: str, **properties: str): self._session = self._create_session() def _create_session(self) -> Session: - """Creates a request session with provided catalog configuration""" + """Creates a request session with provided catalog configuration.""" session = Session() # Sets the client side and server side SSL cert verification, if provided as properties. @@ -225,23 +233,22 @@ def _create_session(self) -> Session: return session def _check_valid_namespace_identifier(self, identifier: Union[str, Identifier]) -> Identifier: - """The identifier should have at least one element""" + """The identifier should have at least one element.""" identifier_tuple = Catalog.identifier_to_tuple(identifier) if len(identifier_tuple) < 1: raise NoSuchNamespaceError(f"Empty namespace identifier: {identifier}") return identifier_tuple def url(self, endpoint: str, prefixed: bool = True, **kwargs: Any) -> str: - """Constructs the endpoint + """Constructs the endpoint. Args: - endpoint: Resource identifier that points to the REST catalog - prefixed: If the prefix return by the config needs to be appended + endpoint: Resource identifier that points to the REST catalog. + prefixed: If the prefix return by the config needs to be appended. Returns: - The base url of the rest catalog + The base url of the rest catalog. """ - url = self.uri url = url + "v1/" if url.endswith("/") else url + "/v1/" @@ -394,6 +401,17 @@ def add_headers(self, request: PreparedRequest, **kwargs: Any) -> None: # pylin session.mount(self.uri, SigV4Adapter(**self.properties)) + def _response_to_table(self, identifier_tuple: Tuple[str, ...], table_response: TableResponse) -> Table: + return Table( + identifier=(self.name,) + identifier_tuple if self.name else identifier_tuple, + metadata_location=table_response.metadata_location, + metadata=table_response.metadata, + io=self._load_file_io( + {**table_response.metadata.properties, **table_response.config}, table_response.metadata_location + ), + catalog=self, + ) + def create_table( self, identifier: Union[str, Identifier], @@ -423,15 +441,7 @@ def create_table( self._handle_non_200_response(exc, {409: TableAlreadyExistsError}) table_response = TableResponse(**response.json()) - - return Table( - identifier=(self.name,) + self.identifier_to_tuple(identifier), - metadata_location=table_response.metadata_location, - metadata=table_response.metadata, - io=self._load_file_io( - {**table_response.metadata.properties, **table_response.config}, table_response.metadata_location - ), - ) + return self._response_to_table(self.identifier_to_tuple(identifier), table_response) def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]: namespace_tuple = self._check_valid_namespace_identifier(namespace) @@ -456,14 +466,7 @@ def load_table(self, identifier: Union[str, Identifier]) -> Table: self._handle_non_200_response(exc, {404: NoSuchTableError}) table_response = TableResponse(**response.json()) - return Table( - identifier=(self.name,) + identifier_tuple if self.name else identifier_tuple, - metadata_location=table_response.metadata_location, - metadata=table_response.metadata, - io=self._load_file_io( - {**table_response.metadata.properties, **table_response.config}, table_response.metadata_location - ), - ) + return self._response_to_table(identifier_tuple, table_response) def drop_table(self, identifier: Union[str, Identifier], purge_requested: bool = False) -> None: response = self._session.delete( @@ -490,6 +493,36 @@ def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: U return self.load_table(to_identifier) + def _commit_table(self, table_request: CommitTableRequest) -> CommitTableResponse: + """Updates the table. + + Args: + table_request (CommitTableRequest): The table requests to be carried out. + + Returns: + CommitTableResponse: The updated metadata. + + Raises: + NoSuchTableError: If a table with the given identifier does not exist. + """ + response = self._session.post( + self.url(Endpoints.update_table, prefixed=True, **self._split_identifier_for_path(table_request.identifier)), + data=table_request.json(), + ) + try: + response.raise_for_status() + except HTTPError as exc: + self._handle_non_200_response( + exc, + { + 409: CommitFailedException, + 500: CommitStateUnknownException, + 502: CommitStateUnknownException, + 504: CommitStateUnknownException, + }, + ) + return CommitTableResponse(**response.json()) + def create_namespace(self, namespace: Union[str, Identifier], properties: Properties = EMPTY_DICT) -> None: namespace_tuple = self._check_valid_namespace_identifier(namespace) payload = {"namespace": namespace_tuple, "properties": properties} @@ -542,7 +575,7 @@ def update_namespace_properties( namespace_tuple = self._check_valid_namespace_identifier(namespace) namespace = NAMESPACE_SEPARATOR.join(namespace_tuple) payload = {"removals": list(removals or []), "updates": updates} - response = self._session.post(self.url(Endpoints.update_properties, namespace=namespace), json=payload) + response = self._session.post(self.url(Endpoints.update_namespace_properties, namespace=namespace), json=payload) try: response.raise_for_status() except HTTPError as exc: diff --git a/python/pyiceberg/catalog/sql.py b/python/pyiceberg/catalog/sql.py new file mode 100644 index 000000000000..4a02deb8f5c0 --- /dev/null +++ b/python/pyiceberg/catalog/sql.py @@ -0,0 +1,480 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from typing import ( + List, + Optional, + Set, + Union, +) + +from sqlalchemy import ( + String, + create_engine, + delete, + insert, + select, + union, + update, +) +from sqlalchemy.exc import IntegrityError +from sqlalchemy.orm import ( + DeclarativeBase, + Mapped, + MappedAsDataclass, + Session, + mapped_column, +) + +from pyiceberg.catalog import ( + METADATA_LOCATION, + Catalog, + Identifier, + Properties, + PropertiesUpdateSummary, +) +from pyiceberg.exceptions import ( + NamespaceAlreadyExistsError, + NamespaceNotEmptyError, + NoSuchNamespaceError, + NoSuchPropertyException, + NoSuchTableError, + TableAlreadyExistsError, +) +from pyiceberg.io import load_file_io +from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec +from pyiceberg.schema import Schema +from pyiceberg.serializers import FromInputFile +from pyiceberg.table import CommitTableRequest, CommitTableResponse, Table +from pyiceberg.table.metadata import new_table_metadata +from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder +from pyiceberg.typedef import EMPTY_DICT + + +class SqlCatalogBaseTable(MappedAsDataclass, DeclarativeBase): + pass + + +class IcebergTables(SqlCatalogBaseTable): + __tablename__ = "iceberg_tables" + + catalog_name: Mapped[str] = mapped_column(String(255), nullable=False, primary_key=True) + table_namespace: Mapped[str] = mapped_column(String(255), nullable=False, primary_key=True) + table_name: Mapped[str] = mapped_column(String(255), nullable=False, primary_key=True) + metadata_location: Mapped[Optional[str]] = mapped_column(String(1000), nullable=True) + previous_metadata_location: Mapped[Optional[str]] = mapped_column(String(1000), nullable=True) + + +class IcebergNamespaceProperties(SqlCatalogBaseTable): + __tablename__ = "iceberg_namespace_properties" + # Catalog minimum Namespace Properties + NAMESPACE_MINIMAL_PROPERTIES = {"exists": "true"} + + catalog_name: Mapped[str] = mapped_column(String(255), nullable=False, primary_key=True) + namespace: Mapped[str] = mapped_column(String(255), nullable=False, primary_key=True) + property_key: Mapped[str] = mapped_column(String(255), nullable=False, primary_key=True) + property_value: Mapped[str] = mapped_column(String(1000), nullable=False) + + +class SqlCatalog(Catalog): + def __init__(self, name: str, **properties: str): + super().__init__(name, **properties) + + if not (uri_prop := self.properties.get("uri")): + raise NoSuchPropertyException("SQL connection URI is required") + self.engine = create_engine(uri_prop, echo=True) + + def create_tables(self) -> None: + SqlCatalogBaseTable.metadata.create_all(self.engine) + + def destroy_tables(self) -> None: + SqlCatalogBaseTable.metadata.drop_all(self.engine) + + def _convert_orm_to_iceberg(self, orm_table: IcebergTables) -> Table: + # Check for expected properties. + if not (metadata_location := orm_table.metadata_location): + raise NoSuchTableError(f"Table property {METADATA_LOCATION} is missing") + if not (table_namespace := orm_table.table_namespace): + raise NoSuchTableError(f"Table property {IcebergTables.table_namespace} is missing") + if not (table_name := orm_table.table_name): + raise NoSuchTableError(f"Table property {IcebergTables.table_name} is missing") + + io = load_file_io(properties=self.properties, location=metadata_location) + file = io.new_input(metadata_location) + metadata = FromInputFile.table_metadata(file) + return Table( + identifier=(self.name, table_namespace, table_name), + metadata=metadata, + metadata_location=metadata_location, + io=self._load_file_io(metadata.properties, metadata_location), + catalog=self, + ) + + def create_table( + self, + identifier: Union[str, Identifier], + schema: Schema, + location: Optional[str] = None, + partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC, + sort_order: SortOrder = UNSORTED_SORT_ORDER, + properties: Properties = EMPTY_DICT, + ) -> Table: + """ + Create an Iceberg table. + + Args: + identifier: Table identifier. + schema: Table's schema. + location: Location for the table. Optional Argument. + partition_spec: PartitionSpec for the table. + sort_order: SortOrder for the table. + properties: Table properties that can be a string based dictionary. + + Returns: + Table: the created table instance. + + Raises: + AlreadyExistsError: If a table with the name already exists. + ValueError: If the identifier is invalid, or no path is given to store metadata. + + """ + database_name, table_name = self.identifier_to_database_and_table(identifier) + if not self._namespace_exists(database_name): + raise NoSuchNamespaceError(f"Namespace does not exist: {database_name}") + + location = self._resolve_table_location(location, database_name, table_name) + metadata_location = self._get_metadata_location(location=location) + metadata = new_table_metadata( + location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order, properties=properties + ) + io = load_file_io(properties=self.properties, location=metadata_location) + self._write_metadata(metadata, io, metadata_location) + + with Session(self.engine) as session: + try: + session.add( + IcebergTables( + catalog_name=self.name, + table_namespace=database_name, + table_name=table_name, + metadata_location=metadata_location, + previous_metadata_location=None, + ) + ) + session.commit() + except IntegrityError as e: + raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e + + return self.load_table(identifier=identifier) + + def load_table(self, identifier: Union[str, Identifier]) -> Table: + """Loads the table's metadata and returns the table instance. + + You can also use this method to check for table existence using 'try catalog.table() except NoSuchTableError'. + Note: This method doesn't scan data stored in the table. + + Args: + identifier (str | Identifier): Table identifier. + + Returns: + Table: the table instance with its metadata. + + Raises: + NoSuchTableError: If a table with the name does not exist. + """ + database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError) + with Session(self.engine) as session: + stmt = select(IcebergTables).where( + IcebergTables.catalog_name == self.name, + IcebergTables.table_namespace == database_name, + IcebergTables.table_name == table_name, + ) + result = session.scalar(stmt) + if result: + return self._convert_orm_to_iceberg(result) + raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") + + def drop_table(self, identifier: Union[str, Identifier]) -> None: + """Drop a table. + + Args: + identifier (str | Identifier): Table identifier. + + Raises: + NoSuchTableError: If a table with the name does not exist. + """ + database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError) + with Session(self.engine) as session: + res = session.execute( + delete(IcebergTables).where( + IcebergTables.catalog_name == self.name, + IcebergTables.table_namespace == database_name, + IcebergTables.table_name == table_name, + ) + ) + session.commit() + if res.rowcount < 1: + raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") + + def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table: + """Rename a fully classified table name. + + Args: + from_identifier (str | Identifier): Existing table identifier. + to_identifier (str | Identifier): New table identifier. + + Returns: + Table: the updated table instance with its metadata. + + Raises: + NoSuchTableError: If a table with the name does not exist. + TableAlreadyExistsError: If a table with the new name already exist. + NoSuchNamespaceError: If the target namespace does not exist. + """ + from_database_name, from_table_name = self.identifier_to_database_and_table(from_identifier, NoSuchTableError) + to_database_name, to_table_name = self.identifier_to_database_and_table(to_identifier) + if not self._namespace_exists(to_database_name): + raise NoSuchNamespaceError(f"Namespace does not exist: {to_database_name}") + with Session(self.engine) as session: + try: + stmt = ( + update(IcebergTables) + .where( + IcebergTables.catalog_name == self.name, + IcebergTables.table_namespace == from_database_name, + IcebergTables.table_name == from_table_name, + ) + .values(table_namespace=to_database_name, table_name=to_table_name) + ) + result = session.execute(stmt) + if result.rowcount < 1: + raise NoSuchTableError(f"Table does not exist: {from_table_name}") + session.commit() + except IntegrityError as e: + raise TableAlreadyExistsError(f"Table {to_database_name}.{to_table_name} already exists") from e + return self.load_table(to_identifier) + + def _commit_table(self, table_request: CommitTableRequest) -> CommitTableResponse: + """Updates one or more tables. + + Args: + table_request (CommitTableRequest): The table requests to be carried out. + + Returns: + CommitTableResponse: The updated metadata. + + Raises: + NoSuchTableError: If a table with the given identifier does not exist. + """ + raise NotImplementedError + + def _namespace_exists(self, identifier: Union[str, Identifier]) -> bool: + namespace = self.identifier_to_database(identifier) + with Session(self.engine) as session: + stmt = ( + select(IcebergTables) + .where(IcebergTables.catalog_name == self.name, IcebergTables.table_namespace == namespace) + .limit(1) + ) + result = session.execute(stmt).all() + if result: + return True + stmt = ( + select(IcebergNamespaceProperties) + .where( + IcebergNamespaceProperties.catalog_name == self.name, + IcebergNamespaceProperties.namespace == namespace, + ) + .limit(1) + ) + result = session.execute(stmt).all() + if result: + return True + return False + + def create_namespace(self, namespace: Union[str, Identifier], properties: Properties = EMPTY_DICT) -> None: + """Create a namespace in the catalog. + + Args: + namespace (str | Identifier): Namespace identifier. + properties (Properties): A string dictionary of properties for the given namespace. + + Raises: + NamespaceAlreadyExistsError: If a namespace with the given name already exists. + """ + if not properties: + properties = IcebergNamespaceProperties.NAMESPACE_MINIMAL_PROPERTIES + database_name = self.identifier_to_database(namespace) + if self._namespace_exists(database_name): + raise NamespaceAlreadyExistsError(f"Database {database_name} already exists") + + create_properties = properties if properties else IcebergNamespaceProperties.NAMESPACE_MINIMAL_PROPERTIES + with Session(self.engine) as session: + for key, value in create_properties.items(): + session.add( + IcebergNamespaceProperties( + catalog_name=self.name, namespace=database_name, property_key=key, property_value=value + ) + ) + session.commit() + + def drop_namespace(self, namespace: Union[str, Identifier]) -> None: + """Drop a namespace. + + Args: + namespace (str | Identifier): Namespace identifier. + + Raises: + NoSuchNamespaceError: If a namespace with the given name does not exist. + NamespaceNotEmptyError: If the namespace is not empty. + """ + database_name = self.identifier_to_database(namespace, NoSuchNamespaceError) + if self._namespace_exists(database_name): + if tables := self.list_tables(database_name): + raise NamespaceNotEmptyError(f"Database {database_name} is not empty. {len(tables)} tables exist.") + + with Session(self.engine) as session: + session.execute( + delete(IcebergNamespaceProperties).where( + IcebergNamespaceProperties.catalog_name == self.name, + IcebergNamespaceProperties.namespace == database_name, + ) + ) + session.commit() + + def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]: + """List tables under the given namespace in the catalog. + + If namespace not provided, will list all tables in the catalog. + + Args: + namespace (str | Identifier): Namespace identifier to search. + + Returns: + List[Identifier]: list of table identifiers. + + Raises: + NoSuchNamespaceError: If a namespace with the given name does not exist. + """ + database_name = self.identifier_to_database(namespace, NoSuchNamespaceError) + + stmt = select(IcebergTables).where( + IcebergTables.catalog_name == self.name, IcebergTables.table_namespace == database_name + ) + with Session(self.engine) as session: + result = session.scalars(stmt) + return [(table.table_namespace, table.table_name) for table in result] + + def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identifier]: + """List namespaces from the given namespace. If not given, list top-level namespaces from the catalog. + + Args: + namespace (str | Identifier): Namespace identifier to search. + + Returns: + List[Identifier]: a List of namespace identifiers. + + Raises: + NoSuchNamespaceError: If a namespace with the given name does not exist. + """ + if namespace and not self._namespace_exists(namespace): + raise NoSuchNamespaceError(f"Namespace does not exist: {namespace}") + + table_stmt = select(IcebergTables.table_namespace).where(IcebergTables.catalog_name == self.name) + namespace_stmt = select(IcebergNamespaceProperties.namespace).where(IcebergNamespaceProperties.catalog_name == self.name) + if namespace: + database_name = self.identifier_to_database(namespace, NoSuchNamespaceError) + table_stmt = table_stmt.where(IcebergTables.table_namespace.like(database_name)) + namespace_stmt = namespace_stmt.where(IcebergNamespaceProperties.namespace.like(database_name)) + stmt = union( + table_stmt, + namespace_stmt, + ) + with Session(self.engine) as session: + return [self.identifier_to_tuple(namespace_col) for namespace_col in session.execute(stmt).scalars()] + + def load_namespace_properties(self, namespace: Union[str, Identifier]) -> Properties: + """Get properties for a namespace. + + Args: + namespace (str | Identifier): Namespace identifier. + + Returns: + Properties: Properties for the given namespace. + + Raises: + NoSuchNamespaceError: If a namespace with the given name does not exist. + """ + database_name = self.identifier_to_database(namespace, NoSuchNamespaceError) + + stmt = select(IcebergNamespaceProperties).where( + IcebergNamespaceProperties.catalog_name == self.name, IcebergNamespaceProperties.namespace == database_name + ) + with Session(self.engine) as session: + result = session.scalars(stmt) + return {props.property_key: props.property_value for props in result} + + def update_namespace_properties( + self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Properties = EMPTY_DICT + ) -> PropertiesUpdateSummary: + """Removes provided property keys and updates properties for a namespace. + + Args: + namespace (str | Identifier): Namespace identifier. + removals (Set[str]): Set of property keys that need to be removed. Optional Argument. + updates (Properties): Properties to be updated for the given namespace. + + Raises: + NoSuchNamespaceError: If a namespace with the given name does not exist. + ValueError: If removals and updates have overlapping keys. + """ + database_name = self.identifier_to_database(namespace) + if not self._namespace_exists(database_name): + raise NoSuchNamespaceError(f"Database {database_name} does not exists") + + current_properties = self.load_namespace_properties(namespace=namespace) + properties_update_summary = self._get_updated_props_and_update_summary( + current_properties=current_properties, removals=removals, updates=updates + )[0] + + with Session(self.engine) as session: + if removals: + delete_stmt = delete(IcebergNamespaceProperties).where( + IcebergNamespaceProperties.catalog_name == self.name, + IcebergNamespaceProperties.namespace == database_name, + IcebergNamespaceProperties.property_key.in_(removals), + ) + session.execute(delete_stmt) + + if updates: + # SQLAlchemy does not (yet) support engine agnostic UPSERT + # https://docs.sqlalchemy.org/en/20/orm/queryguide/dml.html#orm-upsert-statements + # This is not a problem since it runs in a single transaction + delete_stmt = delete(IcebergNamespaceProperties).where( + IcebergNamespaceProperties.catalog_name == self.name, + IcebergNamespaceProperties.namespace == database_name, + IcebergNamespaceProperties.property_key.in_(set(updates.keys())), + ) + session.execute(delete_stmt) + insert_stmt = insert(IcebergNamespaceProperties) + for property_key, property_value in updates.items(): + insert_stmt = insert_stmt.values( + catalog_name=self.name, namespace=database_name, property_key=property_key, property_value=property_value + ) + session.execute(insert_stmt) + session.commit() + return properties_update_summary diff --git a/python/pyiceberg/cli/console.py b/python/pyiceberg/cli/console.py index 7ece1b1bbef5..12170b69dda5 100644 --- a/python/pyiceberg/cli/console.py +++ b/python/pyiceberg/cli/console.py @@ -83,9 +83,7 @@ def run(ctx: Context, catalog: Optional[str], verbose: bool, output: str, uri: O def _catalog_and_output(ctx: Context) -> Tuple[Catalog, Output]: - """ - Small helper to set the types - """ + """Small helper to set the types.""" return ctx.obj["catalog"], ctx.obj["output"] @@ -94,7 +92,7 @@ def _catalog_and_output(ctx: Context) -> Tuple[Catalog, Output]: @click.argument("parent", required=False) @catch_exception() def list(ctx: Context, parent: Optional[str]) -> None: # pylint: disable=redefined-builtin - """Lists tables or namespaces""" + """Lists tables or namespaces.""" catalog, output = _catalog_and_output(ctx) identifiers = catalog.list_namespaces(parent or ()) @@ -109,7 +107,7 @@ def list(ctx: Context, parent: Optional[str]) -> None: # pylint: disable=redefi @click.pass_context @catch_exception() def describe(ctx: Context, entity: Literal["name", "namespace", "table"], identifier: str) -> None: - """Describes a namespace or a table""" + """Describes a namespace or a table.""" catalog, output = _catalog_and_output(ctx) identifier_tuple = Catalog.identifier_to_tuple(identifier) @@ -143,7 +141,7 @@ def describe(ctx: Context, entity: Literal["name", "namespace", "table"], identi @click.pass_context @catch_exception() def files(ctx: Context, identifier: str, history: bool) -> None: - """Lists all the files of the table""" + """Lists all the files of the table.""" catalog, output = _catalog_and_output(ctx) catalog_table = catalog.load_table(identifier) @@ -155,7 +153,7 @@ def files(ctx: Context, identifier: str, history: bool) -> None: @click.pass_context @catch_exception() def schema(ctx: Context, identifier: str) -> None: - """Gets the schema of the table""" + """Gets the schema of the table.""" catalog, output = _catalog_and_output(ctx) table = catalog.load_table(identifier) output.schema(table.schema()) @@ -166,7 +164,7 @@ def schema(ctx: Context, identifier: str) -> None: @click.pass_context @catch_exception() def spec(ctx: Context, identifier: str) -> None: - """Returns the partition spec of the table""" + """Returns the partition spec of the table.""" catalog, output = _catalog_and_output(ctx) table = catalog.load_table(identifier) output.spec(table.spec()) @@ -177,7 +175,7 @@ def spec(ctx: Context, identifier: str) -> None: @click.pass_context @catch_exception() def uuid(ctx: Context, identifier: str) -> None: - """Returns the UUID of the table""" + """Returns the UUID of the table.""" catalog, output = _catalog_and_output(ctx) metadata = catalog.load_table(identifier).metadata output.uuid(metadata.table_uuid) @@ -188,7 +186,7 @@ def uuid(ctx: Context, identifier: str) -> None: @click.pass_context @catch_exception() def location(ctx: Context, identifier: str) -> None: - """Returns the location of the table""" + """Returns the location of the table.""" catalog, output = _catalog_and_output(ctx) table = catalog.load_table(identifier) output.text(table.location()) @@ -196,7 +194,7 @@ def location(ctx: Context, identifier: str) -> None: @run.group() def drop() -> None: - """Operations to drop a namespace or table""" + """Operations to drop a namespace or table.""" @drop.command() @@ -204,7 +202,7 @@ def drop() -> None: @click.pass_context @catch_exception() def table(ctx: Context, identifier: str) -> None: # noqa: F811 - """Drops a table""" + """Drops a table.""" catalog, output = _catalog_and_output(ctx) catalog.drop_table(identifier) @@ -216,7 +214,7 @@ def table(ctx: Context, identifier: str) -> None: # noqa: F811 @click.pass_context @catch_exception() def namespace(ctx: Context, identifier: str) -> None: - """Drops a namespace""" + """Drops a namespace.""" catalog, output = _catalog_and_output(ctx) catalog.drop_namespace(identifier) @@ -229,7 +227,7 @@ def namespace(ctx: Context, identifier: str) -> None: @click.pass_context @catch_exception() def rename(ctx: Context, from_identifier: str, to_identifier: str) -> None: - """Renames a table""" + """Renames a table.""" catalog, output = _catalog_and_output(ctx) catalog.rename_table(from_identifier, to_identifier) @@ -238,12 +236,12 @@ def rename(ctx: Context, from_identifier: str, to_identifier: str) -> None: @run.group() def properties() -> None: - """Properties on tables/namespaces""" + """Properties on tables/namespaces.""" @properties.group() def get() -> None: - """Fetch properties on tables/namespaces""" + """Fetch properties on tables/namespaces.""" @get.command("namespace") @@ -252,7 +250,7 @@ def get() -> None: @click.pass_context @catch_exception() def get_namespace(ctx: Context, identifier: str, property_name: str) -> None: - """Fetch properties on a namespace""" + """Fetch properties on a namespace.""" catalog, output = _catalog_and_output(ctx) identifier_tuple = Catalog.identifier_to_tuple(identifier) @@ -274,7 +272,7 @@ def get_namespace(ctx: Context, identifier: str, property_name: str) -> None: @click.pass_context @catch_exception() def get_table(ctx: Context, identifier: str, property_name: str) -> None: - """Fetch properties on a table""" + """Fetch properties on a table.""" catalog, output = _catalog_and_output(ctx) identifier_tuple = Catalog.identifier_to_tuple(identifier) @@ -292,7 +290,7 @@ def get_table(ctx: Context, identifier: str, property_name: str) -> None: @properties.group() def set() -> None: - """Sets a property on tables/namespaces""" + """Sets a property on tables/namespaces.""" @set.command() # type: ignore @@ -302,7 +300,7 @@ def set() -> None: @click.pass_context @catch_exception() def namespace(ctx: Context, identifier: str, property_name: str, property_value: str) -> None: # noqa: F811 - """Sets a property on a namespace""" + """Sets a property on a namespace.""" catalog, output = _catalog_and_output(ctx) catalog.update_namespace_properties(identifier, updates={property_name: property_value}) @@ -316,7 +314,7 @@ def namespace(ctx: Context, identifier: str, property_name: str, property_value: @click.pass_context @catch_exception() def table(ctx: Context, identifier: str, property_name: str, property_value: str) -> None: # noqa: F811 - """Sets a property on a table""" + """Sets a property on a table.""" catalog, output = _catalog_and_output(ctx) identifier_tuple = Catalog.identifier_to_tuple(identifier) @@ -327,7 +325,7 @@ def table(ctx: Context, identifier: str, property_name: str, property_value: str @properties.group() def remove() -> None: - """Removes a property from tables/namespaces""" + """Removes a property from tables/namespaces.""" @remove.command() # type: ignore @@ -336,7 +334,7 @@ def remove() -> None: @click.pass_context @catch_exception() def namespace(ctx: Context, identifier: str, property_name: str) -> None: # noqa: F811 - """Removes a property from a namespace""" + """Removes a property from a namespace.""" catalog, output = _catalog_and_output(ctx) result = catalog.update_namespace_properties(identifier, removals={property_name}) @@ -353,7 +351,7 @@ def namespace(ctx: Context, identifier: str, property_name: str) -> None: # noq @click.pass_context @catch_exception() def table(ctx: Context, identifier: str, property_name: str) -> None: # noqa: F811 - """Removes a property from a table""" + """Removes a property from a table.""" catalog, output = _catalog_and_output(ctx) table = catalog.load_table(identifier) if property_name in table.metadata.properties: diff --git a/python/pyiceberg/cli/output.py b/python/pyiceberg/cli/output.py index 44c94f6b4cfe..896603041cbc 100644 --- a/python/pyiceberg/cli/output.py +++ b/python/pyiceberg/cli/output.py @@ -30,7 +30,7 @@ class Output(ABC): - """Output interface for exporting""" + """Output interface for exporting.""" @abstractmethod def exception(self, ex: Exception) -> None: @@ -70,7 +70,7 @@ def uuid(self, uuid: Optional[UUID]) -> None: class ConsoleOutput(Output): - """Writes to the console""" + """Writes to the console.""" verbose: bool @@ -142,7 +142,7 @@ def files(self, table: Table, history: bool) -> None: manifest_list = snapshot.manifests(io) for manifest in manifest_list: manifest_tree = list_tree.add(f"Manifest: {manifest.manifest_path}") - for manifest_entry in manifest.fetch_manifest_entry(io): + for manifest_entry in manifest.fetch_manifest_entry(io, discard_deleted=False): manifest_tree.add(f"Datafile: {manifest_entry.data_file.file_path}") Console().print(snapshot_tree) @@ -169,7 +169,7 @@ def uuid(self, uuid: Optional[UUID]) -> None: class JsonOutput(Output): - """Writes json to stdout""" + """Writes json to stdout.""" verbose: bool @@ -187,7 +187,7 @@ def identifiers(self, identifiers: List[Identifier]) -> None: def describe_table(self, table: Table) -> None: class FauxTable(IcebergBaseModel): - """Just to encode it using Pydantic""" + """Just to encode it using Pydantic.""" identifier: Identifier metadata_location: str diff --git a/python/pyiceberg/conversions.py b/python/pyiceberg/conversions.py index 075db4be6d75..26ec71475d85 100644 --- a/python/pyiceberg/conversions.py +++ b/python/pyiceberg/conversions.py @@ -14,12 +14,12 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -"""Utility module for various conversions around PrimitiveType implementations +"""Utility module for various conversions around PrimitiveType implementations. This module enables: - - Converting partition strings to built-in python objects - - Converting a value to a byte buffer - - Converting a byte buffer to a value + - Converting partition strings to built-in python objects. + - Converting a value to a byte buffer. + - Converting a byte buffer to a value. Note: Conversion logic varies based on the PrimitiveType implementation. Therefore conversion functions @@ -67,10 +67,10 @@ def handle_none(func: Callable) -> Callable: # type: ignore - """A decorator function to handle cases where partition values are `None` or "__HIVE_DEFAULT_PARTITION__" + """A decorator function to handle cases where partition values are `None` or "__HIVE_DEFAULT_PARTITION__". Args: - func (Callable): A function registered to the singledispatch function `partition_to_py` + func (Callable): A function registered to the singledispatch function `partition_to_py`. """ def wrapper(primitive_type: PrimitiveType, value_str: Optional[str]) -> Any: @@ -85,11 +85,11 @@ def wrapper(primitive_type: PrimitiveType, value_str: Optional[str]) -> Any: @singledispatch def partition_to_py(primitive_type: PrimitiveType, value_str: str) -> Union[int, float, str, uuid.UUID, bytes, Decimal]: - """A generic function which converts a partition string to a python built-in + """A generic function which converts a partition string to a python built-in. Args: - primitive_type (PrimitiveType): An implementation of the PrimitiveType base class - value_str (str): A string representation of a partition value + primitive_type (PrimitiveType): An implementation of the PrimitiveType base class. + value_str (str): A string representation of a partition value. """ raise TypeError(f"Cannot convert '{value_str}' to unsupported type: {primitive_type}") @@ -108,9 +108,10 @@ def _(primitive_type: BooleanType, value_str: str) -> Union[int, float, str, uui @partition_to_py.register(TimestamptzType) @handle_none def _(primitive_type: PrimitiveType, value_str: str) -> int: - """ + """Converts a string to an integer value. + Raises: - ValueError: If the scale/exponent is not 0 + ValueError: If the scale/exponent is not 0. """ _, _, exponent = Decimal(value_str).as_tuple() if exponent != 0: # Raise if there are digits to the right of the decimal @@ -152,15 +153,15 @@ def _(_: DecimalType, value_str: str) -> Decimal: @singledispatch def to_bytes(primitive_type: PrimitiveType, _: Union[bool, bytes, Decimal, float, int, str, uuid.UUID]) -> bytes: - """A generic function which converts a built-in python value to bytes + """A generic function which converts a built-in python value to bytes. This conversion follows the serialization scheme for storing single values as individual binary values defined in the Iceberg specification that can be found at https://iceberg.apache.org/spec/#appendix-d-single-value-serialization Args: - primitive_type (PrimitiveType): An implementation of the PrimitiveType base class + primitive_type (PrimitiveType): An implementation of the PrimitiveType base class. _: The value to convert to bytes (The type of this value depends on which dispatched function is - used--check dispatchable functions for type hints) + used--check dispatchable functions for type hints). """ raise TypeError(f"scale does not match {primitive_type}") @@ -186,7 +187,8 @@ def _(_: PrimitiveType, value: int) -> bytes: @to_bytes.register(FloatType) def _(_: FloatType, value: float) -> bytes: - """ + """Converts a float value into bytes. + Note: float in python is implemented using a double in C. Therefore this involves a conversion of a 32-bit (single precision) float to a 64-bit (double precision) float which introduces some imprecision. """ @@ -216,18 +218,18 @@ def _(_: PrimitiveType, value: bytes) -> bytes: @to_bytes.register(DecimalType) def _(primitive_type: DecimalType, value: Decimal) -> bytes: - """Convert a Decimal value to bytes given a DecimalType instance with defined precision and scale + """Convert a Decimal value to bytes given a DecimalType instance with defined precision and scale. Args: - primitive_type (DecimalType): A DecimalType instance with precision and scale - value (Decimal): A Decimal instance + primitive_type (DecimalType): A DecimalType instance with precision and scale. + value (Decimal): A Decimal instance. Raises: - ValueError: If either the precision or scale of `value` does not match that defined in the DecimalType instance + ValueError: If either the precision or scale of `value` does not match that defined in the DecimalType instance. Returns: - bytes: The byte representation of `value` + bytes: The byte representation of `value`. """ _, digits, exponent = value.as_tuple() exponent = abs(int(exponent)) @@ -243,11 +245,11 @@ def _(primitive_type: DecimalType, value: Decimal) -> bytes: @singledispatch def from_bytes(primitive_type: PrimitiveType, b: bytes) -> L: - """A generic function which converts bytes to a built-in python value + """A generic function which converts bytes to a built-in python value. Args: - primitive_type (PrimitiveType): An implementation of the PrimitiveType base class - b (bytes): The bytes to convert + primitive_type (PrimitiveType): An implementation of the PrimitiveType base class. + b (bytes): The bytes to convert. """ raise TypeError(f"Cannot deserialize bytes, type {primitive_type} not supported: {str(b)}") diff --git a/python/pyiceberg/exceptions.py b/python/pyiceberg/exceptions.py index 813b34c3ba1e..f5555437234d 100644 --- a/python/pyiceberg/exceptions.py +++ b/python/pyiceberg/exceptions.py @@ -17,75 +17,75 @@ class TableAlreadyExistsError(Exception): - """Raised when creating a table with a name that already exists""" + """Raised when creating a table with a name that already exists.""" class NamespaceNotEmptyError(Exception): - """Raised when a name-space being dropped is not empty""" + """Raised when a name-space being dropped is not empty.""" class NamespaceAlreadyExistsError(Exception): - """Raised when a name-space being created already exists in the catalog""" + """Raised when a name-space being created already exists in the catalog.""" class ValidationError(Exception): - """Raises when there is an issue with the schema""" + """Raises when there is an issue with the schema.""" class NoSuchTableError(Exception): - """Raises when the table can't be found in the REST catalog""" + """Raises when the table can't be found in the REST catalog.""" class NoSuchIcebergTableError(NoSuchTableError): - """Raises when the table found in the REST catalog is not an iceberg table""" + """Raises when the table found in the REST catalog is not an iceberg table.""" class NoSuchNamespaceError(Exception): - """Raised when a referenced name-space is not found""" + """Raised when a referenced name-space is not found.""" class RESTError(Exception): - """Raises when there is an unknown response from the REST Catalog""" + """Raises when there is an unknown response from the REST Catalog.""" class BadRequestError(RESTError): - """Raises when an invalid request is being made""" + """Raises when an invalid request is being made.""" class UnauthorizedError(RESTError): - """Raises when you don't have the proper authorization""" + """Raises when you don't have the proper authorization.""" class ServiceUnavailableError(RESTError): - """Raises when the service doesn't respond""" + """Raises when the service doesn't respond.""" class ServerError(RESTError): - """Raises when there is an unhandled exception on the server side""" + """Raises when there is an unhandled exception on the server side.""" class ForbiddenError(RESTError): - """Raises when you don't have the credentials to perform the action on the REST catalog""" + """Raises when you don't have the credentials to perform the action on the REST catalog.""" class AuthorizationExpiredError(RESTError): - """When the credentials are expired when performing an action on the REST catalog""" + """When the credentials are expired when performing an action on the REST catalog.""" class OAuthError(RESTError): - """Raises when there is an error with the OAuth call""" + """Raises when there is an error with the OAuth call.""" class NoSuchPropertyException(Exception): - """When a property is missing""" + """When a property is missing.""" class NotInstalledError(Exception): - """When an optional dependency is not installed""" + """When an optional dependency is not installed.""" class SignError(Exception): - """Raises when unable to sign a S3 request""" + """Raises when unable to sign a S3 request.""" class ResolveError(Exception): @@ -102,3 +102,11 @@ class ConditionalCheckFailedException(DynamoDbError): class GenericDynamoDbError(DynamoDbError): pass + + +class CommitFailedException(RESTError): + """Commit failed, refresh and try again.""" + + +class CommitStateUnknownException(RESTError): + """Commit failed due to unknown reason.""" diff --git a/python/pyiceberg/expressions/__init__.py b/python/pyiceberg/expressions/__init__.py index 5be291b0e33a..0d9434ea9021 100644 --- a/python/pyiceberg/expressions/__init__.py +++ b/python/pyiceberg/expressions/__init__.py @@ -24,6 +24,7 @@ Generic, Iterable, Set, + Tuple, Type, TypeVar, Union, @@ -57,7 +58,7 @@ def _to_literal(value: Union[L, Literal[L]]) -> Literal[L]: class BooleanExpression(ABC): - """An expression that evaluates to a boolean""" + """An expression that evaluates to a boolean.""" @abstractmethod def __invert__(self) -> BooleanExpression: @@ -65,18 +66,18 @@ def __invert__(self) -> BooleanExpression: class Term(Generic[L], ABC): - """A simple expression that evaluates to a value""" + """A simple expression that evaluates to a value.""" class Bound(ABC): - """Represents a bound value expression""" + """Represents a bound value expression.""" B = TypeVar("B") class Unbound(Generic[B], ABC): - """Represents an unbound value expression""" + """Represents an unbound value expression.""" @abstractmethod def bind(self, schema: Schema, case_sensitive: bool = True) -> B: @@ -89,23 +90,23 @@ def as_bound(self) -> Type[Bound]: class BoundTerm(Term[L], Bound, ABC): - """Represents a bound term""" + """Represents a bound term.""" @abstractmethod def ref(self) -> BoundReference[L]: - """Returns the bound reference""" + """Returns the bound reference.""" @abstractmethod def eval(self, struct: StructProtocol) -> L: # pylint: disable=W0613 - """Returns the value at the referenced field's position in an object that abides by the StructProtocol""" + """Returns the value at the referenced field's position in an object that abides by the StructProtocol.""" class BoundReference(BoundTerm[L]): - """A reference bound to a field in a schema + """A reference bound to a field in a schema. Args: - field (NestedField): A referenced field in an Iceberg schema - accessor (Accessor): An Accessor object to access the value at the field's position + field (NestedField): A referenced field in an Iceberg schema. + accessor (Accessor): An Accessor object to access the value at the field's position. """ field: NestedField @@ -116,19 +117,21 @@ def __init__(self, field: NestedField, accessor: Accessor): self.accessor = accessor def eval(self, struct: StructProtocol) -> L: - """Returns the value at the referenced field's position in an object that abides by the StructProtocol + """Returns the value at the referenced field's position in an object that abides by the StructProtocol. Args: - struct (StructProtocol): A row object that abides by the StructProtocol and returns values given a position + struct (StructProtocol): A row object that abides by the StructProtocol and returns values given a position. Returns: - Any: The value at the referenced field's position in `struct` + Any: The value at the referenced field's position in `struct`. """ return self.accessor.get(struct) def __eq__(self, other: Any) -> bool: + """Returns the equality of two instances of the BoundReference class.""" return self.field == other.field if isinstance(other, BoundReference) else False def __repr__(self) -> str: + """Returns the string representation of the BoundReference class.""" return f"BoundReference(field={repr(self.field)}, accessor={repr(self.accessor)})" def ref(self) -> BoundReference[L]: @@ -144,13 +147,13 @@ def bind(self, schema: Schema, case_sensitive: bool = True) -> BoundTerm[L]: class Reference(UnboundTerm[Any]): - """A reference not yet bound to a field in a schema + """A reference not yet bound to a field in a schema. Args: - name (str): The name of the field + name (str): The name of the field. Note: - An unbound reference is sometimes referred to as a "named" reference + An unbound reference is sometimes referred to as a "named" reference. """ name: str @@ -159,23 +162,25 @@ def __init__(self, name: str) -> None: self.name = name def __repr__(self) -> str: + """Returns the string representation of the Reference class.""" return f"Reference(name={repr(self.name)})" def __eq__(self, other: Any) -> bool: + """Returns the equality of two instances of the Reference class.""" return self.name == other.name if isinstance(other, Reference) else False def bind(self, schema: Schema, case_sensitive: bool = True) -> BoundReference[L]: - """Bind the reference to an Iceberg schema + """Bind the reference to an Iceberg schema. Args: - schema (Schema): An Iceberg schema - case_sensitive (bool): Whether to consider case when binding the reference to the field + schema (Schema): An Iceberg schema. + case_sensitive (bool): Whether to consider case when binding the reference to the field. Raises: - ValueError: If an empty name is provided + ValueError: If an empty name is provided. Returns: - BoundReference: A reference bound to the specific field in the Iceberg schema + BoundReference: A reference bound to the specific field in the Iceberg schema. """ field = schema.find_field(name_or_id=self.name, case_sensitive=case_sensitive) accessor = schema.accessor_for_field(field.field_id) @@ -187,7 +192,7 @@ def as_bound(self) -> Type[BoundReference[L]]: class And(BooleanExpression): - """AND operation expression - logical conjunction""" + """AND operation expression - logical conjunction.""" left: BooleanExpression right: BooleanExpression @@ -208,21 +213,29 @@ def __new__(cls, left: BooleanExpression, right: BooleanExpression, *rest: Boole return obj def __eq__(self, other: Any) -> bool: + """Returns the equality of two instances of the And class.""" return self.left == other.left and self.right == other.right if isinstance(other, And) else False def __str__(self) -> str: + """Returns the string representation of the And class.""" return f"And(left={str(self.left)}, right={str(self.right)})" def __repr__(self) -> str: + """Returns the string representation of the And class.""" return f"And(left={repr(self.left)}, right={repr(self.right)})" def __invert__(self) -> BooleanExpression: + """Transform the Expression into its negated version.""" # De Morgan's law: not (A and B) = (not A) or (not B) return Or(~self.left, ~self.right) + def __getnewargs__(self) -> Tuple[BooleanExpression, BooleanExpression]: + """A magic function for pickling the And class.""" + return (self.left, self.right) + class Or(BooleanExpression): - """OR operation expression - logical disjunction""" + """OR operation expression - logical disjunction.""" left: BooleanExpression right: BooleanExpression @@ -243,18 +256,25 @@ def __new__(cls, left: BooleanExpression, right: BooleanExpression, *rest: Boole return obj def __eq__(self, other: Any) -> bool: + """Returns the equality of two instances of the Or class.""" return self.left == other.left and self.right == other.right if isinstance(other, Or) else False def __repr__(self) -> str: + """Returns the string representation of the Or class.""" return f"Or(left={repr(self.left)}, right={repr(self.right)})" def __invert__(self) -> BooleanExpression: + """Transform the Expression into its negated version.""" # De Morgan's law: not (A or B) = (not A) and (not B) return And(~self.left, ~self.right) + def __getnewargs__(self) -> Tuple[BooleanExpression, BooleanExpression]: + """A magic function for pickling the Or class.""" + return (self.left, self.right) + class Not(BooleanExpression): - """NOT operation expression - logical negation""" + """NOT operation expression - logical negation.""" child: BooleanExpression @@ -270,38 +290,51 @@ def __new__(cls, child: BooleanExpression) -> BooleanExpression: # type: ignore return obj def __repr__(self) -> str: + """Returns the string representation of the Not class.""" return f"Not(child={repr(self.child)})" def __eq__(self, other: Any) -> bool: + """Returns the equality of two instances of the Not class.""" return self.child == other.child if isinstance(other, Not) else False def __invert__(self) -> BooleanExpression: + """Transform the Expression into its negated version.""" return self.child + def __getnewargs__(self) -> Tuple[BooleanExpression]: + """A magic function for pickling the Not class.""" + return (self.child,) + class AlwaysTrue(BooleanExpression, Singleton): - """TRUE expression""" + """TRUE expression.""" def __invert__(self) -> AlwaysFalse: + """Transform the Expression into its negated version.""" return AlwaysFalse() def __str__(self) -> str: + """Returns the string representation of the AlwaysTrue class.""" return "AlwaysTrue()" def __repr__(self) -> str: + """Returns the string representation of the AlwaysTrue class.""" return "AlwaysTrue()" class AlwaysFalse(BooleanExpression, Singleton): - """FALSE expression""" + """FALSE expression.""" def __invert__(self) -> AlwaysTrue: + """Transform the Expression into its negated version.""" return AlwaysTrue() def __str__(self) -> str: + """Returns the string representation of the AlwaysFalse class.""" return "AlwaysFalse()" def __repr__(self) -> str: + """Returns the string representation of the AlwaysFalse class.""" return "AlwaysFalse()" @@ -312,6 +345,7 @@ def __init__(self, term: BoundTerm[L]): self.term = term def __eq__(self, other: Any) -> bool: + """Returns the equality of two instances of the BoundPredicate class.""" if isinstance(other, BoundPredicate): return self.term == other.term return False @@ -329,6 +363,7 @@ def __init__(self, term: Union[str, UnboundTerm[Any]]): self.term = _to_unbound_term(term) def __eq__(self, other: Any) -> bool: + """Returns the equality of two instances of the UnboundPredicate class.""" return self.term == other.term if isinstance(other, UnboundPredicate) else False @abstractmethod @@ -347,6 +382,7 @@ def bind(self, schema: Schema, case_sensitive: bool = True) -> BoundUnaryPredica return self.as_bound(bound_term) def __repr__(self) -> str: + """Returns the string representation of the UnaryPredicate class.""" return f"{str(self.__class__.__name__)}(term={repr(self.term)})" @property @@ -357,6 +393,7 @@ def as_bound(self) -> Type[BoundUnaryPredicate[Any]]: class BoundUnaryPredicate(BoundPredicate[L], ABC): def __repr__(self) -> str: + """Returns the string representation of the BoundUnaryPredicate class.""" return f"{str(self.__class__.__name__)}(term={repr(self.term)})" @property @@ -364,6 +401,10 @@ def __repr__(self) -> str: def as_unbound(self) -> Type[UnaryPredicate]: ... + def __getnewargs__(self) -> Tuple[BoundTerm[L]]: + """A magic function for pickling the BoundUnaryPredicate class.""" + return (self.term,) + class BoundIsNull(BoundUnaryPredicate[L]): def __new__(cls, term: BoundTerm[L]) -> BooleanExpression: # type: ignore # pylint: disable=W0221 @@ -372,6 +413,7 @@ def __new__(cls, term: BoundTerm[L]) -> BooleanExpression: # type: ignore # py return super().__new__(cls) def __invert__(self) -> BoundNotNull[L]: + """Transform the Expression into its negated version.""" return BoundNotNull(self.term) @property @@ -386,6 +428,7 @@ def __new__(cls, term: BoundTerm[L]): # type: ignore # pylint: disable=W0221 return super().__new__(cls) def __invert__(self) -> BoundIsNull[L]: + """Transform the Expression into its negated version.""" return BoundIsNull(self.term) @property @@ -395,6 +438,7 @@ def as_unbound(self) -> Type[NotNull]: class IsNull(UnaryPredicate): def __invert__(self) -> NotNull: + """Transform the Expression into its negated version.""" return NotNull(self.term) @property @@ -404,6 +448,7 @@ def as_bound(self) -> Type[BoundIsNull[L]]: class NotNull(UnaryPredicate): def __invert__(self) -> IsNull: + """Transform the Expression into its negated version.""" return IsNull(self.term) @property @@ -419,6 +464,7 @@ def __new__(cls, term: BoundTerm[L]) -> BooleanExpression: # type: ignore # py return AlwaysFalse() def __invert__(self) -> BoundNotNaN[L]: + """Transform the Expression into its negated version.""" return BoundNotNaN(self.term) @property @@ -434,6 +480,7 @@ def __new__(cls, term: BoundTerm[L]) -> BooleanExpression: # type: ignore # py return AlwaysTrue() def __invert__(self) -> BoundIsNaN[L]: + """Transform the Expression into its negated version.""" return BoundIsNaN(self.term) @property @@ -443,6 +490,7 @@ def as_unbound(self) -> Type[NotNaN]: class IsNaN(UnaryPredicate): def __invert__(self) -> NotNaN: + """Transform the Expression into its negated version.""" return NotNaN(self.term) @property @@ -452,6 +500,7 @@ def as_bound(self) -> Type[BoundIsNaN[L]]: class NotNaN(UnaryPredicate): def __invert__(self) -> IsNaN: + """Transform the Expression into its negated version.""" return IsNaN(self.term) @property @@ -471,16 +520,23 @@ def bind(self, schema: Schema, case_sensitive: bool = True) -> BoundSetPredicate return self.as_bound(bound_term, {lit.to(bound_term.ref().field.field_type) for lit in self.literals}) def __str__(self) -> str: + """Returns the string representation of the SetPredicate class.""" # Sort to make it deterministic return f"{str(self.__class__.__name__)}({str(self.term)}, {{{', '.join(sorted([str(literal) for literal in self.literals]))}}})" def __repr__(self) -> str: + """Returns the string representation of the SetPredicate class.""" # Sort to make it deterministic return f"{str(self.__class__.__name__)}({repr(self.term)}, {{{', '.join(sorted([repr(literal) for literal in self.literals]))}}})" def __eq__(self, other: Any) -> bool: + """Returns the equality of two instances of the SetPredicate class.""" return self.term == other.term and self.literals == other.literals if isinstance(other, SetPredicate) else False + def __getnewargs__(self) -> Tuple[UnboundTerm[L], Set[Literal[L]]]: + """A magic function for pickling the SetPredicate class.""" + return (self.term, self.literals) + @property @abstractmethod def as_bound(self) -> Type[BoundSetPredicate[L]]: @@ -500,16 +556,23 @@ def value_set(self) -> Set[L]: return {lit.value for lit in self.literals} def __str__(self) -> str: + """Returns the string representation of the BoundSetPredicate class.""" # Sort to make it deterministic return f"{str(self.__class__.__name__)}({str(self.term)}, {{{', '.join(sorted([str(literal) for literal in self.literals]))}}})" def __repr__(self) -> str: + """Returns the string representation of the BoundSetPredicate class.""" # Sort to make it deterministic return f"{str(self.__class__.__name__)}({repr(self.term)}, {{{', '.join(sorted([repr(literal) for literal in self.literals]))}}})" def __eq__(self, other: Any) -> bool: + """Returns the equality of two instances of the BoundSetPredicate class.""" return self.term == other.term and self.literals == other.literals if isinstance(other, BoundSetPredicate) else False + def __getnewargs__(self) -> Tuple[BoundTerm[L], Set[Literal[L]]]: + """A magic function for pickling the BoundSetPredicate class.""" + return (self.term, self.literals) + @property @abstractmethod def as_unbound(self) -> Type[SetPredicate[L]]: @@ -527,9 +590,11 @@ def __new__(cls, term: BoundTerm[L], literals: Set[Literal[L]]) -> BooleanExpres return super().__new__(cls) def __invert__(self) -> BoundNotIn[L]: + """Transform the Expression into its negated version.""" return BoundNotIn(self.term, self.literals) def __eq__(self, other: Any) -> bool: + """Returns the equality of two instances of the BoundIn class.""" return self.term == other.term and self.literals == other.literals if isinstance(other, BoundIn) else False @property @@ -552,6 +617,7 @@ def __new__( # type: ignore # pylint: disable=W0221 return super().__new__(cls) def __invert__(self) -> BoundIn[L]: + """Transform the Expression into its negated version.""" return BoundIn(self.term, self.literals) @property @@ -573,6 +639,7 @@ def __new__( # type: ignore # pylint: disable=W0221 return super().__new__(cls) def __invert__(self) -> NotIn[L]: + """Transform the Expression into its negated version.""" return NotIn[L](self.term, self.literals) @property @@ -594,9 +661,11 @@ def __new__( # type: ignore # pylint: disable=W0221 return super().__new__(cls) def __invert__(self) -> In[L]: + """Transform the Expression into its negated version.""" return In[L](self.term, self.literals) def __eq__(self, other: Any) -> bool: + """Returns the equality of two instances of the NotIn class.""" if isinstance(other, NotIn): return self.term == other.term and self.literals == other.literals return False @@ -631,11 +700,13 @@ def bind(self, schema: Schema, case_sensitive: bool = True) -> BoundLiteralPredi return self.as_bound(bound_term, lit) def __eq__(self, other: Any) -> bool: + """Returns the equality of two instances of the LiteralPredicate class.""" if isinstance(other, LiteralPredicate): return self.term == other.term and self.literal == other.literal return False def __repr__(self) -> str: + """Returns the string representation of the LiteralPredicate class.""" return f"{str(self.__class__.__name__)}(term={repr(self.term)}, literal={repr(self.literal)})" @property @@ -653,11 +724,13 @@ def __init__(self, term: BoundTerm[L], literal: Literal[L]): # pylint: disable= self.literal = literal # pylint: disable=W0621 def __eq__(self, other: Any) -> bool: + """Returns the equality of two instances of the BoundLiteralPredicate class.""" if isinstance(other, BoundLiteralPredicate): return self.term == other.term and self.literal == other.literal return False def __repr__(self) -> str: + """Returns the string representation of the BoundLiteralPredicate class.""" return f"{str(self.__class__.__name__)}(term={repr(self.term)}, literal={repr(self.literal)})" @property @@ -668,6 +741,7 @@ def as_unbound(self) -> Type[LiteralPredicate[L]]: class BoundEqualTo(BoundLiteralPredicate[L]): def __invert__(self) -> BoundNotEqualTo[L]: + """Transform the Expression into its negated version.""" return BoundNotEqualTo[L](self.term, self.literal) @property @@ -677,6 +751,7 @@ def as_unbound(self) -> Type[EqualTo[L]]: class BoundNotEqualTo(BoundLiteralPredicate[L]): def __invert__(self) -> BoundEqualTo[L]: + """Transform the Expression into its negated version.""" return BoundEqualTo[L](self.term, self.literal) @property @@ -686,6 +761,7 @@ def as_unbound(self) -> Type[NotEqualTo[L]]: class BoundGreaterThanOrEqual(BoundLiteralPredicate[L]): def __invert__(self) -> BoundLessThan[L]: + """Transform the Expression into its negated version.""" return BoundLessThan[L](self.term, self.literal) @property @@ -695,6 +771,7 @@ def as_unbound(self) -> Type[GreaterThanOrEqual[L]]: class BoundGreaterThan(BoundLiteralPredicate[L]): def __invert__(self) -> BoundLessThanOrEqual[L]: + """Transform the Expression into its negated version.""" return BoundLessThanOrEqual(self.term, self.literal) @property @@ -704,6 +781,7 @@ def as_unbound(self) -> Type[GreaterThan[L]]: class BoundLessThan(BoundLiteralPredicate[L]): def __invert__(self) -> BoundGreaterThanOrEqual[L]: + """Transform the Expression into its negated version.""" return BoundGreaterThanOrEqual[L](self.term, self.literal) @property @@ -713,6 +791,7 @@ def as_unbound(self) -> Type[LessThan[L]]: class BoundLessThanOrEqual(BoundLiteralPredicate[L]): def __invert__(self) -> BoundGreaterThan[L]: + """Transform the Expression into its negated version.""" return BoundGreaterThan[L](self.term, self.literal) @property @@ -722,6 +801,7 @@ def as_unbound(self) -> Type[LessThanOrEqual[L]]: class BoundStartsWith(BoundLiteralPredicate[L]): def __invert__(self) -> BoundNotStartsWith[L]: + """Transform the Expression into its negated version.""" return BoundNotStartsWith[L](self.term, self.literal) @property @@ -731,6 +811,7 @@ def as_unbound(self) -> Type[StartsWith[L]]: class BoundNotStartsWith(BoundLiteralPredicate[L]): def __invert__(self) -> BoundStartsWith[L]: + """Transform the Expression into its negated version.""" return BoundStartsWith[L](self.term, self.literal) @property @@ -740,6 +821,7 @@ def as_unbound(self) -> Type[NotStartsWith[L]]: class EqualTo(LiteralPredicate[L]): def __invert__(self) -> NotEqualTo[L]: + """Transform the Expression into its negated version.""" return NotEqualTo[L](self.term, self.literal) @property @@ -749,6 +831,7 @@ def as_bound(self) -> Type[BoundEqualTo[L]]: class NotEqualTo(LiteralPredicate[L]): def __invert__(self) -> EqualTo[L]: + """Transform the Expression into its negated version.""" return EqualTo[L](self.term, self.literal) @property @@ -758,6 +841,7 @@ def as_bound(self) -> Type[BoundNotEqualTo[L]]: class LessThan(LiteralPredicate[L]): def __invert__(self) -> GreaterThanOrEqual[L]: + """Transform the Expression into its negated version.""" return GreaterThanOrEqual[L](self.term, self.literal) @property @@ -767,6 +851,7 @@ def as_bound(self) -> Type[BoundLessThan[L]]: class GreaterThanOrEqual(LiteralPredicate[L]): def __invert__(self) -> LessThan[L]: + """Transform the Expression into its negated version.""" return LessThan[L](self.term, self.literal) @property @@ -776,6 +861,7 @@ def as_bound(self) -> Type[BoundGreaterThanOrEqual[L]]: class GreaterThan(LiteralPredicate[L]): def __invert__(self) -> LessThanOrEqual[L]: + """Transform the Expression into its negated version.""" return LessThanOrEqual[L](self.term, self.literal) @property @@ -785,6 +871,7 @@ def as_bound(self) -> Type[BoundGreaterThan[L]]: class LessThanOrEqual(LiteralPredicate[L]): def __invert__(self) -> GreaterThan[L]: + """Transform the Expression into its negated version.""" return GreaterThan[L](self.term, self.literal) @property @@ -794,6 +881,7 @@ def as_bound(self) -> Type[BoundLessThanOrEqual[L]]: class StartsWith(LiteralPredicate[L]): def __invert__(self) -> NotStartsWith[L]: + """Transform the Expression into its negated version.""" return NotStartsWith[L](self.term, self.literal) @property @@ -803,6 +891,7 @@ def as_bound(self) -> Type[BoundStartsWith[L]]: class NotStartsWith(LiteralPredicate[L]): def __invert__(self) -> NotStartsWith[L]: + """Transform the Expression into its negated version.""" return NotStartsWith[L](self.term, self.literal) @property diff --git a/python/pyiceberg/expressions/literals.py b/python/pyiceberg/expressions/literals.py index 8414ce533ee4..1e47e87608a9 100644 --- a/python/pyiceberg/expressions/literals.py +++ b/python/pyiceberg/expressions/literals.py @@ -59,7 +59,7 @@ class Literal(Generic[L], ABC): - """Literal which has a value and can be converted between types""" + """Literal which has a value and can be converted between types.""" _value: L @@ -80,44 +80,53 @@ def to(self, type_var: IcebergType) -> Literal[L]: ... # pragma: no cover def __repr__(self) -> str: + """Returns the string representation of the Literal class.""" return f"{type(self).__name__}({self.value!r})" def __str__(self) -> str: + """Returns the string representation of the Literal class.""" return str(self.value) def __hash__(self) -> int: + """Returns a hashed representation of the Literal class.""" return hash(self.value) def __eq__(self, other: Any) -> bool: + """Returns the equality of two instances of the Literal class.""" if not isinstance(other, Literal): return False return self.value == other.value def __ne__(self, other: Any) -> bool: + """Returns the inequality of two instances of the Literal class.""" return not self.__eq__(other) def __lt__(self, other: Any) -> bool: + """Returns if one instance of the Literal class is less than another instance.""" return self.value < other.value def __gt__(self, other: Any) -> bool: + """Returns if one instance of the Literal class is greater than another instance.""" return self.value > other.value def __le__(self, other: Any) -> bool: + """Returns if one instance of the Literal class is less than or equal to another instance.""" return self.value <= other.value def __ge__(self, other: Any) -> bool: + """Returns if one instance of the Literal class is greater than or equal to another instance.""" return self.value >= other.value def literal(value: L) -> Literal[L]: """ - A generic Literal factory to construct an Iceberg Literal based on Python primitive data type + A generic Literal factory to construct an Iceberg Literal based on Python primitive data type. Args: - value (Python primitive type): the value to be associated with literal + value (Python primitive type): the value to be associated with literal. Example: - from pyiceberg.expressions.literals import literal + from pyiceberg.expressions.literals import literal. >>> literal(123) LongLiteral(123) """ @@ -141,17 +150,21 @@ def literal(value: L) -> Literal[L]: class AboveMax(Literal[L]): def __repr__(self) -> str: + """Returns the string representation of the AboveMax class.""" return f"{self.__class__.__name__}()" def __str__(self) -> str: + """Returns the string representation of the AboveMax class.""" return self.__class__.__name__ class BelowMin(Literal[L]): def __repr__(self) -> str: + """Returns the string representation of the BelowMin class.""" return f"{self.__class__.__name__}()" def __str__(self) -> str: + """Returns the string representation of the BelowMin class.""" return self.__class__.__name__ @@ -314,21 +327,27 @@ def __init__(self, value: float) -> None: self._value32 = struct.unpack(" bool: + """Returns the equality of two instances of the FloatLiteral class.""" return self._value32 == other def __lt__(self, other: Any) -> bool: + """Returns if one instance of the FloatLiteral class is less than another instance.""" return self._value32 < other def __gt__(self, other: Any) -> bool: + """Returns if one instance of the FloatLiteral class is greater than another instance.""" return self._value32 > other def __le__(self, other: Any) -> bool: + """Returns if one instance of the FloatLiteral class is less than or equal to another instance.""" return self._value32 <= other def __ge__(self, other: Any) -> bool: + """Returns if one instance of the FloatLiteral class is greater than or equal to another instance.""" return self._value32 >= other def __hash__(self) -> int: + """Returns a hashed representation of the FloatLiteral class.""" return hash(self._value32) @singledispatchmethod @@ -573,6 +592,7 @@ def _(self, type_var: BooleanType) -> Literal[bool]: raise ValueError(f"Could not convert {self.value} into a {type_var}") def __repr__(self) -> str: + """Returns the string representation of the StringLiteral class.""" return f"literal({repr(self.value)})" diff --git a/python/pyiceberg/expressions/parser.py b/python/pyiceberg/expressions/parser.py index 02dd64d1a1af..148637473bcc 100644 --- a/python/pyiceberg/expressions/parser.py +++ b/python/pyiceberg/expressions/parser.py @@ -18,6 +18,7 @@ from pyparsing import ( CaselessKeyword, + DelimitedList, Group, ParserElement, ParseResults, @@ -25,7 +26,6 @@ Word, alphanums, alphas, - delimited_list, infix_notation, one_of, opAssoc, @@ -76,12 +76,12 @@ LIKE = CaselessKeyword("like") identifier = Word(alphas, alphanums + "_$").set_results_name("identifier") -column = delimited_list(identifier, delim=".", combine=True).set_results_name("column") +column = DelimitedList(identifier, delim=".", combine=False).set_results_name("column") @column.set_parse_action def _(result: ParseResults) -> Reference: - return Reference(result.column[0]) + return Reference(result.column[-1]) boolean = one_of(["true", "false"], caseless=True).set_results_name("boolean") @@ -89,7 +89,7 @@ def _(result: ParseResults) -> Reference: decimal = common.real().set_results_name("decimal") integer = common.signed_integer().set_results_name("integer") literal = Group(string | decimal | integer).set_results_name("literal") -literal_set = Group(delimited_list(string) | delimited_list(decimal) | delimited_list(integer)).set_results_name("literal_set") +literal_set = Group(DelimitedList(string) | DelimitedList(decimal) | DelimitedList(integer)).set_results_name("literal_set") @boolean.set_parse_action @@ -251,5 +251,5 @@ def handle_or(result: ParseResults) -> Or: def parse(expr: str) -> BooleanExpression: - """Parses a boolean expression""" + """Parses a boolean expression.""" return boolean_expression.parse_string(expr)[0] diff --git a/python/pyiceberg/expressions/visitors.py b/python/pyiceberg/expressions/visitors.py index 4de1ce423611..b701ce7e3ce2 100644 --- a/python/pyiceberg/expressions/visitors.py +++ b/python/pyiceberg/expressions/visitors.py @@ -81,99 +81,99 @@ class BooleanExpressionVisitor(Generic[T], ABC): @abstractmethod def visit_true(self) -> T: - """Visit method for an AlwaysTrue boolean expression + """Visit method for an AlwaysTrue boolean expression. Note: This visit method has no arguments since AlwaysTrue instances have no context. """ @abstractmethod def visit_false(self) -> T: - """Visit method for an AlwaysFalse boolean expression + """Visit method for an AlwaysFalse boolean expression. Note: This visit method has no arguments since AlwaysFalse instances have no context. """ @abstractmethod def visit_not(self, child_result: T) -> T: - """Visit method for a Not boolean expression + """Visit method for a Not boolean expression. Args: - child_result (T): The result of visiting the child of the Not boolean expression + child_result (T): The result of visiting the child of the Not boolean expression. """ @abstractmethod def visit_and(self, left_result: T, right_result: T) -> T: - """Visit method for an And boolean expression + """Visit method for an And boolean expression. Args: - left_result (T): The result of visiting the left side of the expression - right_result (T): The result of visiting the right side of the expression + left_result (T): The result of visiting the left side of the expression. + right_result (T): The result of visiting the right side of the expression. """ @abstractmethod def visit_or(self, left_result: T, right_result: T) -> T: - """Visit method for an Or boolean expression + """Visit method for an Or boolean expression. Args: - left_result (T): The result of visiting the left side of the expression - right_result (T): The result of visiting the right side of the expression + left_result (T): The result of visiting the left side of the expression. + right_result (T): The result of visiting the right side of the expression. """ @abstractmethod def visit_unbound_predicate(self, predicate: UnboundPredicate[L]) -> T: - """Visit method for an unbound predicate in an expression tree + """Visit method for an unbound predicate in an expression tree. Args: - predicate (UnboundPredicate[L): An instance of an UnboundPredicate + predicate (UnboundPredicate[L): An instance of an UnboundPredicate. """ @abstractmethod def visit_bound_predicate(self, predicate: BoundPredicate[L]) -> T: - """Visit method for a bound predicate in an expression tree + """Visit method for a bound predicate in an expression tree. Args: - predicate (BoundPredicate[L]): An instance of a BoundPredicate + predicate (BoundPredicate[L]): An instance of a BoundPredicate. """ @singledispatch def visit(obj: BooleanExpression, visitor: BooleanExpressionVisitor[T]) -> T: - """A generic function for applying a boolean expression visitor to any point within an expression + """A generic function for applying a boolean expression visitor to any point within an expression. - The function traverses the expression in post-order fashion + The function traverses the expression in post-order fashion. Args: - obj (BooleanExpression): An instance of a BooleanExpression - visitor (BooleanExpressionVisitor[T]): An instance of an implementation of the generic BooleanExpressionVisitor base class + obj (BooleanExpression): An instance of a BooleanExpression. + visitor (BooleanExpressionVisitor[T]): An instance of an implementation of the generic BooleanExpressionVisitor base class. Raises: - NotImplementedError: If attempting to visit an unsupported expression + NotImplementedError: If attempting to visit an unsupported expression. """ raise NotImplementedError(f"Cannot visit unsupported expression: {obj}") @visit.register(AlwaysTrue) def _(_: AlwaysTrue, visitor: BooleanExpressionVisitor[T]) -> T: - """Visit an AlwaysTrue boolean expression with a concrete BooleanExpressionVisitor""" + """Visit an AlwaysTrue boolean expression with a concrete BooleanExpressionVisitor.""" return visitor.visit_true() @visit.register(AlwaysFalse) def _(_: AlwaysFalse, visitor: BooleanExpressionVisitor[T]) -> T: - """Visit an AlwaysFalse boolean expression with a concrete BooleanExpressionVisitor""" + """Visit an AlwaysFalse boolean expression with a concrete BooleanExpressionVisitor.""" return visitor.visit_false() @visit.register(Not) def _(obj: Not, visitor: BooleanExpressionVisitor[T]) -> T: - """Visit a Not boolean expression with a concrete BooleanExpressionVisitor""" + """Visit a Not boolean expression with a concrete BooleanExpressionVisitor.""" child_result: T = visit(obj.child, visitor=visitor) return visitor.visit_not(child_result=child_result) @visit.register(And) def _(obj: And, visitor: BooleanExpressionVisitor[T]) -> T: - """Visit an And boolean expression with a concrete BooleanExpressionVisitor""" + """Visit an And boolean expression with a concrete BooleanExpressionVisitor.""" left_result: T = visit(obj.left, visitor=visitor) right_result: T = visit(obj.right, visitor=visitor) return visitor.visit_and(left_result=left_result, right_result=right_result) @@ -181,47 +181,47 @@ def _(obj: And, visitor: BooleanExpressionVisitor[T]) -> T: @visit.register(UnboundPredicate) def _(obj: UnboundPredicate[L], visitor: BooleanExpressionVisitor[T]) -> T: - """Visit an unbound boolean expression with a concrete BooleanExpressionVisitor""" + """Visit an unbound boolean expression with a concrete BooleanExpressionVisitor.""" return visitor.visit_unbound_predicate(predicate=obj) @visit.register(BoundPredicate) def _(obj: BoundPredicate[L], visitor: BooleanExpressionVisitor[T]) -> T: - """Visit a bound boolean expression with a concrete BooleanExpressionVisitor""" + """Visit a bound boolean expression with a concrete BooleanExpressionVisitor.""" return visitor.visit_bound_predicate(predicate=obj) @visit.register(Or) def _(obj: Or, visitor: BooleanExpressionVisitor[T]) -> T: - """Visit an Or boolean expression with a concrete BooleanExpressionVisitor""" + """Visit an Or boolean expression with a concrete BooleanExpressionVisitor.""" left_result: T = visit(obj.left, visitor=visitor) right_result: T = visit(obj.right, visitor=visitor) return visitor.visit_or(left_result=left_result, right_result=right_result) def bind(schema: Schema, expression: BooleanExpression, case_sensitive: bool) -> BooleanExpression: - """Travers over an expression to bind the predicates to the schema + """Travers over an expression to bind the predicates to the schema. Args: - schema (Schema): A schema to use when binding the expression - expression (BooleanExpression): An expression containing UnboundPredicates that can be bound - case_sensitive (bool): Whether to consider case when binding a reference to a field in a schema, defaults to True + schema (Schema): A schema to use when binding the expression. + expression (BooleanExpression): An expression containing UnboundPredicates that can be bound. + case_sensitive (bool): Whether to consider case when binding a reference to a field in a schema, defaults to True. Raises: - TypeError: In the case a predicate is already bound + TypeError: In the case a predicate is already bound. """ return visit(expression, BindVisitor(schema, case_sensitive)) class BindVisitor(BooleanExpressionVisitor[BooleanExpression]): - """Rewrites a boolean expression by replacing unbound references with references to fields in a struct schema + """Rewrites a boolean expression by replacing unbound references with references to fields in a struct schema. Args: - schema (Schema): A schema to use when binding the expression - case_sensitive (bool): Whether to consider case when binding a reference to a field in a schema, defaults to True + schema (Schema): A schema to use when binding the expression. + case_sensitive (bool): Whether to consider case when binding a reference to a field in a schema, defaults to True. Raises: - TypeError: In the case a predicate is already bound + TypeError: In the case a predicate is already bound. """ schema: Schema @@ -256,93 +256,95 @@ def visit_bound_predicate(self, predicate: BoundPredicate[L]) -> BooleanExpressi class BoundBooleanExpressionVisitor(BooleanExpressionVisitor[T], ABC): @abstractmethod def visit_in(self, term: BoundTerm[L], literals: Set[L]) -> T: - """Visit a bound In predicate""" + """Visit a bound In predicate.""" @abstractmethod def visit_not_in(self, term: BoundTerm[L], literals: Set[L]) -> T: - """Visit a bound NotIn predicate""" + """Visit a bound NotIn predicate.""" @abstractmethod def visit_is_nan(self, term: BoundTerm[L]) -> T: - """Visit a bound IsNan predicate""" + """Visit a bound IsNan predicate.""" @abstractmethod def visit_not_nan(self, term: BoundTerm[L]) -> T: - """Visit a bound NotNan predicate""" + """Visit a bound NotNan predicate.""" @abstractmethod def visit_is_null(self, term: BoundTerm[L]) -> T: - """Visit a bound IsNull predicate""" + """Visit a bound IsNull predicate.""" @abstractmethod def visit_not_null(self, term: BoundTerm[L]) -> T: - """Visit a bound NotNull predicate""" + """Visit a bound NotNull predicate.""" @abstractmethod def visit_equal(self, term: BoundTerm[L], literal: Literal[L]) -> T: - """Visit a bound Equal predicate""" + """Visit a bound Equal predicate.""" @abstractmethod def visit_not_equal(self, term: BoundTerm[L], literal: Literal[L]) -> T: - """Visit a bound NotEqual predicate""" + """Visit a bound NotEqual predicate.""" @abstractmethod def visit_greater_than_or_equal(self, term: BoundTerm[L], literal: Literal[L]) -> T: - """Visit a bound GreaterThanOrEqual predicate""" + """Visit a bound GreaterThanOrEqual predicate.""" @abstractmethod def visit_greater_than(self, term: BoundTerm[L], literal: Literal[L]) -> T: - """Visit a bound GreaterThan predicate""" + """Visit a bound GreaterThan predicate.""" @abstractmethod def visit_less_than(self, term: BoundTerm[L], literal: Literal[L]) -> T: - """Visit a bound LessThan predicate""" + """Visit a bound LessThan predicate.""" @abstractmethod def visit_less_than_or_equal(self, term: BoundTerm[L], literal: Literal[L]) -> T: - """Visit a bound LessThanOrEqual predicate""" + """Visit a bound LessThanOrEqual predicate.""" @abstractmethod def visit_true(self) -> T: - """Visit a bound True predicate""" + """Visit a bound True predicate.""" @abstractmethod def visit_false(self) -> T: - """Visit a bound False predicate""" + """Visit a bound False predicate.""" @abstractmethod def visit_not(self, child_result: T) -> T: - """Visit a bound Not predicate""" + """Visit a bound Not predicate.""" @abstractmethod def visit_and(self, left_result: T, right_result: T) -> T: - """Visit a bound And predicate""" + """Visit a bound And predicate.""" @abstractmethod def visit_or(self, left_result: T, right_result: T) -> T: - """Visit a bound Or predicate""" + """Visit a bound Or predicate.""" @abstractmethod def visit_starts_with(self, term: BoundTerm[L], literal: Literal[L]) -> T: - """Visit bound StartsWith predicate""" + """Visit bound StartsWith predicate.""" @abstractmethod def visit_not_starts_with(self, term: BoundTerm[L], literal: Literal[L]) -> T: - """Visit bound NotStartsWith predicate""" + """Visit bound NotStartsWith predicate.""" def visit_unbound_predicate(self, predicate: UnboundPredicate[L]) -> T: - """Visit an unbound predicate + """Visit an unbound predicate. + Args: - predicate (UnboundPredicate[L]): An unbound predicate + predicate (UnboundPredicate[L]): An unbound predicate. Raises: - TypeError: This always raises since an unbound predicate is not expected in a bound boolean expression + TypeError: This always raises since an unbound predicate is not expected in a bound boolean expression. """ raise TypeError(f"Not a bound predicate: {predicate}") def visit_bound_predicate(self, predicate: BoundPredicate[L]) -> T: - """Visit a bound predicate + """Visit a bound predicate. + Args: - predicate (BoundPredicate[L]): A bound predicate + predicate (BoundPredicate[L]): A bound predicate. """ return visit_bound_predicate(predicate, self) @@ -394,7 +396,7 @@ def _(expr: BoundNotEqualTo[L], visitor: BoundBooleanExpressionVisitor[T]) -> T: @visit_bound_predicate.register(BoundGreaterThanOrEqual) def _(expr: BoundGreaterThanOrEqual[L], visitor: BoundBooleanExpressionVisitor[T]) -> T: - """Visit a bound GreaterThanOrEqual predicate""" + """Visit a bound GreaterThanOrEqual predicate.""" return visitor.visit_greater_than_or_equal(term=expr.term, literal=expr.literal) @@ -428,7 +430,7 @@ def rewrite_not(expr: BooleanExpression) -> BooleanExpression: class _RewriteNotVisitor(BooleanExpressionVisitor[BooleanExpression]): - """Inverts the negations""" + """Inverts the negations.""" def visit_true(self) -> BooleanExpression: return AlwaysTrue() @@ -494,16 +496,20 @@ def visit_not_equal(self, term: BoundTerm[L], literal: Literal[L]) -> bool: return term.eval(self.struct) != literal.value def visit_greater_than_or_equal(self, term: BoundTerm[L], literal: Literal[L]) -> bool: - return term.eval(self.struct) >= literal.value + value = term.eval(self.struct) + return value is not None and value >= literal.value def visit_greater_than(self, term: BoundTerm[L], literal: Literal[L]) -> bool: - return term.eval(self.struct) > literal.value + value = term.eval(self.struct) + return value is not None and value > literal.value def visit_less_than(self, term: BoundTerm[L], literal: Literal[L]) -> bool: - return term.eval(self.struct) < literal.value + value = term.eval(self.struct) + return value is not None and value < literal.value def visit_less_than_or_equal(self, term: BoundTerm[L], literal: Literal[L]) -> bool: - return term.eval(self.struct) <= literal.value + value = term.eval(self.struct) + return value is not None and value <= literal.value def visit_starts_with(self, term: BoundTerm[L], literal: Literal[L]) -> bool: eval_res = term.eval(self.struct) @@ -847,15 +853,15 @@ def inclusive_projection( class _ColumnNameTranslator(BooleanExpressionVisitor[BooleanExpression]): - """Converts the column names with the ones in the actual file + """Converts the column names with the ones in the actual file. Args: - file_schema (Schema): The schema of the file - case_sensitive (bool): Whether to consider case when binding a reference to a field in a schema, defaults to True + file_schema (Schema): The schema of the file. + case_sensitive (bool): Whether to consider case when binding a reference to a field in a schema, defaults to True. Raises: - TypeError: In the case of an UnboundPredicate - ValueError: When a column name cannot be found + TypeError: In the case of an UnboundPredicate. + ValueError: When a column name cannot be found. """ file_schema: Schema @@ -904,7 +910,7 @@ def translate_column_names(expr: BooleanExpression, file_schema: Schema, case_se class _ExpressionFieldIDs(BooleanExpressionVisitor[Set[int]]): - """Extracts the field IDs used in the BooleanExpression""" + """Extracts the field IDs used in the BooleanExpression.""" def visit_true(self) -> Set[int]: return set() @@ -1055,7 +1061,9 @@ def visit_or( def expression_to_plain_format( expressions: Tuple[BooleanExpression, ...], cast_int_to_datetime: bool = False ) -> List[List[Tuple[str, str, Any]]]: - """Formats a Disjunctive Normal Form expression into the format that can be fed into: + """Formats a Disjunctive Normal Form expression. + + These are the formats that the expression can be fed into: - https://arrow.apache.org/docs/python/generated/pyarrow.parquet.read_table.html - https://docs.dask.org/en/stable/generated/dask.dataframe.read_parquet.html @@ -1067,10 +1075,10 @@ def expression_to_plain_format( on a row level. Args: - expressions: Expression in Disjunctive Normal Form + expressions: Expression in Disjunctive Normal Form. Returns: - Formatter filter compatible with Dask and PyArrow + Formatter filter compatible with Dask and PyArrow. """ # In the form of expr1 ∨ expr2 ∨ ... ∨ exprN visitor = ExpressionToPlainFormat(cast_int_to_datetime) @@ -1093,7 +1101,6 @@ def __init__(self, schema: Schema, expr: BooleanExpression, case_sensitive: bool def eval(self, file: DataFile) -> bool: """Test whether the file may contain records that match the expression.""" - if file.record_count == 0: return ROWS_CANNOT_MATCH diff --git a/python/pyiceberg/files.py b/python/pyiceberg/files.py index df19f37e0c40..2f8dedf0f0dc 100644 --- a/python/pyiceberg/files.py +++ b/python/pyiceberg/files.py @@ -18,7 +18,7 @@ class FileContentType(Enum): - """An enum that includes all possible content types for an Iceberg data file""" + """An enum that includes all possible content types for an Iceberg data file.""" DATA = auto() POSITION_DELETES = auto() @@ -26,7 +26,7 @@ class FileContentType(Enum): class FileFormat(Enum): - """An enum that includes all possible formats for an Iceberg data file""" + """An enum that includes all possible formats for an Iceberg data file.""" ORC = "ORC" PARQUET = "PARQUET" diff --git a/python/pyiceberg/io/__init__.py b/python/pyiceberg/io/__init__.py index 86213ef911c6..b611a91d24c8 100644 --- a/python/pyiceberg/io/__init__.py +++ b/python/pyiceberg/io/__init__.py @@ -14,7 +14,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -"""Base FileIO classes for implementing reading and writing table files +"""Base FileIO classes for implementing reading and writing table files. The FileIO abstraction includes a subset of full filesystem implementations. Specifically, Iceberg needs to read or write a file at a given location (as a seekable stream), as well @@ -50,11 +50,16 @@ S3_SECRET_ACCESS_KEY = "s3.secret-access-key" S3_SESSION_TOKEN = "s3.session-token" S3_REGION = "s3.region" +S3_PROXY_URI = "s3.proxy-uri" +HDFS_HOST = "hdfs.host" +HDFS_PORT = "hdfs.port" +HDFS_USER = "hdfs.user" +HDFS_KERB_TICKET = "hdfs.kerberos_ticket" @runtime_checkable class InputStream(Protocol): - """A protocol for the file-like object returned by InputFile.open(...) + """A protocol for the file-like object returned by InputFile.open(...). This outlines the minimally required methods for a seekable input stream returned from an InputFile implementation's `open(...)` method. These methods are a subset of IOBase/RawIOBase. @@ -77,18 +82,18 @@ def close(self) -> None: ... def __enter__(self) -> InputStream: - ... + """Provides setup when opening an InputStream using a 'with' statement.""" @abstractmethod def __exit__( self, exctype: Optional[Type[BaseException]], excinst: Optional[BaseException], exctb: Optional[TracebackType] ) -> None: - ... + """Performs cleanup when exiting the scope of a 'with' statement.""" @runtime_checkable class OutputStream(Protocol): # pragma: no cover - """A protocol for the file-like object returned by OutputFile.create(...) + """A protocol for the file-like object returned by OutputFile.create(...). This outlines the minimally required methods for a writable output stream returned from an OutputFile implementation's `create(...)` method. These methods are a subset of IOBase/RawIOBase. @@ -104,24 +109,24 @@ def close(self) -> None: @abstractmethod def __enter__(self) -> OutputStream: - ... + """Provides setup when opening an OutputStream using a 'with' statement.""" @abstractmethod def __exit__( self, exctype: Optional[Type[BaseException]], excinst: Optional[BaseException], exctb: Optional[TracebackType] ) -> None: - ... + """Performs cleanup when exiting the scope of a 'with' statement.""" class InputFile(ABC): - """A base class for InputFile implementations + """A base class for InputFile implementations. Args: - location (str): A URI or a path to a local file + location (str): A URI or a path to a local file. Attributes: - location (str): The URI or path to a local file for an InputFile instance - exists (bool): Whether the file exists or not + location (str): The URI or path to a local file for an InputFile instance. + exists (bool): Whether the file exists or not. """ def __init__(self, location: str): @@ -129,47 +134,46 @@ def __init__(self, location: str): @abstractmethod def __len__(self) -> int: - """Returns the total length of the file, in bytes""" + """Returns the total length of the file, in bytes.""" @property def location(self) -> str: - """The fully-qualified location of the input file""" + """The fully-qualified location of the input file.""" return self._location @abstractmethod def exists(self) -> bool: - """Checks whether the location exists - + """Checks whether the location exists. Raises: - PermissionError: If the file at self.location cannot be accessed due to a permission error + PermissionError: If the file at self.location cannot be accessed due to a permission error. """ @abstractmethod def open(self, seekable: bool = True) -> InputStream: - """This method should return an object that matches the InputStream protocol + """This method should return an object that matches the InputStream protocol. Args: - seekable: If the stream should support seek, or if it is consumed sequential + seekable: If the stream should support seek, or if it is consumed sequential. Returns: - InputStream: An object that matches the InputStream protocol + InputStream: An object that matches the InputStream protocol. Raises: - PermissionError: If the file at self.location cannot be accessed due to a permission error - FileNotFoundError: If the file at self.location does not exist + PermissionError: If the file at self.location cannot be accessed due to a permission error. + FileNotFoundError: If the file at self.location does not exist. """ class OutputFile(ABC): - """A base class for OutputFile implementations + """A base class for OutputFile implementations. Args: - location (str): A URI or a path to a local file + location (str): A URI or a path to a local file. Attributes: - location (str): The URI or path to a local file for an OutputFile instance - exists (bool): Whether the file exists or not + location (str): The URI or path to a local file for an OutputFile instance. + exists (bool): Whether the file exists or not. """ def __init__(self, location: str): @@ -177,25 +181,24 @@ def __init__(self, location: str): @abstractmethod def __len__(self) -> int: - """Returns the total length of the file, in bytes""" + """Returns the total length of the file, in bytes.""" @property def location(self) -> str: - """The fully-qualified location of the output file""" + """The fully-qualified location of the output file.""" return self._location @abstractmethod def exists(self) -> bool: - """Checks whether the location exists - + """Checks whether the location exists. Raises: - PermissionError: If the file at self.location cannot be accessed due to a permission error + PermissionError: If the file at self.location cannot be accessed due to a permission error. """ @abstractmethod def to_input_file(self) -> InputFile: - """Returns an InputFile for the location of this output file""" + """Returns an InputFile for the location of this output file.""" @abstractmethod def create(self, overwrite: bool = False) -> OutputStream: @@ -203,19 +206,19 @@ def create(self, overwrite: bool = False) -> OutputStream: Args: overwrite (bool): If the file already exists at `self.location` - and `overwrite` is False a FileExistsError should be raised + and `overwrite` is False a FileExistsError should be raised. Returns: - OutputStream: An object that matches the OutputStream protocol + OutputStream: An object that matches the OutputStream protocol. Raises: - PermissionError: If the file at self.location cannot be accessed due to a permission error - FileExistsError: If the file at self.location already exists and `overwrite=False` + PermissionError: If the file at self.location cannot be accessed due to a permission error. + FileExistsError: If the file at self.location already exists and `overwrite=False`. """ class FileIO(ABC): - """A base class for FileIO implementations""" + """A base class for FileIO implementations.""" properties: Properties @@ -224,31 +227,31 @@ def __init__(self, properties: Properties = EMPTY_DICT): @abstractmethod def new_input(self, location: str) -> InputFile: - """Get an InputFile instance to read bytes from the file at the given location + """Get an InputFile instance to read bytes from the file at the given location. Args: - location (str): A URI or a path to a local file + location (str): A URI or a path to a local file. """ @abstractmethod def new_output(self, location: str) -> OutputFile: - """Get an OutputFile instance to write bytes to the file at the given location + """Get an OutputFile instance to write bytes to the file at the given location. Args: - location (str): A URI or a path to a local file + location (str): A URI or a path to a local file. """ @abstractmethod def delete(self, location: Union[str, InputFile, OutputFile]) -> None: - """Delete the file at the given path + """Delete the file at the given path. Args: location (Union[str, InputFile, OutputFile]): A URI or a path to a local file--if an InputFile instance or - an OutputFile instance is provided, the location attribute for that instance is used as the URI to delete + an OutputFile instance is provided, the location attribute for that instance is used as the URI to delete. Raises: - PermissionError: If the file at location cannot be accessed due to a permission error - FileNotFoundError: When the file at the provided location does not exist + PermissionError: If the file at location cannot be accessed due to a permission error. + FileNotFoundError: When the file at the provided location does not exist. """ @@ -282,7 +285,7 @@ def _import_file_io(io_impl: str, properties: Properties) -> Optional[FileIO]: class_ = getattr(module, class_name) return class_(properties) except ModuleNotFoundError: - warnings.warn(f"Could not initialize FileIO: {io_impl}") + logger.warning("Could not initialize FileIO: %s", io_impl) return None diff --git a/python/pyiceberg/io/fsspec.py b/python/pyiceberg/io/fsspec.py index b1859758d221..65472904d049 100644 --- a/python/pyiceberg/io/fsspec.py +++ b/python/pyiceberg/io/fsspec.py @@ -14,7 +14,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -"""FileIO implementation for reading and writing table files that uses fsspec compatible filesystems""" +"""FileIO implementation for reading and writing table files that uses fsspec compatible filesystems.""" import errno import logging import os @@ -39,6 +39,7 @@ from pyiceberg.io import ( S3_ACCESS_KEY_ID, S3_ENDPOINT, + S3_PROXY_URI, S3_REGION, S3_SECRET_ACCESS_KEY, S3_SESSION_TOKEN, @@ -112,6 +113,9 @@ def _s3(properties: Properties) -> AbstractFileSystem: else: raise ValueError(f"Signer not available: {signer}") + if proxy_uri := properties.get(S3_PROXY_URI): + config_kwargs["proxies"] = {"http": proxy_uri, "https": proxy_uri} + fs = S3FileSystem(client_kwargs=client_kwargs, config_kwargs=config_kwargs) for event_name, event_function in register_events.items(): @@ -145,11 +149,11 @@ def _adlfs(properties: Properties) -> AbstractFileSystem: class FsspecInputFile(InputFile): - """An input file implementation for the FsspecFileIO + """An input file implementation for the FsspecFileIO. Args: - location (str): A URI to a file location - fs (AbstractFileSystem): An fsspec filesystem instance + location (str): A URI to a file location. + fs (AbstractFileSystem): An fsspec filesystem instance. """ def __init__(self, location: str, fs: AbstractFileSystem): @@ -157,7 +161,7 @@ def __init__(self, location: str, fs: AbstractFileSystem): super().__init__(location=location) def __len__(self) -> int: - """Returns the total length of the file, in bytes""" + """Returns the total length of the file, in bytes.""" object_info = self._fs.info(self.location) if size := object_info.get("Size"): return size @@ -166,20 +170,20 @@ def __len__(self) -> int: raise RuntimeError(f"Cannot retrieve object info: {self.location}") def exists(self) -> bool: - """Checks whether the location exists""" + """Checks whether the location exists.""" return self._fs.lexists(self.location) def open(self, seekable: bool = True) -> InputStream: - """Create an input stream for reading the contents of the file + """Create an input stream for reading the contents of the file. Args: - seekable: If the stream should support seek, or if it is consumed sequential + seekable: If the stream should support seek, or if it is consumed sequential. Returns: - OpenFile: An fsspec compliant file-like object + OpenFile: An fsspec compliant file-like object. Raises: - FileNotFoundError: If the file does not exist + FileNotFoundError: If the file does not exist. """ try: return self._fs.open(self.location, "rb") @@ -189,11 +193,11 @@ def open(self, seekable: bool = True) -> InputStream: class FsspecOutputFile(OutputFile): - """An output file implementation for the FsspecFileIO + """An output file implementation for the FsspecFileIO. Args: - location (str): A URI to a file location - fs (AbstractFileSystem): An fsspec filesystem instance + location (str): A URI to a file location. + fs (AbstractFileSystem): An fsspec filesystem instance. """ def __init__(self, location: str, fs: AbstractFileSystem): @@ -201,7 +205,7 @@ def __init__(self, location: str, fs: AbstractFileSystem): super().__init__(location=location) def __len__(self) -> int: - """Returns the total length of the file, in bytes""" + """Returns the total length of the file, in bytes.""" object_info = self._fs.info(self.location) if size := object_info.get("Size"): return size @@ -210,20 +214,20 @@ def __len__(self) -> int: raise RuntimeError(f"Cannot retrieve object info: {self.location}") def exists(self) -> bool: - """Checks whether the location exists""" + """Checks whether the location exists.""" return self._fs.lexists(self.location) def create(self, overwrite: bool = False) -> OutputStream: - """Create an output stream for reading the contents of the file + """Create an output stream for reading the contents of the file. Args: - overwrite (bool): Whether to overwrite the file if it already exists + overwrite (bool): Whether to overwrite the file if it already exists. Returns: - OpenFile: An fsspec compliant file-like object + OpenFile: An fsspec compliant file-like object. Raises: - FileExistsError: If the file already exists at the location and overwrite is set to False + FileExistsError: If the file already exists at the location and overwrite is set to False. Note: If overwrite is set to False, a check is first performed to verify that the file does not exist. @@ -236,12 +240,12 @@ def create(self, overwrite: bool = False) -> OutputStream: return self._fs.open(self.location, "wb") def to_input_file(self) -> FsspecInputFile: - """Returns a new FsspecInputFile for the location at `self.location`""" + """Returns a new FsspecInputFile for the location at `self.location`.""" return FsspecInputFile(location=self.location, fs=self._fs) class FsspecFileIO(FileIO): - """A FileIO implementation that uses fsspec""" + """A FileIO implementation that uses fsspec.""" def __init__(self, properties: Properties): self._scheme_to_fs = {} @@ -250,38 +254,38 @@ def __init__(self, properties: Properties): super().__init__(properties=properties) def new_input(self, location: str) -> FsspecInputFile: - """Get an FsspecInputFile instance to read bytes from the file at the given location + """Get an FsspecInputFile instance to read bytes from the file at the given location. Args: - location (str): A URI or a path to a local file + location (str): A URI or a path to a local file. Returns: - FsspecInputFile: An FsspecInputFile instance for the given location + FsspecInputFile: An FsspecInputFile instance for the given location. """ uri = urlparse(location) fs = self.get_fs(uri.scheme) return FsspecInputFile(location=location, fs=fs) def new_output(self, location: str) -> FsspecOutputFile: - """Get an FsspecOutputFile instance to write bytes to the file at the given location + """Get an FsspecOutputFile instance to write bytes to the file at the given location. Args: - location (str): A URI or a path to a local file + location (str): A URI or a path to a local file. Returns: - FsspecOutputFile: An FsspecOutputFile instance for the given location + FsspecOutputFile: An FsspecOutputFile instance for the given location. """ uri = urlparse(location) fs = self.get_fs(uri.scheme) return FsspecOutputFile(location=location, fs=fs) def delete(self, location: Union[str, InputFile, OutputFile]) -> None: - """Delete the file at the given location + """Delete the file at the given location. Args: location (Union[str, InputFile, OutputFile]): The URI to the file--if an InputFile instance or an OutputFile instance is provided, the location attribute for that instance is used as the location - to delete + to delete. """ if isinstance(location, (InputFile, OutputFile)): str_location = location.location # Use InputFile or OutputFile location @@ -293,7 +297,7 @@ def delete(self, location: Union[str, InputFile, OutputFile]) -> None: fs.rm(str_location) def _get_fs(self, scheme: str) -> AbstractFileSystem: - """Get a filesystem for a specific scheme""" + """Get a filesystem for a specific scheme.""" if scheme not in self._scheme_to_fs: raise ValueError(f"No registered filesystem for scheme: {scheme}") return self._scheme_to_fs[scheme](self.properties) diff --git a/python/pyiceberg/io/memory.py b/python/pyiceberg/io/memory.py deleted file mode 100644 index a4eb89d76f8a..000000000000 --- a/python/pyiceberg/io/memory.py +++ /dev/null @@ -1,83 +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. -from __future__ import annotations - -from io import SEEK_CUR, SEEK_END, SEEK_SET -from types import TracebackType -from typing import Optional, Type - -from pyiceberg.io import InputStream - - -class MemoryInputStream(InputStream): - """ - Simple in memory stream that we use to store decompressed blocks - - Examples: - >>> stream = MemoryInputStream(b'22memory1925') - >>> stream.tell() - 0 - >>> stream.read(2) - b'22' - >>> stream.tell() - 2 - >>> stream.seek(8) - >>> stream.read(4) - b'1925' - >>> stream.close() - """ - - buffer: bytes - len: int - pos: int - - def __init__(self, buffer: bytes): - self.buffer = buffer - self.len = len(buffer) - self.pos = 0 - - def read(self, size: int = 0) -> bytes: - b = self.buffer[self.pos : self.pos + size] - self.pos += size - return b - - def seek(self, offset: int, whence: int = SEEK_SET) -> int: - if whence == SEEK_SET: - self.pos = offset - elif whence == SEEK_CUR: - self.pos += offset - elif whence == SEEK_END: - self.pos = self.len + offset - else: - raise ValueError(f"Unknown whence {offset}") - - return self.pos - - def tell(self) -> int: - return self.pos - - def close(self) -> None: - del self.buffer - self.pos = 0 - - def __enter__(self) -> MemoryInputStream: - return self - - def __exit__( - self, exctype: Optional[Type[BaseException]], excinst: Optional[BaseException], exctb: Optional[TracebackType] - ) -> None: - self.close() diff --git a/python/pyiceberg/io/pyarrow.py b/python/pyiceberg/io/pyarrow.py index 88920000f517..425507200321 100644 --- a/python/pyiceberg/io/pyarrow.py +++ b/python/pyiceberg/io/pyarrow.py @@ -15,7 +15,7 @@ # specific language governing permissions and limitations # under the License. # pylint: disable=redefined-outer-name,arguments-renamed,fixme -"""FileIO implementation for reading and writing table files that uses pyarrow.fs +"""FileIO implementation for reading and writing table files that uses pyarrow.fs. This file contains a FileIO implementation that relies on the filesystem interface provided by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem @@ -24,18 +24,19 @@ """ from __future__ import annotations -import multiprocessing import os from abc import ABC, abstractmethod +from concurrent.futures import Executor from functools import lru_cache, singledispatch -from multiprocessing.pool import ThreadPool -from multiprocessing.sharedctypes import Synchronized +from itertools import chain from typing import ( TYPE_CHECKING, Any, Callable, + Dict, Generic, Iterable, + Iterator, List, Optional, Set, @@ -46,14 +47,17 @@ ) from urllib.parse import urlparse +import numpy as np import pyarrow as pa import pyarrow.compute as pc import pyarrow.dataset as ds +from pyarrow import ChunkedArray from pyarrow.fs import ( FileInfo, FileSystem, FileType, FSSpecHandler, + HadoopFileSystem, LocalFileSystem, PyFileSystem, S3FileSystem, @@ -74,8 +78,13 @@ ) from pyiceberg.expressions.visitors import visit as boolean_expression_visit from pyiceberg.io import ( + HDFS_HOST, + HDFS_KERB_TICKET, + HDFS_PORT, + HDFS_USER, S3_ACCESS_KEY_ID, S3_ENDPOINT, + S3_PROXY_URI, S3_REGION, S3_SECRET_ACCESS_KEY, S3_SESSION_TOKEN, @@ -85,6 +94,7 @@ OutputFile, OutputStream, ) +from pyiceberg.manifest import DataFile, FileFormat from pyiceberg.schema import ( PartnerAccessor, Schema, @@ -118,6 +128,7 @@ TimeType, UUIDType, ) +from pyiceberg.utils.concurrent import ManagedThreadPoolExecutor, Synchronized from pyiceberg.utils.singleton import Singleton if TYPE_CHECKING: @@ -135,13 +146,13 @@ class PyArrowFile(InputFile, OutputFile): - """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances + """A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances. Args: - location (str): A URI or a path to a local file + location (str): A URI or a path to a local file. Attributes: - location(str): The URI or path to a local file for a PyArrowFile instance + location(str): The URI or path to a local file for a PyArrowFile instance. Examples: >>> from pyiceberg.io.pyarrow import PyArrowFile @@ -167,11 +178,11 @@ def __init__(self, location: str, path: str, fs: FileSystem, buffer_size: int = super().__init__(location=location) def _file_info(self) -> FileInfo: - """Retrieves a pyarrow.fs.FileInfo object for the location + """Retrieves a pyarrow.fs.FileInfo object for the location. Raises: PermissionError: If the file at self.location cannot be accessed due to a permission error such as - an AWS error code 15 + an AWS error code 15. """ try: file_info = self._filesystem.get_file_info(self._path) @@ -185,12 +196,12 @@ def _file_info(self) -> FileInfo: return file_info def __len__(self) -> int: - """Returns the total length of the file, in bytes""" + """Returns the total length of the file, in bytes.""" file_info = self._file_info() return file_info.size def exists(self) -> bool: - """Checks whether the location exists""" + """Checks whether the location exists.""" try: self._file_info() # raises FileNotFoundError if it does not exist return True @@ -198,18 +209,18 @@ def exists(self) -> bool: return False def open(self, seekable: bool = True) -> InputStream: - """Opens the location using a PyArrow FileSystem inferred from the location + """Opens the location using a PyArrow FileSystem inferred from the location. Args: - seekable: If the stream should support seek, or if it is consumed sequential + seekable: If the stream should support seek, or if it is consumed sequential. Returns: - pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location` + pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`. Raises: - FileNotFoundError: If the file at self.location does not exist + FileNotFoundError: If the file at self.location does not exist. PermissionError: If the file at self.location cannot be accessed due to a permission error such as - an AWS error code 15 + an AWS error code 15. """ try: if seekable: @@ -229,16 +240,16 @@ def open(self, seekable: bool = True) -> InputStream: return input_file def create(self, overwrite: bool = False) -> OutputStream: - """Creates a writable pyarrow.lib.NativeFile for this PyArrowFile's location + """Creates a writable pyarrow.lib.NativeFile for this PyArrowFile's location. Args: - overwrite (bool): Whether to overwrite the file if it already exists + overwrite (bool): Whether to overwrite the file if it already exists. Returns: - pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location + pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location. Raises: - FileExistsError: If the file already exists at `self.location` and `overwrite` is False + FileExistsError: If the file already exists at `self.location` and `overwrite` is False. Note: This retrieves a pyarrow NativeFile by opening an output stream. If overwrite is set to False, @@ -260,7 +271,7 @@ def create(self, overwrite: bool = False) -> OutputStream: return output_file def to_input_file(self) -> PyArrowFile: - """Returns a new PyArrowFile for the location of an existing PyArrowFile instance + """Returns a new PyArrowFile for the location of an existing PyArrowFile instance. This method is included to abide by the OutputFile abstract base class. Since this implementation uses a single PyArrowFile class (as opposed to separate InputFile and OutputFile implementations), this method effectively returns @@ -276,7 +287,7 @@ def __init__(self, properties: Properties = EMPTY_DICT): @staticmethod def parse_location(location: str) -> Tuple[str, str]: - """Returns the path without the scheme""" + """Returns the path without the scheme.""" uri = urlparse(location) return uri.scheme or "file", os.path.abspath(location) if not uri.scheme else f"{uri.netloc}{uri.path}" @@ -289,49 +300,61 @@ def _get_fs(self, scheme: str) -> FileSystem: "session_token": self.properties.get(S3_SESSION_TOKEN), "region": self.properties.get(S3_REGION), } + + if proxy_uri := self.properties.get(S3_PROXY_URI): + client_kwargs["proxy_options"] = proxy_uri + return S3FileSystem(**client_kwargs) + elif scheme == "hdfs": + client_kwargs = { + "host": self.properties.get(HDFS_HOST), + "port": self.properties.get(HDFS_PORT), + "user": self.properties.get(HDFS_USER), + "kerb_ticket": self.properties.get(HDFS_KERB_TICKET), + } + return HadoopFileSystem(**client_kwargs) elif scheme == "file": return LocalFileSystem() else: raise ValueError(f"Unrecognized filesystem type in URI: {scheme}") def new_input(self, location: str) -> PyArrowFile: - """Get a PyArrowFile instance to read bytes from the file at the given location + """Get a PyArrowFile instance to read bytes from the file at the given location. Args: - location (str): A URI or a path to a local file + location (str): A URI or a path to a local file. Returns: - PyArrowFile: A PyArrowFile instance for the given location + PyArrowFile: A PyArrowFile instance for the given location. """ scheme, path = self.parse_location(location) fs = self._get_fs(scheme) return PyArrowFile(fs=fs, location=location, path=path, buffer_size=int(self.properties.get(BUFFER_SIZE, ONE_MEGABYTE))) def new_output(self, location: str) -> PyArrowFile: - """Get a PyArrowFile instance to write bytes to the file at the given location + """Get a PyArrowFile instance to write bytes to the file at the given location. Args: - location (str): A URI or a path to a local file + location (str): A URI or a path to a local file. Returns: - PyArrowFile: A PyArrowFile instance for the given location + PyArrowFile: A PyArrowFile instance for the given location. """ scheme, path = self.parse_location(location) fs = self._get_fs(scheme) return PyArrowFile(fs=fs, location=location, path=path, buffer_size=int(self.properties.get(BUFFER_SIZE, ONE_MEGABYTE))) def delete(self, location: Union[str, InputFile, OutputFile]) -> None: - """Delete the file at the given location + """Delete the file at the given location. Args: location (Union[str, InputFile, OutputFile]): The URI to the file--if an InputFile instance or an OutputFile instance is provided, - the location attribute for that instance is used as the location to delete + the location attribute for that instance is used as the location to delete. Raises: - FileNotFoundError: When the file at the provided location does not exist + FileNotFoundError: When the file at the provided location does not exist. PermissionError: If the file at the provided location cannot be accessed due to a permission error such as - an AWS error code 15 + an AWS error code 15. """ str_location = location.location if isinstance(location, (InputFile, OutputFile)) else location scheme, path = self.parse_location(str_location) @@ -412,7 +435,7 @@ def visit_time(self, _: TimeType) -> pa.DataType: def visit_timestamp(self, _: TimestampType) -> pa.DataType: return pa.timestamp(unit="us") - def visit_timestampz(self, _: TimestamptzType) -> pa.DataType: + def visit_timestamptz(self, _: TimestamptzType) -> pa.DataType: return pa.timestamp(unit="us", tz="UTC") def visit_string(self, _: StringType) -> pa.DataType: @@ -498,23 +521,56 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression: return boolean_expression_visit(expr, _ConvertToArrowExpression()) +@lru_cache +def _get_file_format(file_format: FileFormat, **kwargs: Dict[str, Any]) -> ds.FileFormat: + if file_format == FileFormat.PARQUET: + return ds.ParquetFileFormat(**kwargs) + else: + raise ValueError(f"Unsupported file format: {file_format}") + + +def _construct_fragment(fs: FileSystem, data_file: DataFile, file_format_kwargs: Dict[str, Any] = EMPTY_DICT) -> ds.Fragment: + _, path = PyArrowFileIO.parse_location(data_file.file_path) + return _get_file_format(data_file.file_format, **file_format_kwargs).make_fragment(path, fs) + + +def _read_deletes(fs: FileSystem, data_file: DataFile) -> Dict[str, pa.ChunkedArray]: + delete_fragment = _construct_fragment( + fs, data_file, file_format_kwargs={"dictionary_columns": ("file_path",), "pre_buffer": True, "buffer_size": ONE_MEGABYTE} + ) + table = ds.Scanner.from_fragment(fragment=delete_fragment).to_table() + table = table.unify_dictionaries() + return { + file.as_py(): table.filter(pc.field("file_path") == file).column("pos") + for file in table.column("file_path").chunks[0].dictionary + } + + +def _combine_positional_deletes(positional_deletes: List[pa.ChunkedArray], rows: int) -> pa.Array: + if len(positional_deletes) == 1: + all_chunks = positional_deletes[0] + else: + all_chunks = pa.chunked_array(chain(*[arr.chunks for arr in positional_deletes])) + return np.setdiff1d(np.arange(rows), all_chunks, assume_unique=False) + + def pyarrow_to_schema(schema: pa.Schema) -> Schema: visitor = _ConvertToIceberg() return visit_pyarrow(schema, visitor) @singledispatch -def visit_pyarrow(obj: pa.DataType | pa.Schema, visitor: PyArrowSchemaVisitor[T]) -> T: - """A generic function for applying a pyarrow schema visitor to any point within a schema +def visit_pyarrow(obj: Union[pa.DataType, pa.Schema], visitor: PyArrowSchemaVisitor[T]) -> T: + """A generic function for applying a pyarrow schema visitor to any point within a schema. - The function traverses the schema in post-order fashion + The function traverses the schema in post-order fashion. Args: - obj(pa.DataType): An instance of a Schema or an IcebergType - visitor (PyArrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class + obj (Union[pa.DataType, pa.Schema]): An instance of a Schema or an IcebergType. + visitor (PyArrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class. Raises: - NotImplementedError: If attempting to visit an unrecognized object type + NotImplementedError: If attempting to visit an unrecognized object type. """ raise NotImplementedError("Cannot visit non-type: %s" % obj) @@ -578,23 +634,23 @@ def after_field(self, field: pa.Field) -> None: @abstractmethod def schema(self, schema: pa.Schema, field_results: List[Optional[T]]) -> Optional[T]: - """visit a schema""" + """Visit a schema.""" @abstractmethod def struct(self, struct: pa.StructType, field_results: List[Optional[T]]) -> Optional[T]: - """visit a struct""" + """Visit a struct.""" @abstractmethod def list(self, list_type: pa.ListType, element_result: Optional[T]) -> Optional[T]: - """visit a list""" + """Visit a list.""" @abstractmethod def map(self, map_type: pa.MapType, key_result: Optional[T], value_result: Optional[T]) -> Optional[T]: - """visit a map""" + """Visit a map.""" @abstractmethod def primitive(self, primitive: pa.DataType) -> Optional[T]: - """visit a primitive type""" + """Visit a primitive type.""" def _get_field_id(field: pa.Field) -> Optional[int]: @@ -682,12 +738,13 @@ def primitive(self, primitive: pa.DataType) -> IcebergType: raise TypeError(f"Unsupported type: {primitive}") -def _file_to_table( +def _task_to_table( fs: FileSystem, task: FileScanTask, bound_row_filter: BooleanExpression, projected_schema: Schema, projected_field_ids: Set[int], + positional_deletes: Optional[List[ChunkedArray]], case_sensitive: bool, rows_counter: Synchronized[int], limit: Optional[int] = None, @@ -721,18 +778,44 @@ def _file_to_table( fragment_scanner = ds.Scanner.from_fragment( fragment=fragment, schema=physical_schema, - filter=pyarrow_filter, + # This will push down the query to Arrow. + # But in case there are positional deletes, we have to apply them first + filter=pyarrow_filter if not positional_deletes else None, columns=[col.name for col in file_project_schema.columns], ) + if positional_deletes: + # Create the mask of indices that we're interested in + indices = _combine_positional_deletes(positional_deletes, fragment.count_rows()) + + if limit: + if pyarrow_filter is not None: + # In case of the filter, we don't exactly know how many rows + # we need to fetch upfront, can be optimized in the future: + # https://github.com/apache/arrow/issues/35301 + arrow_table = fragment_scanner.take(indices) + arrow_table = arrow_table.filter(pyarrow_filter) + arrow_table = arrow_table.slice(0, limit) + else: + arrow_table = fragment_scanner.take(indices[0:limit]) + else: + arrow_table = fragment_scanner.take(indices) + # Apply the user filter + if pyarrow_filter is not None: + arrow_table = arrow_table.filter(pyarrow_filter) + else: + # If there are no deletes, we can just take the head + # and the user-filter is already applied + if limit: + arrow_table = fragment_scanner.head(limit) + else: + arrow_table = fragment_scanner.to_table() + if limit: - arrow_table = fragment_scanner.head(limit) - with rows_counter.get_lock(): + with rows_counter: if rows_counter.value >= limit: return None rows_counter.value += len(arrow_table) - else: - arrow_table = fragment_scanner.to_table() # If there is no data, we don't have to go through the schema if len(arrow_table) > 0: @@ -741,27 +824,44 @@ def _file_to_table( return None +def _read_all_delete_files(fs: FileSystem, executor: Executor, tasks: Iterable[FileScanTask]) -> Dict[str, List[ChunkedArray]]: + deletes_per_file: Dict[str, List[ChunkedArray]] = {} + unique_deletes = set(chain.from_iterable([task.delete_files for task in tasks])) + if len(unique_deletes) > 0: + deletes_per_files: Iterator[Dict[str, ChunkedArray]] = executor.map( + lambda args: _read_deletes(*args), [(fs, delete) for delete in unique_deletes] + ) + for delete in deletes_per_files: + for file, arr in delete.items(): + if file in deletes_per_file: + deletes_per_file[file].append(arr) + else: + deletes_per_file[file] = [arr] + + return deletes_per_file + + def project_table( tasks: Iterable[FileScanTask], table: Table, row_filter: BooleanExpression, projected_schema: Schema, - case_sensitive: bool, + case_sensitive: bool = True, limit: Optional[int] = None, ) -> pa.Table: - """Resolves the right columns based on the identifier + """Resolves the right columns based on the identifier. Args: - tasks (Iterable[FileScanTask]): A URI or a path to a local file - table (Table): The table that's being queried - row_filter (BooleanExpression): The expression for filtering rows - projected_schema (Schema): The output schema - case_sensitive (bool): Case sensitivity when looking up column names + tasks (Iterable[FileScanTask]): A URI or a path to a local file. + table (Table): The table that's being queried. + row_filter (BooleanExpression): The expression for filtering rows. + projected_schema (Schema): The output schema. + case_sensitive (bool): Case sensitivity when looking up column names. + limit (Optional[int]): Limit the number of records. Raises: - ResolveError: When an incompatible query is done + ResolveError: When an incompatible query is done. """ - scheme, _ = PyArrowFileIO.parse_location(table.location()) if isinstance(table.io, PyArrowFileIO): fs = table.io.get_fs(scheme) @@ -783,18 +883,27 @@ def project_table( id for id in projected_schema.field_ids if not isinstance(projected_schema.find_type(id), (MapType, ListType)) }.union(extract_field_ids(bound_row_filter)) - rows_counter = multiprocessing.Value("i", 0) - - with ThreadPool() as pool: + with ManagedThreadPoolExecutor() as executor: + rows_counter = executor.synchronized(0) + deletes_per_file = _read_all_delete_files(fs, executor, tasks) tables = [ table - for table in pool.starmap( - func=_file_to_table, - iterable=[ - (fs, task, bound_row_filter, projected_schema, projected_field_ids, case_sensitive, rows_counter, limit) + for table in executor.map( + lambda args: _task_to_table(*args), + [ + ( + fs, + task, + bound_row_filter, + projected_schema, + projected_field_ids, + deletes_per_file.get(task.file.file_path), + case_sensitive, + rows_counter, + limit, + ) for task in tasks ], - chunksize=None, # we could use this to control how to materialize the generator of tasks (we should also make the expression above lazy) ) if table is not None ] diff --git a/python/pyiceberg/manifest.py b/python/pyiceberg/manifest.py index ea1d6a611d84..879872f892db 100644 --- a/python/pyiceberg/manifest.py +++ b/python/pyiceberg/manifest.py @@ -46,6 +46,7 @@ class DataFileContent(int, Enum): EQUALITY_DELETES = 2 def __repr__(self) -> str: + """Returns the string representation of the DataFileContent class.""" return f"DataFileContent.{self.name}" @@ -54,6 +55,7 @@ class ManifestContent(int, Enum): DELETES = 1 def __repr__(self) -> str: + """Returns the string representation of the ManifestContent class.""" return f"ManifestContent.{self.name}" @@ -63,6 +65,7 @@ class ManifestEntryStatus(int, Enum): DELETED = 2 def __repr__(self) -> str: + """Returns the string representation of the ManifestEntryStatus class.""" return f"ManifestEntryStatus.{self.name}" @@ -72,6 +75,7 @@ class FileFormat(str, Enum): ORC = "ORC" def __repr__(self) -> str: + """Returns the string representation of the FileFormat class.""" return f"FileFormat.{self.name}" @@ -82,6 +86,7 @@ def __repr__(self) -> str: field_type=IntegerType(), required=False, doc="Contents of the file: 0=data, 1=position deletes, 2=equality deletes", + initial_default=DataFileContent.DATA, ), NestedField(field_id=100, name="file_path", field_type=StringType(), required=True, doc="Location URI with FS scheme"), NestedField( @@ -100,14 +105,14 @@ def __repr__(self) -> str: field_id=108, name="column_sizes", field_type=MapType(key_id=117, key_type=IntegerType(), value_id=118, value_type=LongType()), - required=True, + required=False, doc="Map of column id to total size on disk", ), NestedField( field_id=109, name="value_counts", field_type=MapType(key_id=119, key_type=IntegerType(), value_id=120, value_type=LongType()), - required=True, + required=False, doc="Map of column id to total count, including null and NaN", ), NestedField( @@ -159,7 +164,26 @@ def __repr__(self) -> str: class DataFile(Record): - content: Optional[DataFileContent] + __slots__ = ( + "content", + "file_path", + "file_format", + "partition", + "record_count", + "file_size_in_bytes", + "column_sizes", + "value_counts", + "null_value_counts", + "nan_value_counts", + "lower_bounds", + "upper_bounds", + "key_metadata", + "split_offsets", + "equality_ids", + "sort_order_id", + "spec_id", + ) + content: DataFileContent file_path: str file_format: FileFormat partition: Record @@ -178,6 +202,7 @@ class DataFile(Record): spec_id: Optional[int] def __setattr__(self, name: str, value: Any) -> None: + """Used for assigning a key/value to a DataFile.""" # The file_format is written as a string, so we need to cast it to the Enum if name == "file_format": value = FileFormat[value] @@ -186,25 +211,39 @@ def __setattr__(self, name: str, value: Any) -> None: def __init__(self, *data: Any, **named_data: Any) -> None: super().__init__(*data, **{"struct": DATA_FILE_TYPE, **named_data}) + def __hash__(self) -> int: + """Returns the hash of the file path.""" + return hash(self.file_path) + + def __eq__(self, other: Any) -> bool: + """Compares the datafile with another object. + + If it is a datafile, it will compare based on the file_path. + """ + return self.file_path == other.file_path if isinstance(other, DataFile) else False + MANIFEST_ENTRY_SCHEMA = Schema( NestedField(0, "status", IntegerType(), required=True), NestedField(1, "snapshot_id", LongType(), required=False), - NestedField(3, "sequence_number", LongType(), required=False), + NestedField(3, "data_sequence_number", LongType(), required=False), NestedField(4, "file_sequence_number", LongType(), required=False), - NestedField(2, "data_file", DATA_FILE_TYPE, required=False), + NestedField(2, "data_file", DATA_FILE_TYPE, required=True), ) +MANIFEST_ENTRY_SCHEMA_STRUCT = MANIFEST_ENTRY_SCHEMA.as_struct() + class ManifestEntry(Record): + __slots__ = ("status", "snapshot_id", "data_sequence_number", "file_sequence_number", "data_file") status: ManifestEntryStatus snapshot_id: Optional[int] - sequence_number: Optional[int] + data_sequence_number: Optional[int] file_sequence_number: Optional[int] data_file: DataFile def __init__(self, *data: Any, **named_data: Any) -> None: - super().__init__(*data, **{"struct": MANIFEST_ENTRY_SCHEMA.as_struct(), **named_data}) + super().__init__(*data, **{"struct": MANIFEST_ENTRY_SCHEMA_STRUCT, **named_data}) PARTITION_FIELD_SUMMARY_TYPE = StructType( @@ -216,6 +255,7 @@ def __init__(self, *data: Any, **named_data: Any) -> None: class PartitionFieldSummary(Record): + __slots__ = ("contains_null", "contains_nan", "lower_bound", "upper_bound") contains_null: bool contains_nan: Optional[bool] lower_bound: Optional[bytes] @@ -243,15 +283,38 @@ def __init__(self, *data: Any, **named_data: Any) -> None: NestedField(519, "key_metadata", BinaryType(), required=False), ) +MANIFEST_FILE_SCHEMA_STRUCT = MANIFEST_FILE_SCHEMA.as_struct() + +POSITIONAL_DELETE_SCHEMA = Schema( + NestedField(2147483546, "file_path", StringType()), NestedField(2147483545, "pos", IntegerType()) +) + class ManifestFile(Record): + __slots__ = ( + "manifest_path", + "manifest_length", + "partition_spec_id", + "content", + "sequence_number", + "min_sequence_number", + "added_snapshot_id", + "added_files_count", + "existing_files_count", + "deleted_files_count", + "added_rows_count", + "existing_rows_count", + "deleted_rows_count", + "partitions", + "key_metadata", + ) manifest_path: str manifest_length: int partition_spec_id: int - content: Optional[ManifestContent] - sequence_number: Optional[int] - min_sequence_number: Optional[int] - added_snapshot_id: Optional[int] + content: ManifestContent + sequence_number: int + min_sequence_number: int + added_snapshot_id: int added_files_count: Optional[int] existing_files_count: Optional[int] deleted_files_count: Optional[int] @@ -262,28 +325,83 @@ class ManifestFile(Record): key_metadata: Optional[bytes] def __init__(self, *data: Any, **named_data: Any) -> None: - super().__init__(*data, **{"struct": MANIFEST_FILE_SCHEMA.as_struct(), **named_data}) + super().__init__(*data, **{"struct": MANIFEST_FILE_SCHEMA_STRUCT, **named_data}) + + def has_added_files(self) -> bool: + return self.added_files_count is None or self.added_files_count > 0 + + def has_existing_files(self) -> bool: + return self.existing_files_count is None or self.existing_files_count > 0 + + def fetch_manifest_entry(self, io: FileIO, discard_deleted: bool = True) -> List[ManifestEntry]: + """ + Reads the manifest entries from the manifest file. + + Args: + io: The FileIO to fetch the file. + discard_deleted: Filter on live entries. + + Returns: + An Iterator of manifest entries. + """ + input_file = io.new_input(self.manifest_path) + with AvroFile[ManifestEntry]( + input_file, + MANIFEST_ENTRY_SCHEMA, + read_types={-1: ManifestEntry, 2: DataFile}, + read_enums={0: ManifestEntryStatus, 101: FileFormat, 134: DataFileContent}, + ) as reader: + return [ + _inherit_sequence_number(entry, self) + for entry in reader + if not discard_deleted or entry.status != ManifestEntryStatus.DELETED + ] - def fetch_manifest_entry(self, io: FileIO) -> List[ManifestEntry]: - file = io.new_input(self.manifest_path) - return list(read_manifest_entry(file)) +def read_manifest_list(input_file: InputFile) -> Iterator[ManifestFile]: + """ + Reads the manifests from the manifest list. + + Args: + input_file: The input file where the stream can be read from. -def read_manifest_entry(input_file: InputFile) -> Iterator[ManifestEntry]: - with AvroFile[ManifestEntry](input_file, MANIFEST_ENTRY_SCHEMA, {-1: ManifestEntry, 2: DataFile}) as reader: + Returns: + An iterator of ManifestFiles that are part of the list. + """ + with AvroFile[ManifestFile]( + input_file, + MANIFEST_FILE_SCHEMA, + read_types={-1: ManifestFile, 508: PartitionFieldSummary}, + read_enums={517: ManifestContent}, + ) as reader: yield from reader -def live_entries(input_file: InputFile) -> Iterator[ManifestEntry]: - return (entry for entry in read_manifest_entry(input_file) if entry.status != ManifestEntryStatus.DELETED) +def _inherit_sequence_number(entry: ManifestEntry, manifest: ManifestFile) -> ManifestEntry: + """Inherits the sequence numbers. + More information in the spec: https://iceberg.apache.org/spec/#sequence-number-inheritance -def files(input_file: InputFile) -> Iterator[DataFile]: - return (entry.data_file for entry in live_entries(input_file)) + Args: + entry: The manifest entry that has null sequence numbers. + manifest: The manifest that has a sequence number. + Returns: + The manifest entry with the sequence numbers set. + """ + # The snapshot_id is required in V1, inherit with V2 when null + if entry.snapshot_id is None: + entry.snapshot_id = manifest.added_snapshot_id -def read_manifest_list(input_file: InputFile) -> Iterator[ManifestFile]: - with AvroFile[ManifestFile]( - input_file, MANIFEST_FILE_SCHEMA, {-1: ManifestFile, 508: PartitionFieldSummary}, {517: ManifestContent} - ) as reader: - yield from reader + # in v1 tables, the data sequence number is not persisted and can be safely defaulted to 0 + # in v2 tables, the data sequence number should be inherited iff the entry status is ADDED + if entry.data_sequence_number is None and (manifest.sequence_number == 0 or entry.status == ManifestEntryStatus.ADDED): + entry.data_sequence_number = manifest.sequence_number + + # in v1 tables, the file sequence number is not persisted and can be safely defaulted to 0 + # in v2 tables, the file sequence number should be inherited iff the entry status is ADDED + if entry.file_sequence_number is None and (manifest.sequence_number == 0 or entry.status == ManifestEntryStatus.ADDED): + # Only available in V2, always 0 in V1 + entry.file_sequence_number = manifest.sequence_number + + return entry diff --git a/python/pyiceberg/partitioning.py b/python/pyiceberg/partitioning.py index 9313b1606c7a..935dbea68fe1 100644 --- a/python/pyiceberg/partitioning.py +++ b/python/pyiceberg/partitioning.py @@ -35,15 +35,13 @@ class PartitionField(IcebergBaseModel): - """ - PartitionField is a single element with name and unique id, - It represents how one partition value is derived from the source column via transformation + """PartitionField represents how one partition value is derived from the source column via transformation. Attributes: - source_id(int): The source column id of table's schema - field_id(int): The partition field id across all the table partition specs - transform(Transform): The transform used to produce partition values from source column - name(str): The name of this partition field + source_id(int): The source column id of table's schema. + field_id(int): The partition field id across all the table partition specs. + transform(Transform): The transform used to produce partition values from source column. + name(str): The name of this partition field. """ source_id: int = Field(alias="source-id") @@ -70,16 +68,17 @@ def __init__( super().__init__(**data) def __str__(self) -> str: + """Returns the string representation of the PartitionField class.""" return f"{self.field_id}: {self.name}: {self.transform}({self.source_id})" class PartitionSpec(IcebergBaseModel): """ - PartitionSpec captures the transformation from table data to partition values + PartitionSpec captures the transformation from table data to partition values. Attributes: - spec_id(int): any change to PartitionSpec will produce a new specId - fields(Tuple[PartitionField): list of partition fields to produce partition values + spec_id(int): any change to PartitionSpec will produce a new specId. + fields(Tuple[PartitionField): list of partition fields to produce partition values. """ spec_id: int = Field(alias="spec-id", default=INITIAL_PARTITION_SPEC_ID) @@ -96,10 +95,10 @@ def __init__( def __eq__(self, other: Any) -> bool: """ - Produce a boolean to return True if two objects are considered equal + Produce a boolean to return True if two objects are considered equal. Note: - Equality of PartitionSpec is determined by spec_id and partition fields only + Equality of PartitionSpec is determined by spec_id and partition fields only. """ if not isinstance(other, PartitionSpec): return False @@ -107,10 +106,10 @@ def __eq__(self, other: Any) -> bool: def __str__(self) -> str: """ - Produce a human-readable string representation of PartitionSpec + Produce a human-readable string representation of PartitionSpec. Note: - Only include list of partition fields in the PartitionSpec's string representation + Only include list of partition fields in the PartitionSpec's string representation. """ result_str = "[" if self.fields: @@ -119,6 +118,7 @@ def __str__(self) -> str: return result_str def __repr__(self) -> str: + """Returns the string representation of the PartitionSpec class.""" fields = f"{', '.join(repr(column) for column in self.fields)}, " if self.fields else "" return f"PartitionSpec({fields}spec_id={self.spec_id})" @@ -144,9 +144,7 @@ def fields_by_source_id(self, field_id: int) -> List[PartitionField]: return self.source_id_to_fields_map.get(field_id, []) def compatible_with(self, other: "PartitionSpec") -> bool: - """ - Produce a boolean to return True if two PartitionSpec are considered compatible - """ + """Produce a boolean to return True if two PartitionSpec are considered compatible.""" if self == other: return True if len(self.fields) != len(other.fields): @@ -159,20 +157,20 @@ def compatible_with(self, other: "PartitionSpec") -> bool: ) def partition_type(self, schema: Schema) -> StructType: - """Produces a struct of the PartitionSpec + """Produces a struct of the PartitionSpec. The partition fields should be optional: - All partition transforms are required to produce null if the input value is null, so it can - happen when the source column is optional + happen when the source column is optional. - Partition fields may be added later, in which case not all files would have the result field, and it may be null. There is a case where we can guarantee that a partition field in the first and only partition spec that uses a required source column will never be null, but it doesn't seem worth tracking this case. - :param schema: The schema to bind to - :return: A StructType that represents the PartitionSpec, with a NestedField for each PartitionField + :param schema: The schema to bind to. + :return: A StructType that represents the PartitionSpec, with a NestedField for each PartitionField. """ nested_fields = [] for field in self.fields: diff --git a/python/pyiceberg/schema.py b/python/pyiceberg/schema.py index c3ff9abaf9d9..32a8370ac80f 100644 --- a/python/pyiceberg/schema.py +++ b/python/pyiceberg/schema.py @@ -67,7 +67,7 @@ class Schema(IcebergBaseModel): - """A table Schema + """A table Schema. Example: >>> from pyiceberg import schema @@ -88,15 +88,19 @@ def __init__(self, *fields: NestedField, **data: Any): self._name_to_id = index_by_name(self) def __str__(self) -> str: + """Returns the string representation of the Schema class.""" return "table {\n" + "\n".join([" " + str(field) for field in self.columns]) + "\n}" def __repr__(self) -> str: + """Returns the string representation of the Schema class.""" return f"Schema({', '.join(repr(column) for column in self.columns)}, schema_id={self.schema_id}, identifier_field_ids={self.identifier_field_ids})" def __len__(self) -> int: + """Returns the length of an instance of the Literal class.""" return len(self.fields) def __eq__(self, other: Any) -> bool: + """Returns the equality of two instances of the Schema class.""" if not other: return False @@ -113,12 +117,12 @@ def __eq__(self, other: Any) -> bool: @property def columns(self) -> Tuple[NestedField, ...]: - """A tuple of the top-level fields""" + """A tuple of the top-level fields.""" return self.fields @cached_property def _lazy_id_to_field(self) -> Dict[int, NestedField]: - """Returns an index of field ID to NestedField instance + """Returns an index of field ID to NestedField instance. This is calculated once when called for the first time. Subsequent calls to this method will use a cached index. """ @@ -126,7 +130,7 @@ def _lazy_id_to_field(self) -> Dict[int, NestedField]: @cached_property def _lazy_name_to_id_lower(self) -> Dict[str, int]: - """Returns an index of lower-case field names to field IDs + """Returns an index of lower-case field names to field IDs. This is calculated once when called for the first time. Subsequent calls to this method will use a cached index. """ @@ -134,7 +138,7 @@ def _lazy_name_to_id_lower(self) -> Dict[str, int]: @cached_property def _lazy_id_to_name(self) -> Dict[int, str]: - """Returns an index of field ID to full name + """Returns an index of field ID to full name. This is calculated once when called for the first time. Subsequent calls to this method will use a cached index. """ @@ -142,28 +146,28 @@ def _lazy_id_to_name(self) -> Dict[int, str]: @cached_property def _lazy_id_to_accessor(self) -> Dict[int, "Accessor"]: - """Returns an index of field ID to accessor + """Returns an index of field ID to accessor. This is calculated once when called for the first time. Subsequent calls to this method will use a cached index. """ return build_position_accessors(self) def as_struct(self) -> StructType: - """Returns the schema as a struct""" + """Returns the schema as a struct.""" return StructType(*self.fields) def find_field(self, name_or_id: Union[str, int], case_sensitive: bool = True) -> NestedField: - """Find a field using a field name or field ID + """Find a field using a field name or field ID. Args: - name_or_id (Union[str, int]): Either a field name or a field ID + name_or_id (Union[str, int]): Either a field name or a field ID. case_sensitive (bool, optional): Whether to perform a case-sensitive lookup using a field name. Defaults to True. Raises: - ValueError: When the value cannot be found + ValueError: When the value cannot be found. Returns: - NestedField: The matched NestedField + NestedField: The matched NestedField. """ if isinstance(name_or_id, int): if name_or_id not in self._lazy_id_to_field: @@ -181,14 +185,14 @@ def find_field(self, name_or_id: Union[str, int], case_sensitive: bool = True) - return self._lazy_id_to_field[field_id] def find_type(self, name_or_id: Union[str, int], case_sensitive: bool = True) -> IcebergType: - """Find a field type using a field name or field ID + """Find a field type using a field name or field ID. Args: - name_or_id (Union[str, int]): Either a field name or a field ID + name_or_id (Union[str, int]): Either a field name or a field ID. case_sensitive (bool, optional): Whether to perform a case-sensitive lookup using a field name. Defaults to True. Returns: - NestedField: The type of the matched NestedField + NestedField: The type of the matched NestedField. """ field = self.find_field(name_or_id=name_or_id, case_sensitive=case_sensitive) if not field: @@ -200,60 +204,58 @@ def highest_field_id(self) -> int: return visit(self.as_struct(), _FindLastFieldId()) def find_column_name(self, column_id: int) -> Optional[str]: - """Find a column name given a column ID + """Find a column name given a column ID. Args: - column_id (int): The ID of the column + column_id (int): The ID of the column. Returns: - str: The column name (or None if the column ID cannot be found) + str: The column name (or None if the column ID cannot be found). """ return self._lazy_id_to_name.get(column_id) @property def column_names(self) -> List[str]: """ - Returns a list of all the column names, including nested fields + Returns a list of all the column names, including nested fields. - Excludes short names + Excludes short names. Returns: - List[str]: The column names + List[str]: The column names. """ return list(self._lazy_id_to_name.values()) def accessor_for_field(self, field_id: int) -> "Accessor": - """Find a schema position accessor given a field ID + """Find a schema position accessor given a field ID. Args: - field_id (int): The ID of the field + field_id (int): The ID of the field. Raises: - ValueError: When the value cannot be found + ValueError: When the value cannot be found. Returns: - Accessor: An accessor for the given field ID + Accessor: An accessor for the given field ID. """ - if field_id not in self._lazy_id_to_accessor: raise ValueError(f"Could not find accessor for field with id: {field_id}") return self._lazy_id_to_accessor[field_id] def select(self, *names: str, case_sensitive: bool = True) -> "Schema": - """Return a new schema instance pruned to a subset of columns + """Return a new schema instance pruned to a subset of columns. Args: - names (List[str]): A list of column names + names (List[str]): A list of column names. case_sensitive (bool, optional): Whether to perform a case-sensitive lookup for each column name. Defaults to True. Returns: - Schema: A new schema with pruned columns + Schema: A new schema with pruned columns. Raises: - ValueError: If a column is selected that doesn't exist + ValueError: If a column is selected that doesn't exist. """ - try: if case_sensitive: ids = {self._name_to_id[name] for name in names} @@ -266,151 +268,151 @@ def select(self, *names: str, case_sensitive: bool = True) -> "Schema": @property def field_ids(self) -> Set[int]: - """Returns the IDs of the current schema""" + """Returns the IDs of the current schema.""" return set(self._name_to_id.values()) class SchemaVisitor(Generic[T], ABC): def before_field(self, field: NestedField) -> None: - """Override this method to perform an action immediately before visiting a field""" + """Override this method to perform an action immediately before visiting a field.""" def after_field(self, field: NestedField) -> None: - """Override this method to perform an action immediately after visiting a field""" + """Override this method to perform an action immediately after visiting a field.""" def before_list_element(self, element: NestedField) -> None: - """Override this method to perform an action immediately before visiting an element within a ListType""" + """Override this method to perform an action immediately before visiting an element within a ListType.""" self.before_field(element) def after_list_element(self, element: NestedField) -> None: - """Override this method to perform an action immediately after visiting an element within a ListType""" + """Override this method to perform an action immediately after visiting an element within a ListType.""" self.after_field(element) def before_map_key(self, key: NestedField) -> None: - """Override this method to perform an action immediately before visiting a key within a MapType""" + """Override this method to perform an action immediately before visiting a key within a MapType.""" self.before_field(key) def after_map_key(self, key: NestedField) -> None: - """Override this method to perform an action immediately after visiting a key within a MapType""" + """Override this method to perform an action immediately after visiting a key within a MapType.""" self.after_field(key) def before_map_value(self, value: NestedField) -> None: - """Override this method to perform an action immediately before visiting a value within a MapType""" + """Override this method to perform an action immediately before visiting a value within a MapType.""" self.before_field(value) def after_map_value(self, value: NestedField) -> None: - """Override this method to perform an action immediately after visiting a value within a MapType""" + """Override this method to perform an action immediately after visiting a value within a MapType.""" self.after_field(value) @abstractmethod def schema(self, schema: Schema, struct_result: T) -> T: - """Visit a Schema""" + """Visit a Schema.""" @abstractmethod def struct(self, struct: StructType, field_results: List[T]) -> T: - """Visit a StructType""" + """Visit a StructType.""" @abstractmethod def field(self, field: NestedField, field_result: T) -> T: - """Visit a NestedField""" + """Visit a NestedField.""" @abstractmethod def list(self, list_type: ListType, element_result: T) -> T: - """Visit a ListType""" + """Visit a ListType.""" @abstractmethod def map(self, map_type: MapType, key_result: T, value_result: T) -> T: - """Visit a MapType""" + """Visit a MapType.""" @abstractmethod def primitive(self, primitive: PrimitiveType) -> T: - """Visit a PrimitiveType""" + """Visit a PrimitiveType.""" class PreOrderSchemaVisitor(Generic[T], ABC): @abstractmethod def schema(self, schema: Schema, struct_result: Callable[[], T]) -> T: - """Visit a Schema""" + """Visit a Schema.""" @abstractmethod def struct(self, struct: StructType, field_results: List[Callable[[], T]]) -> T: - """Visit a StructType""" + """Visit a StructType.""" @abstractmethod def field(self, field: NestedField, field_result: Callable[[], T]) -> T: - """Visit a NestedField""" + """Visit a NestedField.""" @abstractmethod def list(self, list_type: ListType, element_result: Callable[[], T]) -> T: - """Visit a ListType""" + """Visit a ListType.""" @abstractmethod def map(self, map_type: MapType, key_result: Callable[[], T], value_result: Callable[[], T]) -> T: - """Visit a MapType""" + """Visit a MapType.""" @abstractmethod def primitive(self, primitive: PrimitiveType) -> T: - """Visit a PrimitiveType""" + """Visit a PrimitiveType.""" class SchemaWithPartnerVisitor(Generic[P, T], ABC): def before_field(self, field: NestedField, field_partner: Optional[P]) -> None: - """Override this method to perform an action immediately before visiting a field""" + """Override this method to perform an action immediately before visiting a field.""" def after_field(self, field: NestedField, field_partner: Optional[P]) -> None: - """Override this method to perform an action immediately after visiting a field""" + """Override this method to perform an action immediately after visiting a field.""" def before_list_element(self, element: NestedField, element_partner: Optional[P]) -> None: - """Override this method to perform an action immediately before visiting an element within a ListType""" + """Override this method to perform an action immediately before visiting an element within a ListType.""" self.before_field(element, element_partner) def after_list_element(self, element: NestedField, element_partner: Optional[P]) -> None: - """Override this method to perform an action immediately after visiting an element within a ListType""" + """Override this method to perform an action immediately after visiting an element within a ListType.""" self.after_field(element, element_partner) def before_map_key(self, key: NestedField, key_partner: Optional[P]) -> None: - """Override this method to perform an action immediately before visiting a key within a MapType""" + """Override this method to perform an action immediately before visiting a key within a MapType.""" self.before_field(key, key_partner) def after_map_key(self, key: NestedField, key_partner: Optional[P]) -> None: - """Override this method to perform an action immediately after visiting a key within a MapType""" + """Override this method to perform an action immediately after visiting a key within a MapType.""" self.after_field(key, key_partner) def before_map_value(self, value: NestedField, value_partner: Optional[P]) -> None: - """Override this method to perform an action immediately before visiting a value within a MapType""" + """Override this method to perform an action immediately before visiting a value within a MapType.""" self.before_field(value, value_partner) def after_map_value(self, value: NestedField, value_partner: Optional[P]) -> None: - """Override this method to perform an action immediately after visiting a value within a MapType""" + """Override this method to perform an action immediately after visiting a value within a MapType.""" self.after_field(value, value_partner) @abstractmethod def schema(self, schema: Schema, schema_partner: Optional[P], struct_result: T) -> T: - """Visit a schema with a partner""" + """Visit a schema with a partner.""" @abstractmethod def struct(self, struct: StructType, struct_partner: Optional[P], field_results: List[T]) -> T: - """Visit a struct type with a partner""" + """Visit a struct type with a partner.""" @abstractmethod def field(self, field: NestedField, field_partner: Optional[P], field_result: T) -> T: - """Visit a nested field with a partner""" + """Visit a nested field with a partner.""" @abstractmethod def list(self, list_type: ListType, list_partner: Optional[P], element_result: T) -> T: - """Visit a list type with a partner""" + """Visit a list type with a partner.""" @abstractmethod def map(self, map_type: MapType, map_partner: Optional[P], key_result: T, value_result: T) -> T: - """Visit a map type with a partner""" + """Visit a map type with a partner.""" @abstractmethod def primitive(self, primitive: PrimitiveType, primitive_partner: Optional[P]) -> T: - """Visit a primitive type with a partner""" + """Visit a primitive type with a partner.""" class PrimitiveWithPartnerVisitor(SchemaWithPartnerVisitor[P, T]): def primitive(self, primitive: PrimitiveType, primitive_partner: Optional[P]) -> T: - """Visit a PrimitiveType""" + """Visit a PrimitiveType.""" if isinstance(primitive, BooleanType): return self.visit_boolean(primitive, primitive_partner) elif isinstance(primitive, IntegerType): @@ -430,7 +432,7 @@ def primitive(self, primitive: PrimitiveType, primitive_partner: Optional[P]) -> elif isinstance(primitive, TimestampType): return self.visit_timestamp(primitive, primitive_partner) elif isinstance(primitive, TimestamptzType): - return self.visit_timestampz(primitive, primitive_partner) + return self.visit_timestamptz(primitive, primitive_partner) elif isinstance(primitive, StringType): return self.visit_string(primitive, primitive_partner) elif isinstance(primitive, UUIDType): @@ -444,81 +446,81 @@ def primitive(self, primitive: PrimitiveType, primitive_partner: Optional[P]) -> @abstractmethod def visit_boolean(self, boolean_type: BooleanType, partner: Optional[P]) -> T: - """Visit a BooleanType""" + """Visit a BooleanType.""" @abstractmethod def visit_integer(self, integer_type: IntegerType, partner: Optional[P]) -> T: - """Visit a IntegerType""" + """Visit a IntegerType.""" @abstractmethod def visit_long(self, long_type: LongType, partner: Optional[P]) -> T: - """Visit a LongType""" + """Visit a LongType.""" @abstractmethod def visit_float(self, float_type: FloatType, partner: Optional[P]) -> T: - """Visit a FloatType""" + """Visit a FloatType.""" @abstractmethod def visit_double(self, double_type: DoubleType, partner: Optional[P]) -> T: - """Visit a DoubleType""" + """Visit a DoubleType.""" @abstractmethod def visit_decimal(self, decimal_type: DecimalType, partner: Optional[P]) -> T: - """Visit a DecimalType""" + """Visit a DecimalType.""" @abstractmethod def visit_date(self, date_type: DateType, partner: Optional[P]) -> T: - """Visit a DecimalType""" + """Visit a DecimalType.""" @abstractmethod def visit_time(self, time_type: TimeType, partner: Optional[P]) -> T: - """Visit a DecimalType""" + """Visit a DecimalType.""" @abstractmethod def visit_timestamp(self, timestamp_type: TimestampType, partner: Optional[P]) -> T: - """Visit a TimestampType""" + """Visit a TimestampType.""" @abstractmethod - def visit_timestampz(self, timestamptz_type: TimestamptzType, partner: Optional[P]) -> T: - """Visit a TimestamptzType""" + def visit_timestamptz(self, timestamptz_type: TimestamptzType, partner: Optional[P]) -> T: + """Visit a TimestamptzType.""" @abstractmethod def visit_string(self, string_type: StringType, partner: Optional[P]) -> T: - """Visit a StringType""" + """Visit a StringType.""" @abstractmethod def visit_uuid(self, uuid_type: UUIDType, partner: Optional[P]) -> T: - """Visit a UUIDType""" + """Visit a UUIDType.""" @abstractmethod def visit_fixed(self, fixed_type: FixedType, partner: Optional[P]) -> T: - """Visit a FixedType""" + """Visit a FixedType.""" @abstractmethod def visit_binary(self, binary_type: BinaryType, partner: Optional[P]) -> T: - """Visit a BinaryType""" + """Visit a BinaryType.""" class PartnerAccessor(Generic[P], ABC): @abstractmethod def schema_partner(self, partner: Optional[P]) -> Optional[P]: - """Returns the equivalent of the schema as a struct""" + """Returns the equivalent of the schema as a struct.""" @abstractmethod def field_partner(self, partner_struct: Optional[P], field_id: int, field_name: str) -> Optional[P]: - """Returns the equivalent struct field by name or id in the partner struct""" + """Returns the equivalent struct field by name or id in the partner struct.""" @abstractmethod def list_element_partner(self, partner_list: Optional[P]) -> Optional[P]: - """Returns the equivalent list element in the partner list""" + """Returns the equivalent list element in the partner list.""" @abstractmethod def map_key_partner(self, partner_map: Optional[P]) -> Optional[P]: - """Returns the equivalent map key in the partner map""" + """Returns the equivalent map key in the partner map.""" @abstractmethod def map_value_partner(self, partner_map: Optional[P]) -> Optional[P]: - """Returns the equivalent map value in the partner map""" + """Returns the equivalent map value in the partner map.""" @singledispatch @@ -586,7 +588,7 @@ def _(primitive: PrimitiveType, partner: P, visitor: SchemaWithPartnerVisitor[P, class SchemaVisitorPerPrimitiveType(SchemaVisitor[T], ABC): def primitive(self, primitive: PrimitiveType) -> T: - """Visit a PrimitiveType""" + """Visit a PrimitiveType.""" if isinstance(primitive, FixedType): return self.visit_fixed(primitive) elif isinstance(primitive, DecimalType): @@ -608,7 +610,7 @@ def primitive(self, primitive: PrimitiveType) -> T: elif isinstance(primitive, TimestampType): return self.visit_timestamp(primitive) elif isinstance(primitive, TimestamptzType): - return self.visit_timestampz(primitive) + return self.visit_timestamptz(primitive) elif isinstance(primitive, StringType): return self.visit_string(primitive) elif isinstance(primitive, UUIDType): @@ -620,82 +622,84 @@ def primitive(self, primitive: PrimitiveType) -> T: @abstractmethod def visit_fixed(self, fixed_type: FixedType) -> T: - """Visit a FixedType""" + """Visit a FixedType.""" @abstractmethod def visit_decimal(self, decimal_type: DecimalType) -> T: - """Visit a DecimalType""" + """Visit a DecimalType.""" @abstractmethod def visit_boolean(self, boolean_type: BooleanType) -> T: - """Visit a BooleanType""" + """Visit a BooleanType.""" @abstractmethod def visit_integer(self, integer_type: IntegerType) -> T: - """Visit a IntegerType""" + """Visit a IntegerType.""" @abstractmethod def visit_long(self, long_type: LongType) -> T: - """Visit a LongType""" + """Visit a LongType.""" @abstractmethod def visit_float(self, float_type: FloatType) -> T: - """Visit a FloatType""" + """Visit a FloatType.""" @abstractmethod def visit_double(self, double_type: DoubleType) -> T: - """Visit a DoubleType""" + """Visit a DoubleType.""" @abstractmethod def visit_date(self, date_type: DateType) -> T: - """Visit a DecimalType""" + """Visit a DecimalType.""" @abstractmethod def visit_time(self, time_type: TimeType) -> T: - """Visit a DecimalType""" + """Visit a DecimalType.""" @abstractmethod def visit_timestamp(self, timestamp_type: TimestampType) -> T: - """Visit a TimestampType""" + """Visit a TimestampType.""" @abstractmethod - def visit_timestampz(self, timestamptz_type: TimestamptzType) -> T: - """Visit a TimestamptzType""" + def visit_timestamptz(self, timestamptz_type: TimestamptzType) -> T: + """Visit a TimestamptzType.""" @abstractmethod def visit_string(self, string_type: StringType) -> T: - """Visit a StringType""" + """Visit a StringType.""" @abstractmethod def visit_uuid(self, uuid_type: UUIDType) -> T: - """Visit a UUIDType""" + """Visit a UUIDType.""" @abstractmethod def visit_binary(self, binary_type: BinaryType) -> T: - """Visit a BinaryType""" + """Visit a BinaryType.""" @dataclass(init=True, eq=True, frozen=True) class Accessor: - """An accessor for a specific position in a container that implements the StructProtocol""" + """An accessor for a specific position in a container that implements the StructProtocol.""" position: int inner: Optional["Accessor"] = None def __str__(self) -> str: + """Returns the string representation of the Accessor class.""" return f"Accessor(position={self.position},inner={self.inner})" def __repr__(self) -> str: + """Returns the string representation of the Accessor class.""" return self.__str__() def get(self, container: StructProtocol) -> Any: - """Returns the value at self.position in `container` + """Returns the value at self.position in `container`. Args: - container (StructProtocol): A container to access at position `self.position` + container (StructProtocol): A container to access at position `self.position`. Returns: - Any: The value at position `self.position` in the container + Any: The value at position `self.position` in the container. """ pos = self.position val = container[pos] @@ -709,29 +713,29 @@ def get(self, container: StructProtocol) -> Any: @singledispatch def visit(obj: Union[Schema, IcebergType], visitor: SchemaVisitor[T]) -> T: - """A generic function for applying a schema visitor to any point within a schema + """A generic function for applying a schema visitor to any point within a schema. - The function traverses the schema in post-order fashion + The function traverses the schema in post-order fashion. Args: - obj (Union[Schema, IcebergType]): An instance of a Schema or an IcebergType - visitor (SchemaVisitor[T]): An instance of an implementation of the generic SchemaVisitor base class + obj (Union[Schema, IcebergType]): An instance of a Schema or an IcebergType. + visitor (SchemaVisitor[T]): An instance of an implementation of the generic SchemaVisitor base class. Raises: - NotImplementedError: If attempting to visit an unrecognized object type + NotImplementedError: If attempting to visit an unrecognized object type. """ raise NotImplementedError("Cannot visit non-type: %s" % obj) @visit.register(Schema) def _(obj: Schema, visitor: SchemaVisitor[T]) -> T: - """Visit a Schema with a concrete SchemaVisitor""" + """Visit a Schema with a concrete SchemaVisitor.""" return visitor.schema(obj, visit(obj.as_struct(), visitor)) @visit.register(StructType) def _(obj: StructType, visitor: SchemaVisitor[T]) -> T: - """Visit a StructType with a concrete SchemaVisitor""" + """Visit a StructType with a concrete SchemaVisitor.""" results = [] for field in obj.fields: @@ -745,8 +749,7 @@ def _(obj: StructType, visitor: SchemaVisitor[T]) -> T: @visit.register(ListType) def _(obj: ListType, visitor: SchemaVisitor[T]) -> T: - """Visit a ListType with a concrete SchemaVisitor""" - + """Visit a ListType with a concrete SchemaVisitor.""" visitor.before_list_element(obj.element_field) result = visit(obj.element_type, visitor) visitor.after_list_element(obj.element_field) @@ -756,7 +759,7 @@ def _(obj: ListType, visitor: SchemaVisitor[T]) -> T: @visit.register(MapType) def _(obj: MapType, visitor: SchemaVisitor[T]) -> T: - """Visit a MapType with a concrete SchemaVisitor""" + """Visit a MapType with a concrete SchemaVisitor.""" visitor.before_map_key(obj.key_field) key_result = visit(obj.key_type, visitor) visitor.after_map_key(obj.key_field) @@ -770,37 +773,37 @@ def _(obj: MapType, visitor: SchemaVisitor[T]) -> T: @visit.register(PrimitiveType) def _(obj: PrimitiveType, visitor: SchemaVisitor[T]) -> T: - """Visit a PrimitiveType with a concrete SchemaVisitor""" + """Visit a PrimitiveType with a concrete SchemaVisitor.""" return visitor.primitive(obj) @singledispatch def pre_order_visit(obj: Union[Schema, IcebergType], visitor: PreOrderSchemaVisitor[T]) -> T: - """A generic function for applying a schema visitor to any point within a schema + """A generic function for applying a schema visitor to any point within a schema. The function traverses the schema in pre-order fashion. This is a slimmed down version compared to the post-order traversal (missing before and after methods), mostly because we don't use the pre-order traversal much. Args: - obj (Union[Schema, IcebergType]): An instance of a Schema or an IcebergType - visitor (PreOrderSchemaVisitor[T]): An instance of an implementation of the generic PreOrderSchemaVisitor base class + obj (Union[Schema, IcebergType]): An instance of a Schema or an IcebergType. + visitor (PreOrderSchemaVisitor[T]): An instance of an implementation of the generic PreOrderSchemaVisitor base class. Raises: - NotImplementedError: If attempting to visit an unrecognized object type + NotImplementedError: If attempting to visit an unrecognized object type. """ raise NotImplementedError("Cannot visit non-type: %s" % obj) @pre_order_visit.register(Schema) def _(obj: Schema, visitor: PreOrderSchemaVisitor[T]) -> T: - """Visit a Schema with a concrete PreOrderSchemaVisitor""" + """Visit a Schema with a concrete PreOrderSchemaVisitor.""" return visitor.schema(obj, lambda: pre_order_visit(obj.as_struct(), visitor)) @pre_order_visit.register(StructType) def _(obj: StructType, visitor: PreOrderSchemaVisitor[T]) -> T: - """Visit a StructType with a concrete PreOrderSchemaVisitor""" + """Visit a StructType with a concrete PreOrderSchemaVisitor.""" return visitor.struct( obj, [ @@ -815,24 +818,24 @@ def _(obj: StructType, visitor: PreOrderSchemaVisitor[T]) -> T: @pre_order_visit.register(ListType) def _(obj: ListType, visitor: PreOrderSchemaVisitor[T]) -> T: - """Visit a ListType with a concrete PreOrderSchemaVisitor""" + """Visit a ListType with a concrete PreOrderSchemaVisitor.""" return visitor.list(obj, lambda: pre_order_visit(obj.element_type, visitor)) @pre_order_visit.register(MapType) def _(obj: MapType, visitor: PreOrderSchemaVisitor[T]) -> T: - """Visit a MapType with a concrete PreOrderSchemaVisitor""" + """Visit a MapType with a concrete PreOrderSchemaVisitor.""" return visitor.map(obj, lambda: pre_order_visit(obj.key_type, visitor), lambda: pre_order_visit(obj.value_type, visitor)) @pre_order_visit.register(PrimitiveType) def _(obj: PrimitiveType, visitor: PreOrderSchemaVisitor[T]) -> T: - """Visit a PrimitiveType with a concrete PreOrderSchemaVisitor""" + """Visit a PrimitiveType with a concrete PreOrderSchemaVisitor.""" return visitor.primitive(obj) class _IndexById(SchemaVisitor[Dict[int, NestedField]]): - """A schema visitor for generating a field ID to NestedField index""" + """A schema visitor for generating a field ID to NestedField index.""" def __init__(self) -> None: self._index: Dict[int, NestedField] = {} @@ -844,19 +847,19 @@ def struct(self, struct: StructType, field_results: List[Dict[int, NestedField]] return self._index def field(self, field: NestedField, field_result: Dict[int, NestedField]) -> Dict[int, NestedField]: - """Add the field ID to the index""" + """Add the field ID to the index.""" self._index[field.field_id] = field return self._index def list(self, list_type: ListType, element_result: Dict[int, NestedField]) -> Dict[int, NestedField]: - """Add the list element ID to the index""" + """Add the list element ID to the index.""" self._index[list_type.element_field.field_id] = list_type.element_field return self._index def map( self, map_type: MapType, key_result: Dict[int, NestedField], value_result: Dict[int, NestedField] ) -> Dict[int, NestedField]: - """Add the key ID and value ID as individual items in the index""" + """Add the key ID and value ID as individual items in the index.""" self._index[map_type.key_field.field_id] = map_type.key_field self._index[map_type.value_field.field_id] = map_type.value_field return self._index @@ -866,19 +869,19 @@ def primitive(self, primitive: PrimitiveType) -> Dict[int, NestedField]: def index_by_id(schema_or_type: Union[Schema, IcebergType]) -> Dict[int, NestedField]: - """Generate an index of field IDs to NestedField instances + """Generate an index of field IDs to NestedField instances. Args: - schema_or_type (Union[Schema, IcebergType]): A schema or type to index + schema_or_type (Union[Schema, IcebergType]): A schema or type to index. Returns: - Dict[int, NestedField]: An index of field IDs to NestedField instances + Dict[int, NestedField]: An index of field IDs to NestedField instances. """ return visit(schema_or_type, _IndexById()) class _IndexByName(SchemaVisitor[Dict[str, int]]): - """A schema visitor for generating a field name to field ID index""" + """A schema visitor for generating a field name to field ID index.""" def __init__(self) -> None: self._index: Dict[str, int] = {} @@ -888,7 +891,7 @@ def __init__(self) -> None: self._short_field_names: List[str] = [] def before_list_element(self, element: NestedField) -> None: - """Short field names omit element when the element is a StructType""" + """Short field names omit element when the element is a StructType.""" if not isinstance(element.field_type, StructType): self._short_field_names.append(element.name) self._field_names.append(element.name) @@ -899,12 +902,12 @@ def after_list_element(self, element: NestedField) -> None: self._field_names.pop() def before_field(self, field: NestedField) -> None: - """Store the field name""" + """Store the field name.""" self._field_names.append(field.name) self._short_field_names.append(field.name) def after_field(self, field: NestedField) -> None: - """Remove the last field name stored""" + """Remove the last field name stored.""" self._field_names.pop() self._short_field_names.pop() @@ -915,30 +918,30 @@ def struct(self, struct: StructType, field_results: List[Dict[str, int]]) -> Dic return self._index def field(self, field: NestedField, field_result: Dict[str, int]) -> Dict[str, int]: - """Add the field name to the index""" + """Add the field name to the index.""" self._add_field(field.name, field.field_id) return self._index def list(self, list_type: ListType, element_result: Dict[str, int]) -> Dict[str, int]: - """Add the list element name to the index""" + """Add the list element name to the index.""" self._add_field(list_type.element_field.name, list_type.element_field.field_id) return self._index def map(self, map_type: MapType, key_result: Dict[str, int], value_result: Dict[str, int]) -> Dict[str, int]: - """Add the key name and value name as individual items in the index""" + """Add the key name and value name as individual items in the index.""" self._add_field(map_type.key_field.name, map_type.key_field.field_id) self._add_field(map_type.value_field.name, map_type.value_field.field_id) return self._index def _add_field(self, name: str, field_id: int) -> None: - """Add a field name to the index, mapping its full name to its field ID + """Add a field name to the index, mapping its full name to its field ID. Args: - name (str): The field name - field_id (int): The field ID + name (str): The field name. + field_id (int): The field ID. Raises: - ValueError: If the field name is already contained in the index + ValueError: If the field name is already contained in the index. """ full_name = name @@ -957,7 +960,7 @@ def primitive(self, primitive: PrimitiveType) -> Dict[str, int]: return self._index def by_name(self) -> Dict[str, int]: - """Returns an index of combined full and short names + """Returns an index of combined full and short names. Note: Only short names that do not conflict with full names are included. """ @@ -966,19 +969,19 @@ def by_name(self) -> Dict[str, int]: return combined_index def by_id(self) -> Dict[int, str]: - """Returns an index of ID to full names""" + """Returns an index of ID to full names.""" id_to_full_name = {value: key for key, value in self._index.items()} return id_to_full_name def index_by_name(schema_or_type: Union[Schema, IcebergType]) -> Dict[str, int]: - """Generate an index of field names to field IDs + """Generate an index of field names to field IDs. Args: - schema_or_type (Union[Schema, IcebergType]): A schema or type to index + schema_or_type (Union[Schema, IcebergType]): A schema or type to index. Returns: - Dict[str, int]: An index of field names to field IDs + Dict[str, int]: An index of field names to field IDs. """ if len(schema_or_type.fields) > 0: indexer = _IndexByName() @@ -989,13 +992,13 @@ def index_by_name(schema_or_type: Union[Schema, IcebergType]) -> Dict[str, int]: def index_name_by_id(schema_or_type: Union[Schema, IcebergType]) -> Dict[int, str]: - """Generate an index of field IDs full field names + """Generate an index of field IDs full field names. Args: - schema_or_type (Union[Schema, IcebergType]): A schema or type to index + schema_or_type (Union[Schema, IcebergType]): A schema or type to index. Returns: - Dict[str, int]: An index of field IDs to full names + Dict[str, int]: An index of field IDs to full names. """ indexer = _IndexByName() visit(schema_or_type, indexer) @@ -1006,7 +1009,7 @@ def index_name_by_id(schema_or_type: Union[Schema, IcebergType]) -> Dict[int, st class _BuildPositionAccessors(SchemaVisitor[Dict[Position, Accessor]]): - """A schema visitor for generating a field ID to accessor index + """A schema visitor for generating a field ID to accessor index. Example: >>> from pyiceberg.schema import Schema @@ -1068,19 +1071,19 @@ def primitive(self, primitive: PrimitiveType) -> Dict[Position, Accessor]: def build_position_accessors(schema_or_type: Union[Schema, IcebergType]) -> Dict[int, Accessor]: - """Generate an index of field IDs to schema position accessors + """Generate an index of field IDs to schema position accessors. Args: - schema_or_type (Union[Schema, IcebergType]): A schema or type to index + schema_or_type (Union[Schema, IcebergType]): A schema or type to index. Returns: - Dict[int, Accessor]: An index of field IDs to accessors + Dict[int, Accessor]: An index of field IDs to accessors. """ return visit(schema_or_type, _BuildPositionAccessors()) class _FindLastFieldId(SchemaVisitor[int]): - """Traverses the schema to get the highest field-id""" + """Traverses the schema to get the highest field-id.""" def schema(self, schema: Schema, struct_result: int) -> int: return struct_result @@ -1102,12 +1105,12 @@ def primitive(self, primitive: PrimitiveType) -> int: def assign_fresh_schema_ids(schema: Schema) -> Schema: - """Traverses the schema, and sets new IDs""" + """Traverses the schema, and sets new IDs.""" return pre_order_visit(schema, _SetFreshIDs()) class _SetFreshIDs(PreOrderSchemaVisitor[IcebergType]): - """Traverses the schema and assigns monotonically increasing ids""" + """Traverses the schema and assigns monotonically increasing ids.""" counter: itertools.count # type: ignore reserved_ids: Dict[int, int] @@ -1306,14 +1309,14 @@ def _project_map(map_type: MapType, value_result: IcebergType) -> MapType: @singledispatch def promote(file_type: IcebergType, read_type: IcebergType) -> IcebergType: - """Promotes reading a file type to a read type + """Promotes reading a file type to a read type. Args: - file_type (IcebergType): The type of the Avro file - read_type (IcebergType): The requested read type + file_type (IcebergType): The type of the Avro file. + read_type (IcebergType): The requested read type. Raises: - ResolveError: If attempting to resolve an unrecognized object type + ResolveError: If attempting to resolve an unrecognized object type. """ if file_type == read_type: return file_type @@ -1364,12 +1367,3 @@ def _(file_type: DecimalType, read_type: IcebergType) -> IcebergType: raise ResolveError(f"Cannot reduce precision from {file_type} to {read_type}") else: raise ResolveError(f"Cannot promote an decimal to {read_type}") - - -@promote.register(FixedType) -def _(file_type: FixedType, read_type: IcebergType) -> IcebergType: - if isinstance(read_type, UUIDType) and len(file_type) == 16: - # Since pyarrow reads parquet UUID as fixed 16-byte binary, the promotion is needed to ensure read compatibility - return read_type - else: - raise ResolveError(f"Cannot promote {file_type} to {read_type}") diff --git a/python/pyiceberg/serializers.py b/python/pyiceberg/serializers.py index 7f4c8842144b..6341c149b2f2 100644 --- a/python/pyiceberg/serializers.py +++ b/python/pyiceberg/serializers.py @@ -14,59 +14,119 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +from __future__ import annotations import codecs +import gzip import json +from abc import ABC, abstractmethod +from typing import Callable from pyiceberg.io import InputFile, InputStream, OutputFile from pyiceberg.table.metadata import TableMetadata, TableMetadataUtil +GZIP = "gzip" + + +class Compressor(ABC): + @staticmethod + def get_compressor(location: str) -> Compressor: + return GzipCompressor() if location.endswith(".gz.metadata.json") else NOOP_COMPRESSOR + + @abstractmethod + def stream_decompressor(self, inp: InputStream) -> InputStream: + """Returns a stream decompressor. + + Args: + inp: The input stream that needs decompressing. + + Returns: + The wrapped stream + """ + + @abstractmethod + def bytes_compressor(self) -> Callable[[bytes], bytes]: + """Returns a function to compress bytes. + + Returns: + A function that can be used to compress bytes. + """ + + +class NoopCompressor(Compressor): + def stream_decompressor(self, inp: InputStream) -> InputStream: + return inp + + def bytes_compressor(self) -> Callable[[bytes], bytes]: + return lambda b: b + + +NOOP_COMPRESSOR = NoopCompressor() + + +class GzipCompressor(Compressor): + def stream_decompressor(self, inp: InputStream) -> InputStream: + return gzip.open(inp) + + def bytes_compressor(self) -> Callable[[bytes], bytes]: + return gzip.compress + class FromByteStream: - """A collection of methods that deserialize dictionaries into Iceberg objects""" + """A collection of methods that deserialize dictionaries into Iceberg objects.""" @staticmethod - def table_metadata(byte_stream: InputStream, encoding: str = "utf-8") -> TableMetadata: - """Instantiate a TableMetadata object from a byte stream + def table_metadata( + byte_stream: InputStream, encoding: str = "utf-8", compression: Compressor = NOOP_COMPRESSOR + ) -> TableMetadata: + """Instantiate a TableMetadata object from a byte stream. Args: - byte_stream: A file-like byte stream object - encoding (default "utf-8"): The byte encoder to use for the reader + byte_stream: A file-like byte stream object. + encoding (default "utf-8"): The byte encoder to use for the reader. + compression: Optional compression method """ - reader = codecs.getreader(encoding) - metadata = json.load(reader(byte_stream)) + with compression.stream_decompressor(byte_stream) as byte_stream: + reader = codecs.getreader(encoding) + json_bytes = reader(byte_stream) + metadata = json.load(json_bytes) + return TableMetadataUtil.parse_obj(metadata) class FromInputFile: - """A collection of methods that deserialize InputFiles into Iceberg objects""" + """A collection of methods that deserialize InputFiles into Iceberg objects.""" @staticmethod def table_metadata(input_file: InputFile, encoding: str = "utf-8") -> TableMetadata: - """Create a TableMetadata instance from an input file + """Create a TableMetadata instance from an input file. Args: - input_file (InputFile): A custom implementation of the iceberg.io.file.InputFile abstract base class - encoding (str): Encoding to use when loading bytestream + input_file (InputFile): A custom implementation of the iceberg.io.file.InputFile abstract base class. + encoding (str): Encoding to use when loading bytestream. Returns: - TableMetadata: A table metadata instance + TableMetadata: A table metadata instance. """ with input_file.open() as input_stream: - return FromByteStream.table_metadata(byte_stream=input_stream, encoding=encoding) + return FromByteStream.table_metadata( + byte_stream=input_stream, encoding=encoding, compression=Compressor.get_compressor(location=input_file.location) + ) class ToOutputFile: - """A collection of methods that serialize Iceberg objects into files given an OutputFile instance""" + """A collection of methods that serialize Iceberg objects into files given an OutputFile instance.""" @staticmethod def table_metadata(metadata: TableMetadata, output_file: OutputFile, overwrite: bool = False) -> None: - """Write a TableMetadata instance to an output file + """Write a TableMetadata instance to an output file. Args: - output_file (OutputFile): A custom implementation of the iceberg.io.file.OutputFile abstract base class + output_file (OutputFile): A custom implementation of the iceberg.io.file.OutputFile abstract base class. overwrite (bool): Where to overwrite the file if it already exists. Defaults to `False`. """ with output_file.create(overwrite=overwrite) as output_stream: - output_stream.write(metadata.json().encode("utf-8")) + json_bytes = metadata.json().encode("utf-8") + json_bytes = Compressor.get_compressor(output_file.location).bytes_compressor()(json_bytes) + output_stream.write(json_bytes) diff --git a/python/pyiceberg/table/__init__.py b/python/pyiceberg/table/__init__.py index 8f19d93acf4b..997729c23996 100644 --- a/python/pyiceberg/table/__init__.py +++ b/python/pyiceberg/table/__init__.py @@ -17,47 +17,55 @@ from __future__ import annotations from abc import ABC, abstractmethod +from concurrent.futures import ThreadPoolExecutor from dataclasses import dataclass +from enum import Enum from functools import cached_property from itertools import chain -from multiprocessing.pool import ThreadPool from typing import ( TYPE_CHECKING, Any, Callable, Dict, - Iterator, + Iterable, List, + Literal, Optional, + Set, Tuple, TypeVar, Union, ) from pydantic import Field +from sortedcontainers import SortedList from pyiceberg.expressions import ( AlwaysTrue, And, BooleanExpression, + EqualTo, parser, visitors, ) from pyiceberg.expressions.visitors import _InclusiveMetricsEvaluator, inclusive_projection from pyiceberg.io import FileIO, load_file_io from pyiceberg.manifest import ( + POSITIONAL_DELETE_SCHEMA, DataFile, + DataFileContent, ManifestContent, + ManifestEntry, ManifestFile, - files, ) from pyiceberg.partitioning import PartitionSpec from pyiceberg.schema import Schema -from pyiceberg.table.metadata import TableMetadata +from pyiceberg.table.metadata import INITIAL_SEQUENCE_NUMBER, TableMetadata from pyiceberg.table.snapshots import Snapshot, SnapshotLogEntry from pyiceberg.table.sorting import SortOrder from pyiceberg.typedef import ( EMPTY_DICT, + IcebergBaseModel, Identifier, KeyDefaultDict, Properties, @@ -69,28 +77,332 @@ import ray from duckdb import DuckDBPyConnection + from pyiceberg.catalog import Catalog + ALWAYS_TRUE = AlwaysTrue() +class Transaction: + _table: Table + _updates: Tuple[TableUpdate, ...] + _requirements: Tuple[TableRequirement, ...] + + def __init__( + self, + table: Table, + actions: Optional[Tuple[TableUpdate, ...]] = None, + requirements: Optional[Tuple[TableRequirement, ...]] = None, + ): + self._table = table + self._updates = actions or () + self._requirements = requirements or () + + def __enter__(self) -> Transaction: + """Starts a transaction to update the table.""" + return self + + def __exit__(self, _: Any, value: Any, traceback: Any) -> None: + """Closes and commits the transaction.""" + fresh_table = self.commit_transaction() + # Update the new data in place + self._table.metadata = fresh_table.metadata + self._table.metadata_location = fresh_table.metadata_location + + def _append_updates(self, *new_updates: TableUpdate) -> Transaction: + """Appends updates to the set of staged updates. + + Args: + *new_updates: Any new updates. + + Raises: + ValueError: When the type of update is not unique. + + Returns: + A new AlterTable object with the new updates appended. + """ + for new_update in new_updates: + type_new_update = type(new_update) + if any(type(update) == type_new_update for update in self._updates): + raise ValueError(f"Updates in a single commit need to be unique, duplicate: {type_new_update}") + self._updates = self._updates + new_updates + return self + + def set_table_version(self, format_version: Literal[1, 2]) -> Transaction: + """Sets the table to a certain version. + + Args: + format_version: The newly set version. + + Returns: + The alter table builder. + """ + raise NotImplementedError("Not yet implemented") + + def set_properties(self, **updates: str) -> Transaction: + """Set properties. + + When a property is already set, it will be overwritten. + + Args: + updates: The properties set on the table. + + Returns: + The alter table builder. + """ + return self._append_updates(SetPropertiesUpdate(updates=updates)) + + def remove_properties(self, *removals: str) -> Transaction: + """Removes properties. + + Args: + removals: Properties to be removed. + + Returns: + The alter table builder. + """ + return self._append_updates(RemovePropertiesUpdate(removals=removals)) + + def update_location(self, location: str) -> Transaction: + """Sets the new table location. + + Args: + location: The new location of the table. + + Returns: + The alter table builder. + """ + raise NotImplementedError("Not yet implemented") + + def commit_transaction(self) -> Table: + """Commits the changes to the catalog. + + Returns: + The table with the updates applied. + """ + # Strip the catalog name + if len(self._updates) > 0: + response = self._table.catalog._commit_table( # pylint: disable=W0212 + CommitTableRequest( + identifier=self._table.identifier[1:], + requirements=self._requirements, + updates=self._updates, + ) + ) + # Update the metadata with the new one + self._table.metadata = response.metadata + self._table.metadata_location = response.metadata_location + + return self._table + else: + return self._table + + +class TableUpdateAction(Enum): + upgrade_format_version = "upgrade-format-version" + add_schema = "add-schema" + set_current_schema = "set-current-schema" + add_spec = "add-spec" + set_default_spec = "set-default-spec" + add_sort_order = "add-sort-order" + set_default_sort_order = "set-default-sort-order" + add_snapshot = "add-snapshot" + set_snapshot_ref = "set-snapshot-ref" + remove_snapshots = "remove-snapshots" + remove_snapshot_ref = "remove-snapshot-ref" + set_location = "set-location" + set_properties = "set-properties" + remove_properties = "remove-properties" + + +class TableUpdate(IcebergBaseModel): + action: TableUpdateAction + + +class UpgradeFormatVersionUpdate(TableUpdate): + action: TableUpdateAction = TableUpdateAction.upgrade_format_version + format_version: int = Field(alias="format-version") + + +class AddSchemaUpdate(TableUpdate): + action: TableUpdateAction = TableUpdateAction.add_schema + schema_: Schema = Field(alias="schema") + + +class SetCurrentSchemaUpdate(TableUpdate): + action: TableUpdateAction = TableUpdateAction.set_current_schema + schema_id: int = Field( + alias="schema-id", description="Schema ID to set as current, or -1 to set last added schema", default=-1 + ) + + +class AddPartitionSpecUpdate(TableUpdate): + action: TableUpdateAction = TableUpdateAction.add_spec + spec: PartitionSpec + + +class SetDefaultSpecUpdate(TableUpdate): + action: TableUpdateAction = TableUpdateAction.set_default_spec + spec_id: int = Field( + alias="spec-id", description="Partition spec ID to set as the default, or -1 to set last added spec", default=-1 + ) + + +class AddSortOrderUpdate(TableUpdate): + action: TableUpdateAction = TableUpdateAction.add_sort_order + sort_order: SortOrder = Field(alias="sort-order") + + +class SetDefaultSortOrderUpdate(TableUpdate): + action: TableUpdateAction = TableUpdateAction.set_default_sort_order + sort_order_id: int = Field( + alias="sort-order-id", description="Sort order ID to set as the default, or -1 to set last added sort order", default=-1 + ) + + +class AddSnapshotUpdate(TableUpdate): + action: TableUpdateAction = TableUpdateAction.add_snapshot + snapshot: Snapshot + + +class SetSnapshotRefUpdate(TableUpdate): + action: TableUpdateAction = TableUpdateAction.set_snapshot_ref + ref_name: str = Field(alias="ref-name") + type: Literal["tag", "branch"] + snapshot_id: int = Field(alias="snapshot-id") + max_age_ref_ms: int = Field(alias="max-ref-age-ms") + max_snapshot_age_ms: int = Field(alias="max-snapshot-age-ms") + min_snapshots_to_keep: int = Field(alias="min-snapshots-to-keep") + + +class RemoveSnapshotsUpdate(TableUpdate): + action: TableUpdateAction = TableUpdateAction.remove_snapshots + snapshot_ids: List[int] = Field(alias="snapshot-ids") + + +class RemoveSnapshotRefUpdate(TableUpdate): + action: TableUpdateAction = TableUpdateAction.remove_snapshot_ref + ref_name: str = Field(alias="ref-name") + + +class SetLocationUpdate(TableUpdate): + action: TableUpdateAction = TableUpdateAction.set_location + location: str + + +class SetPropertiesUpdate(TableUpdate): + action: TableUpdateAction = TableUpdateAction.set_properties + updates: Dict[str, str] + + +class RemovePropertiesUpdate(TableUpdate): + action: TableUpdateAction = TableUpdateAction.remove_properties + removals: List[str] + + +class TableRequirement(IcebergBaseModel): + type: str + + +class AssertCreate(TableRequirement): + """The table must not already exist; used for create transactions.""" + + type: Literal["assert-create"] + + +class AssertTableUUID(TableRequirement): + """The table UUID must match the requirement's `uuid`.""" + + type: Literal["assert-table-uuid"] + uuid: str + + +class AssertRefSnapshotId(TableRequirement): + """The table branch or tag identified by the requirement's `ref` must reference the requirement's `snapshot-id`. + + if `snapshot-id` is `null` or missing, the ref must not already exist. + """ + + type: Literal["assert-ref-snapshot-id"] + ref: str + snapshot_id: int = Field(..., alias="snapshot-id") + + +class AssertLastAssignedFieldId(TableRequirement): + """The table's last assigned column id must match the requirement's `last-assigned-field-id`.""" + + type: Literal["assert-last-assigned-field-id"] + last_assigned_field_id: int = Field(..., alias="last-assigned-field-id") + + +class AssertCurrentSchemaId(TableRequirement): + """The table's current schema id must match the requirement's `current-schema-id`.""" + + type: Literal["assert-current-schema-id"] + current_schema_id: int = Field(..., alias="current-schema-id") + + +class AssertLastAssignedPartitionId(TableRequirement): + """The table's last assigned partition id must match the requirement's `last-assigned-partition-id`.""" + + type: Literal["assert-last-assigned-partition-id"] + last_assigned_partition_id: int = Field(..., alias="last-assigned-partition-id") + + +class AssertDefaultSpecId(TableRequirement): + """The table's default spec id must match the requirement's `default-spec-id`.""" + + type: Literal["assert-default-spec-id"] + default_spec_id: int = Field(..., alias="default-spec-id") + + +class AssertDefaultSortOrderId(TableRequirement): + """The table's default sort order id must match the requirement's `default-sort-order-id`.""" + + type: Literal["assert-default-sort-order-id"] + default_sort_order_id: int = Field(..., alias="default-sort-order-id") + + +class CommitTableRequest(IcebergBaseModel): + identifier: Identifier = Field() + requirements: List[TableRequirement] = Field(default_factory=list) + updates: List[TableUpdate] = Field(default_factory=list) + + +class CommitTableResponse(IcebergBaseModel): + metadata: TableMetadata = Field() + metadata_location: str = Field(alias="metadata-location") + + class Table: identifier: Identifier = Field() metadata: TableMetadata = Field() metadata_location: str = Field() io: FileIO + catalog: Catalog - def __init__(self, identifier: Identifier, metadata: TableMetadata, metadata_location: str, io: FileIO) -> None: + def __init__( + self, identifier: Identifier, metadata: TableMetadata, metadata_location: str, io: FileIO, catalog: Catalog + ) -> None: self.identifier = identifier self.metadata = metadata self.metadata_location = metadata_location self.io = io + self.catalog = catalog + + def transaction(self) -> Transaction: + return Transaction(self) def refresh(self) -> Table: - """Refresh the current table metadata""" - raise NotImplementedError("To be implemented") + """Refresh the current table metadata.""" + fresh = self.catalog.load_table(self.identifier[1:]) + self.metadata = fresh.metadata + self.io = fresh.io + self.metadata_location = fresh.metadata_location + return self def name(self) -> Identifier: - """Return the identifier of this table""" + """Return the identifier of this table.""" return self.identifier def scan( @@ -113,31 +425,36 @@ def scan( ) def schema(self) -> Schema: - """Return the schema for this table""" + """Return the schema for this table.""" return next(schema for schema in self.metadata.schemas if schema.schema_id == self.metadata.current_schema_id) def schemas(self) -> Dict[int, Schema]: - """Return a dict of the schema of this table""" + """Return a dict of the schema of this table.""" return {schema.schema_id: schema for schema in self.metadata.schemas} def spec(self) -> PartitionSpec: - """Return the partition spec of this table""" + """Return the partition spec of this table.""" return next(spec for spec in self.metadata.partition_specs if spec.spec_id == self.metadata.default_spec_id) def specs(self) -> Dict[int, PartitionSpec]: - """Return a dict the partition specs this table""" + """Return a dict the partition specs this table.""" return {spec.spec_id: spec for spec in self.metadata.partition_specs} def sort_order(self) -> SortOrder: - """Return the sort order of this table""" + """Return the sort order of this table.""" return next( sort_order for sort_order in self.metadata.sort_orders if sort_order.order_id == self.metadata.default_sort_order_id ) def sort_orders(self) -> Dict[int, SortOrder]: - """Return a dict of the sort orders of this table""" + """Return a dict of the sort orders of this table.""" return {sort_order.order_id: sort_order for sort_order in self.metadata.sort_orders} + @property + def properties(self) -> Dict[str, str]: + """Properties of the table.""" + return self.metadata.properties + def location(self) -> str: """Return the table's base location.""" return self.metadata.location @@ -166,6 +483,7 @@ def history(self) -> List[SnapshotLogEntry]: return self.metadata.snapshot_log def __eq__(self, other: Any) -> bool: + """Returns the equality of two instances of the Table class.""" return ( self.identifier == other.identifier and self.metadata == other.metadata @@ -179,7 +497,7 @@ class StaticTable(Table): """Load a table directly from a metadata file (i.e., without using a catalog).""" def refresh(self) -> Table: - """Refresh the current table metadata""" + """Refresh the current table metadata.""" raise NotImplementedError("To be implemented") @classmethod @@ -191,23 +509,26 @@ def from_metadata(cls, metadata_location: str, properties: Properties = EMPTY_DI metadata = FromInputFile.table_metadata(file) + from pyiceberg.catalog.noop import NoopCatalog + return cls( identifier=("static-table", metadata_location), metadata_location=metadata_location, metadata=metadata, io=load_file_io({**properties, **metadata.properties}), + catalog=NoopCatalog("static-table"), ) def _parse_row_filter(expr: Union[str, BooleanExpression]) -> BooleanExpression: - """Accepts an expression in the form of a BooleanExpression or a string + """Accepts an expression in the form of a BooleanExpression or a string. - In the case of a string, it will be converted into a unbound BooleanExpression + In the case of a string, it will be converted into a unbound BooleanExpression. Args: - expr: Expression as a BooleanExpression or a string + expr: Expression as a BooleanExpression or a string. - Returns: An unbound BooleanExpression + Returns: An unbound BooleanExpression. """ return parser.parse(expr) if isinstance(expr, str) else expr @@ -259,7 +580,7 @@ def projection(self) -> Schema: return snapshot_schema.select(*self.selected_fields, case_sensitive=self.case_sensitive) @abstractmethod - def plan_files(self) -> Iterator[ScanTask]: + def plan_files(self) -> Iterable[ScanTask]: ... @abstractmethod @@ -301,35 +622,71 @@ class ScanTask(ABC): @dataclass(init=False) class FileScanTask(ScanTask): file: DataFile + delete_files: Set[DataFile] start: int length: int - def __init__(self, data_file: DataFile, start: Optional[int] = None, length: Optional[int] = None): + def __init__( + self, + data_file: DataFile, + delete_files: Optional[Set[DataFile]] = None, + start: Optional[int] = None, + length: Optional[int] = None, + ) -> None: self.file = data_file + self.delete_files = delete_files or set() self.start = start or 0 self.length = length or data_file.file_size_in_bytes -def _check_content(file: DataFile) -> DataFile: - try: - if file.content == ManifestContent.DELETES: - raise ValueError("PyIceberg does not support deletes: https://github.com/apache/iceberg/issues/6568") - return file - except AttributeError: - # If the attribute is not there, it is a V1 record - return file - - def _open_manifest( io: FileIO, manifest: ManifestFile, partition_filter: Callable[[DataFile], bool], metrics_evaluator: Callable[[DataFile], bool], -) -> List[FileScanTask]: - all_files = files(io.new_input(manifest.manifest_path)) - matching_partition_files = filter(partition_filter, all_files) - matching_partition_data_files = map(_check_content, matching_partition_files) - return [FileScanTask(file) for file in matching_partition_data_files if metrics_evaluator(file)] +) -> List[ManifestEntry]: + return [ + manifest_entry + for manifest_entry in manifest.fetch_manifest_entry(io, discard_deleted=True) + if partition_filter(manifest_entry.data_file) and metrics_evaluator(manifest_entry.data_file) + ] + + +def _min_data_file_sequence_number(manifests: List[ManifestFile]) -> int: + try: + return min( + manifest.min_sequence_number or INITIAL_SEQUENCE_NUMBER + for manifest in manifests + if manifest.content == ManifestContent.DATA + ) + except ValueError: + # In case of an empty iterator + return INITIAL_SEQUENCE_NUMBER + + +def _match_deletes_to_datafile(data_entry: ManifestEntry, positional_delete_entries: SortedList[ManifestEntry]) -> Set[DataFile]: + """This method will check if the delete file is relevant for the data file. + + Using the column metrics to see if the filename is in the lower and upper bound. + + Args: + data_entry (ManifestEntry): The manifest entry path of the datafile. + positional_delete_entries (List[ManifestEntry]): All the candidate positional deletes manifest entries. + + Returns: + A set of files that are relevant for the data file. + """ + relevant_entries = positional_delete_entries[positional_delete_entries.bisect_right(data_entry) :] + + if len(relevant_entries) > 0: + evaluator = _InclusiveMetricsEvaluator(POSITIONAL_DELETE_SCHEMA, EqualTo("file_path", data_entry.data_file.file_path)) + return { + positional_delete_entry.data_file + for positional_delete_entry in relevant_entries + if evaluator.eval(positional_delete_entry.data_file) + } + else: + return set() class DataScan(TableScan): @@ -366,7 +723,28 @@ def _build_partition_evaluator(self, spec_id: int) -> Callable[[DataFile], bool] evaluator = visitors.expression_evaluator(partition_schema, partition_expr, self.case_sensitive) return lambda data_file: evaluator(data_file.partition) - def plan_files(self) -> Iterator[FileScanTask]: + def _check_sequence_number(self, min_data_sequence_number: int, manifest: ManifestFile) -> bool: + """A helper function to make sure that no manifests are loaded that contain deletes that are older than the data. + + Args: + min_data_sequence_number (int): The minimal sequence number. + manifest (ManifestFile): A ManifestFile that can be either data or deletes. + + Returns: + Boolean indicating if it is either a data file, or a relevant delete file. + """ + return manifest.content == ManifestContent.DATA or ( + # Not interested in deletes that are older than the data + manifest.content == ManifestContent.DELETES + and (manifest.sequence_number or INITIAL_SEQUENCE_NUMBER) >= min_data_sequence_number + ) + + def plan_files(self) -> Iterable[FileScanTask]: + """Plans the relevant files by filtering on the PartitionSpecs. + + Returns: + List of FileScanTasks that contain both data and delete files. + """ snapshot = self.snapshot() if not snapshot: return iter([]) @@ -389,11 +767,17 @@ def plan_files(self) -> Iterator[FileScanTask]: partition_evaluators: Dict[int, Callable[[DataFile], bool]] = KeyDefaultDict(self._build_partition_evaluator) metrics_evaluator = _InclusiveMetricsEvaluator(self.table.schema(), self.row_filter, self.case_sensitive).eval - with ThreadPool() as pool: - return chain( - *pool.starmap( - func=_open_manifest, - iterable=[ + + min_data_sequence_number = _min_data_file_sequence_number(manifests) + + data_entries: List[ManifestEntry] = [] + positional_delete_entries = SortedList(key=lambda entry: entry.data_sequence_number or INITIAL_SEQUENCE_NUMBER) + + with ThreadPoolExecutor() as executor: + for manifest_entry in chain( + *executor.map( + lambda args: _open_manifest(*args), + [ ( io, manifest, @@ -401,9 +785,32 @@ def plan_files(self) -> Iterator[FileScanTask]: metrics_evaluator, ) for manifest in manifests + if self._check_sequence_number(min_data_sequence_number, manifest) ], ) + ): + data_file = manifest_entry.data_file + if data_file.content == DataFileContent.DATA: + data_entries.append(manifest_entry) + elif data_file.content == DataFileContent.POSITION_DELETES: + positional_delete_entries.add(manifest_entry) + elif data_file.content == DataFileContent.EQUALITY_DELETES: + raise ValueError( + "PyIceberg does not yet support equality deletes: https://github.com/apache/iceberg/issues/6568" + ) + else: + raise ValueError(f"Unknown DataFileContent ({data_file.content}): {manifest_entry}") + + return [ + FileScanTask( + data_entry.data_file, + delete_files=_match_deletes_to_datafile( + data_entry, + positional_delete_entries, + ), ) + for data_entry in data_entries + ] def to_arrow(self) -> pa.Table: from pyiceberg.io.pyarrow import project_table diff --git a/python/pyiceberg/table/metadata.py b/python/pyiceberg/table/metadata.py index 31e74e708407..d15ec3a64d07 100644 --- a/python/pyiceberg/table/metadata.py +++ b/python/pyiceberg/table/metadata.py @@ -40,7 +40,7 @@ assign_fresh_sort_order_ids, ) from pyiceberg.typedef import EMPTY_DICT, IcebergBaseModel, Properties -from pyiceberg.utils.datetime import datetime_to_micros +from pyiceberg.utils.datetime import datetime_to_millis CURRENT_SNAPSHOT_ID = "current_snapshot_id" CURRENT_SCHEMA_ID = "current_schema_id" @@ -57,7 +57,7 @@ def check_schemas(values: Dict[str, Any]) -> Dict[str, Any]: - """Validator to check if the current-schema-id is actually present in schemas""" + """Validator to check if the current-schema-id is actually present in schemas.""" current_schema_id = values[CURRENT_SCHEMA_ID] for schema in values[SCHEMAS]: @@ -68,7 +68,7 @@ def check_schemas(values: Dict[str, Any]) -> Dict[str, Any]: def check_partition_specs(values: Dict[str, Any]) -> Dict[str, Any]: - """Validator to check if the default-spec-id is present in partition-specs""" + """Validator to check if the default-spec-id is present in partition-specs.""" default_spec_id = values["default_spec_id"] partition_specs: List[PartitionSpec] = values[PARTITION_SPECS] @@ -80,7 +80,7 @@ def check_partition_specs(values: Dict[str, Any]) -> Dict[str, Any]: def check_sort_orders(values: Dict[str, Any]) -> Dict[str, Any]: - """Validator to check if the default_sort_order_id is present in sort-orders""" + """Validator to check if the default_sort_order_id is present in sort-orders.""" default_sort_order_id: int = values["default_sort_order_id"] if default_sort_order_id != UNSORTED_SORT_ORDER_ID: @@ -94,8 +94,10 @@ def check_sort_orders(values: Dict[str, Any]) -> Dict[str, Any]: class TableMetadataCommonFields(IcebergBaseModel): - """Metadata for an Iceberg table as specified in the Apache Iceberg - spec (https://iceberg.apache.org/spec/#iceberg-table-spec)""" + """Metadata for an Iceberg table as specified in the Apache Iceberg spec. + + https://iceberg.apache.org/spec/#iceberg-table-spec + """ @root_validator(skip_on_failure=True) def cleanup_snapshot_id(cls, data: Dict[str, Any]) -> Dict[str, Any]: @@ -122,7 +124,9 @@ def construct_refs(cls, data: Dict[str, Any]) -> Dict[str, Any]: Implementations must throw an exception if a table’s UUID does not match the expected UUID after refreshing metadata.""" - last_updated_ms: int = Field(alias="last-updated-ms", default_factory=lambda: datetime_to_micros(datetime.datetime.now())) + last_updated_ms: int = Field( + alias="last-updated-ms", default_factory=lambda: datetime_to_millis(datetime.datetime.now().astimezone()) + ) """Timestamp in milliseconds from the unix epoch when the table was last updated. Each table metadata file should update this field just before writing.""" @@ -144,7 +148,7 @@ def construct_refs(cls, data: Dict[str, Any]) -> Dict[str, Any]: default_spec_id: int = Field(alias="default-spec-id", default=INITIAL_SPEC_ID) """ID of the “current” spec that writers should use by default.""" - last_partition_id: Optional[int] = Field(alias="last-partition-id") + last_partition_id: Optional[int] = Field(alias="last-partition-id", default=None) """An integer; the highest assigned partition field ID across all partition specs for the table. This is used to ensure partition fields are always assigned an unused ID when evolving specs.""" @@ -197,7 +201,7 @@ def construct_refs(cls, data: Dict[str, Any]) -> Dict[str, Any]: class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel): - """Represents version 1 of the Table Metadata + """Represents version 1 of the Table Metadata. More information about the specification: https://iceberg.apache.org/spec/#version-1-analytic-data-tables @@ -212,13 +216,13 @@ class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel): @root_validator def set_v2_compatible_defaults(cls, data: Dict[str, Any]) -> Dict[str, Any]: - """Sets default values to be compatible with the format v2 + """Sets default values to be compatible with the format v2. Args: - data: The raw arguments when initializing a V1 TableMetadata + data: The raw arguments when initializing a V1 TableMetadata. Returns: - The TableMetadata with the defaults applied + The TableMetadata with the defaults applied. """ # When the schema doesn't have an ID if data.get("schema") and "schema_id" not in data["schema"]: @@ -228,17 +232,17 @@ def set_v2_compatible_defaults(cls, data: Dict[str, Any]) -> Dict[str, Any]: @root_validator(skip_on_failure=True) def construct_schemas(cls, data: Dict[str, Any]) -> Dict[str, Any]: - """Converts the schema into schemas + """Converts the schema into schemas. For V1 schemas is optional, and if they aren't set, we'll set them in this validator. This was we can always use the schemas when reading table metadata, and we don't have to worry if it is a v1 or v2 format. Args: - data: The raw data after validation, meaning that the aliases are applied + data: The raw data after validation, meaning that the aliases are applied. Returns: - The TableMetadata with the schemas set, if not provided + The TableMetadata with the schemas set, if not provided. """ if not data.get("schemas"): schema = data["schema_"] @@ -249,17 +253,17 @@ def construct_schemas(cls, data: Dict[str, Any]) -> Dict[str, Any]: @root_validator(skip_on_failure=True) def construct_partition_specs(cls, data: Dict[str, Any]) -> Dict[str, Any]: - """Converts the partition_spec into partition_specs + """Converts the partition_spec into partition_specs. For V1 partition_specs is optional, and if they aren't set, we'll set them in this validator. This was we can always use the partition_specs when reading table metadata, and we don't have to worry if it is a v1 or v2 format. Args: - data: The raw data after validation, meaning that the aliases are applied + data: The raw data after validation, meaning that the aliases are applied. Returns: - The TableMetadata with the partition_specs set, if not provided + The TableMetadata with the partition_specs set, if not provided. """ if not data.get(PARTITION_SPECS): fields = data[PARTITION_SPEC] @@ -277,16 +281,16 @@ def construct_partition_specs(cls, data: Dict[str, Any]) -> Dict[str, Any]: @root_validator(skip_on_failure=True) def set_sort_orders(cls, data: Dict[str, Any]) -> Dict[str, Any]: - """Sets the sort_orders if not provided + """Sets the sort_orders if not provided. For V1 sort_orders is optional, and if they aren't set, we'll set them in this validator. Args: - data: The raw data after validation, meaning that the aliases are applied + data: The raw data after validation, meaning that the aliases are applied. Returns: - The TableMetadata with the sort_orders set, if not provided + The TableMetadata with the sort_orders set, if not provided. """ if not data.get(SORT_ORDERS): data[SORT_ORDERS] = [UNSORTED_SORT_ORDER] @@ -306,18 +310,18 @@ def to_v2(self) -> "TableMetadataV2": schema_: Schema = Field(alias="schema") """The table’s current schema. (Deprecated: use schemas and - current-schema-id instead)""" + current-schema-id instead).""" partition_spec: List[Dict[str, Any]] = Field(alias="partition-spec") """The table’s current partition spec, stored as only fields. Note that this is used by writers to partition data, but is not used when reading because reads use the specs stored in manifest files. (Deprecated: use partition-specs and default-spec-id - instead)""" + instead).""" class TableMetadataV2(TableMetadataCommonFields, IcebergBaseModel): - """Represents version 2 of the Table Metadata + """Represents version 2 of the Table Metadata. This extends Version 1 with row-level deletes, and adds some additional information to the schema, such as all the historical schemas, partition-specs, @@ -374,7 +378,7 @@ def new_table_metadata( class TableMetadataUtil: - """Helper class for parsing TableMetadata""" + """Helper class for parsing TableMetadata.""" # Once this has been resolved, we can simplify this: https://github.com/samuelcolvin/pydantic/issues/3846 # TableMetadata = Annotated[TableMetadata, Field(alias="format-version", discriminator="format-version")] diff --git a/python/pyiceberg/table/refs.py b/python/pyiceberg/table/refs.py index 09cbad27f13a..a9089f84af5f 100644 --- a/python/pyiceberg/table/refs.py +++ b/python/pyiceberg/table/refs.py @@ -29,6 +29,7 @@ class SnapshotRefType(str, Enum): TAG = "tag" def __repr__(self) -> str: + """Returns the string representation of the SnapshotRefType class.""" return f"SnapshotRefType.{self.name}" diff --git a/python/pyiceberg/table/snapshots.py b/python/pyiceberg/table/snapshots.py index 83dd3f66a7cf..b3d90eebb7ba 100644 --- a/python/pyiceberg/table/snapshots.py +++ b/python/pyiceberg/table/snapshots.py @@ -33,7 +33,7 @@ class Operation(Enum): - """Describes the operation + """Describes the operation. Possible operation values are: - append: Only data files were added and no files were removed. @@ -48,11 +48,13 @@ class Operation(Enum): DELETE = "delete" def __repr__(self) -> str: + """Returns the string representation of the Operation class.""" return f"Operation.{self.name}" class Summary(IcebergBaseModel): - """ + """A class that stores the summary information for a Snapshot. + The snapshot summary’s operation field is used by some operations, like snapshot expiration, to skip processing certain snapshots. """ @@ -91,20 +93,24 @@ def additional_properties(self) -> Dict[str, str]: return self._additional_properties def __repr__(self) -> str: + """Returns the string representation of the Summary class.""" repr_properties = f", **{repr(self._additional_properties)}" if self._additional_properties else "" return f"Summary({repr(self.operation)}{repr_properties})" class Snapshot(IcebergBaseModel): snapshot_id: int = Field(alias="snapshot-id") - parent_snapshot_id: Optional[int] = Field(alias="parent-snapshot-id") + parent_snapshot_id: Optional[int] = Field(alias="parent-snapshot-id", default=None) sequence_number: Optional[int] = Field(alias="sequence-number", default=None) timestamp_ms: int = Field(alias="timestamp-ms") - manifest_list: Optional[str] = Field(alias="manifest-list", description="Location of the snapshot's manifest list file") - summary: Optional[Summary] = Field() + manifest_list: Optional[str] = Field( + alias="manifest-list", description="Location of the snapshot's manifest list file", default=None + ) + summary: Optional[Summary] = Field(default=None) schema_id: Optional[int] = Field(alias="schema-id", default=None) def __str__(self) -> str: + """Returns the string representation of the Snapshot class.""" operation = f"{self.summary.operation}: " if self.summary else "" parent_id = f", parent_id={self.parent_snapshot_id}" if self.parent_snapshot_id else "" schema_id = f", schema_id={self.schema_id}" if self.schema_id is not None else "" @@ -124,5 +130,5 @@ class MetadataLogEntry(IcebergBaseModel): class SnapshotLogEntry(IcebergBaseModel): - snapshot_id: str = Field(alias="snapshot-id") + snapshot_id: int = Field(alias="snapshot-id") timestamp_ms: int = Field(alias="timestamp-ms") diff --git a/python/pyiceberg/table/sorting.py b/python/pyiceberg/table/sorting.py index 78589440f580..3fbc82465f55 100644 --- a/python/pyiceberg/table/sorting.py +++ b/python/pyiceberg/table/sorting.py @@ -38,9 +38,11 @@ class SortDirection(Enum): DESC = "desc" def __str__(self) -> str: + """Returns the string representation of the SortDirection class.""" return self.name def __repr__(self) -> str: + """Returns the string representation of the SortDirection class.""" return f"SortDirection.{self.name}" @@ -49,21 +51,23 @@ class NullOrder(Enum): NULLS_LAST = "nulls-last" def __str__(self) -> str: + """Returns the string representation of the NullOrder class.""" return self.name.replace("_", " ") def __repr__(self) -> str: + """Returns the string representation of the NullOrder class.""" return f"NullOrder.{self.name}" class SortField(IcebergBaseModel): - """Sort order field + """Sort order field. Args: - source_id (int): Source column id from the table’s schema + source_id (int): Source column id from the table’s schema. transform (str): Transform that is used to produce values to be sorted on from the source column. This is the same transform as described in partition transforms. - direction (SortDirection): Sort direction, that can only be either asc or desc - null_order (NullOrder): Null order that describes the order of null values when sorted. Can only be either nulls-first or nulls-last + direction (SortDirection): Sort direction, that can only be either asc or desc. + null_order (NullOrder): Null order that describes the order of null values when sorted. Can only be either nulls-first or nulls-last. """ def __init__( @@ -97,6 +101,7 @@ def set_null_order(cls, values: Dict[str, Any]) -> Dict[str, Any]: null_order: NullOrder = Field(alias="null-order") def __str__(self) -> str: + """Returns the string representation of the SortField class.""" if type(self.transform) == IdentityTransform: # In the case of an identity transform, we can omit the transform return f"{self.source_id} {self.direction} {self.null_order}" @@ -108,7 +113,7 @@ def __str__(self) -> str: class SortOrder(IcebergBaseModel): - """Describes how the data is sorted within the table + """Describes how the data is sorted within the table. Users can sort their data within partitions by columns to gain performance. @@ -134,6 +139,7 @@ def is_unsorted(self) -> bool: return len(self.fields) == 0 def __str__(self) -> str: + """Returns the string representation of the SortOrder class.""" result_str = "[" if self.fields: result_str += "\n " + "\n ".join([str(field) for field in self.fields]) + "\n" @@ -141,6 +147,7 @@ def __str__(self) -> str: return result_str def __repr__(self) -> str: + """Returns the string representation of the SortOrder class.""" fields = f"{', '.join(repr(column) for column in self.fields)}, " if self.fields else "" return f"SortOrder({fields}order_id={self.order_id})" diff --git a/python/pyiceberg/transforms.py b/python/pyiceberg/transforms.py index 34af1258bc7b..4b67f6687375 100644 --- a/python/pyiceberg/transforms.py +++ b/python/pyiceberg/transforms.py @@ -101,7 +101,7 @@ def _transform_literal(func: Callable[[L], L], lit: Literal[L]) -> Literal[L]: - """Small helper to upwrap the value from the literal, and wrap it again""" + """Small helper to upwrap the value from the literal, and wrap it again.""" return literal(func(lit.value)) @@ -116,6 +116,7 @@ class Transform(IcebergBaseModel, ABC, Generic[S, T]): @classmethod def __get_validators__(cls) -> Generator[AnyCallable, None, None]: + """Called to validate the input of the Transform class.""" # one or more validators may be yielded which will be called in the # order to validate the input, each validator will receive as an input # the value returned from the previous validator @@ -177,16 +178,18 @@ def dedup_name(self) -> str: return self.__str__() def __str__(self) -> str: + """Returns the string representation of the Transform class.""" return self.__root__ def __eq__(self, other: Any) -> bool: + """Returns the equality of two instances of the Transform class.""" if isinstance(other, Transform): return self.__root__ == other.__root__ return False class BucketTransform(Transform[S, int]): - """Base Transform class to transform a value into a bucket partition value + """Base Transform class to transform a value into a bucket partition value. Transforms are parameterized by a number of buckets. Bucket partition transforms use a 32-bit hash of the source value to produce a positive value by mod the bucket number. @@ -282,6 +285,7 @@ def hash_func(v: Any) -> int: return hash_func def __repr__(self) -> str: + """Returns the string representation of the BucketTransform class.""" return f"BucketTransform(num_buckets={self._num_buckets})" @@ -376,6 +380,7 @@ def to_human_string(self, _: IcebergType, value: Optional[S]) -> str: return datetime.to_human_year(value) if isinstance(value, int) else "null" def __repr__(self) -> str: + """Returns the string representation of the YearTransform class.""" return "YearTransform()" @@ -422,6 +427,7 @@ def to_human_string(self, _: IcebergType, value: Optional[S]) -> str: return datetime.to_human_month(value) if isinstance(value, int) else "null" def __repr__(self) -> str: + """Returns the string representation of the MonthTransform class.""" return "MonthTransform()" @@ -471,6 +477,7 @@ def to_human_string(self, _: IcebergType, value: Optional[S]) -> str: return datetime.to_human_day(value) if isinstance(value, int) else "null" def __repr__(self) -> str: + """Returns the string representation of the DayTransform class.""" return "DayTransform()" @@ -510,11 +517,12 @@ def to_human_string(self, _: IcebergType, value: Optional[S]) -> str: return datetime.to_human_hour(value) if isinstance(value, int) else "null" def __repr__(self) -> str: + """Returns the string representation of the HourTransform class.""" return "HourTransform()" def _base64encode(buffer: bytes) -> str: - """Converts bytes to base64 string""" + """Converts bytes to base64 string.""" return base64.b64encode(buffer).decode("ISO-8859-1") @@ -555,25 +563,28 @@ def preserves_order(self) -> bool: return True def satisfies_order_of(self, other: Transform[S, T]) -> bool: - """ordering by value is the same as long as the other preserves order""" + """Ordering by value is the same as long as the other preserves order.""" return other.preserves_order def to_human_string(self, source_type: IcebergType, value: Optional[S]) -> str: return _human_string(value, source_type) if value is not None else "null" def __str__(self) -> str: + """Returns the string representation of the IdentityTransform class.""" return "identity" def __repr__(self) -> str: + """Returns the string representation of the IdentityTransform class.""" return "IdentityTransform()" class TruncateTransform(Transform[S, S]): """A transform for truncating a value to a specified width. + Args: - width (int): The truncate width, should be positive + width (int): The truncate width, should be positive. Raises: - ValueError: If a type is provided that is incompatible with a Truncate transform + ValueError: If a type is provided that is incompatible with a Truncate transform. """ __root__: str = Field() @@ -663,6 +674,7 @@ def to_human_string(self, _: IcebergType, value: Optional[S]) -> str: return str(value) def __repr__(self) -> str: + """Returns the string representation of the TruncateTransform class.""" return f"TruncateTransform(width={self._width})" @@ -707,12 +719,13 @@ def _(_type: IcebergType, value: int) -> str: class UnknownTransform(Transform[S, T]): - """A transform that represents when an unknown transform is provided + """A transform that represents when an unknown transform is provided. + Args: - transform (str): A string name of a transform + transform (str): A string name of a transform. Keyword Args: - source_type (IcebergType): An Iceberg `Type` + source_type (IcebergType): An Iceberg `Type`. """ __root__: LiteralType["unknown"] = Field(default="unknown") # noqa: F821 @@ -735,11 +748,12 @@ def project(self, name: str, pred: BoundPredicate[L]) -> Optional[UnboundPredica return None def __repr__(self) -> str: + """Returns the string representation of the UnknownTransform class.""" return f"UnknownTransform(transform={repr(self._transform)})" class VoidTransform(Transform[S, None], Singleton): - """A transform that always returns None""" + """A transform that always returns None.""" __root__ = "void" @@ -759,6 +773,7 @@ def to_human_string(self, _: IcebergType, value: Optional[S]) -> str: return "null" def __repr__(self) -> str: + """Returns the string representation of the VoidTransform class.""" return "VoidTransform()" @@ -832,7 +847,7 @@ def _set_apply_transform(name: str, pred: BoundSetPredicate[L], transform: Calla class BoundTransform(BoundTerm[L]): - """A transform expression""" + """A transform expression.""" transform: Transform[L, Any] diff --git a/python/pyiceberg/typedef.py b/python/pyiceberg/typedef.py index 23d3217a9c79..f9931bb344da 100644 --- a/python/pyiceberg/typedef.py +++ b/python/pyiceberg/typedef.py @@ -18,12 +18,13 @@ from abc import abstractmethod from decimal import Decimal -from functools import cached_property +from functools import cached_property, lru_cache from typing import ( TYPE_CHECKING, Any, Callable, Dict, + List, Optional, Protocol, Set, @@ -42,6 +43,7 @@ class FrozenDict(Dict[Any, Any]): def __setitem__(self, instance: Any, value: Any) -> None: + """Used for assigning a value to a FrozenDict.""" raise AttributeError("FrozenDict does not support assignment") def update(self, *args: Any, **kwargs: Any) -> None: @@ -61,6 +63,7 @@ def __init__(self, default_factory: Callable[[K], V]): self.default_factory = default_factory def __missing__(self, key: K) -> V: + """Defines behavior if you access a non-existent key in a KeyDefaultDict.""" val = self.default_factory(key) self[key] = val return val @@ -76,15 +79,15 @@ def __missing__(self, key: K) -> V: @runtime_checkable class StructProtocol(Protocol): # pragma: no cover - """A generic protocol used by accessors to get and set at positions of an object""" + """A generic protocol used by accessors to get and set at positions of an object.""" @abstractmethod def __getitem__(self, pos: int) -> Any: - ... + """Used for fetching a value from a StructProtocol.""" @abstractmethod def __setitem__(self, pos: int, value: Any) -> None: - ... + """Used for assigning a value to a StructProtocol.""" class IcebergBaseModel(BaseModel): @@ -124,17 +127,23 @@ def json(self, exclude_none: bool = True, exclude: Optional[Set[str]] = None, by ) +@lru_cache +def _get_struct_fields(struct_type: StructType) -> Tuple[str, ...]: + return tuple([field.name for field in struct_type.fields]) + + class Record(StructProtocol): - _position_to_field_name: Dict[int, str] + __slots__ = ("_position_to_field_name",) + _position_to_field_name: Tuple[str, ...] def __init__(self, *data: Any, struct: Optional[StructType] = None, **named_data: Any) -> None: if struct is not None: - self._position_to_field_name = {idx: field.name for idx, field in enumerate(struct.fields)} + self._position_to_field_name = _get_struct_fields(struct) elif named_data: # Order of named_data is preserved (PEP 468) so this can be used to generate the position dict - self._position_to_field_name = dict(enumerate(named_data.keys())) + self._position_to_field_name = tuple(named_data.keys()) else: - self._position_to_field_name = {idx: f"field{idx + 1}" for idx in range(len(data))} + self._position_to_field_name = tuple(f"field{idx + 1}" for idx in range(len(data))) for idx, d in enumerate(data): self[idx] = d @@ -143,15 +152,22 @@ def __init__(self, *data: Any, struct: Optional[StructType] = None, **named_data self.__setattr__(field_name, d) def __setitem__(self, pos: int, value: Any) -> None: + """Used for assigning a value to a Record.""" self.__setattr__(self._position_to_field_name[pos], value) def __getitem__(self, pos: int) -> Any: + """Used for fetching a value from a Record.""" return self.__getattribute__(self._position_to_field_name[pos]) def __eq__(self, other: Any) -> bool: + """Returns the equality of two instances of the Record class.""" if not isinstance(other, Record): return False return self.__dict__ == other.__dict__ def __repr__(self) -> str: + """Returns the string representation of the Record class.""" return f"{self.__class__.__name__}[{', '.join(f'{key}={repr(value)}' for key, value in self.__dict__.items() if not key.startswith('_'))}]" + + def record_fields(self) -> List[str]: + return [self.__getattribute__(v) if hasattr(self, v) else None for v in self._position_to_field_name] diff --git a/python/pyiceberg/types.py b/python/pyiceberg/types.py index 3fd023ce9a0c..6a1888fcf436 100644 --- a/python/pyiceberg/types.py +++ b/python/pyiceberg/types.py @@ -14,7 +14,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -"""Data types used in describing Iceberg schemas +"""Data types used in describing Iceberg schemas. This module implements the data types described in the Iceberg specification for Iceberg schemas. To describe an Iceberg table schema, these classes can be used in the construction of a StructType instance. @@ -55,7 +55,7 @@ class IcebergType(IcebergBaseModel, Singleton): - """Base type for all Iceberg Types + """Base type for all Iceberg Types. Example: >>> str(IcebergType()) @@ -66,6 +66,7 @@ class IcebergType(IcebergBaseModel, Singleton): @classmethod def __get_validators__(cls) -> Generator[AnyCallable, None, None]: + """Called to validate the input of the IcebergType class.""" # one or more validators may be yielded which will be called in the # order to validate the input, each validator will receive as an input # the value returned from the previous validator @@ -107,19 +108,22 @@ def is_struct(self) -> bool: class PrimitiveType(IcebergType): - """Base class for all Iceberg Primitive Types""" + """Base class for all Iceberg Primitive Types.""" __root__: str = Field() def __repr__(self) -> str: + """Returns the string representation of the PrimitiveType class.""" return f"{type(self).__name__}()" def __str__(self) -> str: + """Returns the string representation of the PrimitiveType class.""" return self.__root__ class FixedType(PrimitiveType): """A fixed data type in Iceberg. + Example: >>> FixedType(8) FixedType(length=8) @@ -141,14 +145,21 @@ def __init__(self, length: int): self._len = length def __len__(self) -> int: + """Returns the length of an instance of the FixedType class.""" return self._len def __repr__(self) -> str: + """Returns the string representation of the FixedType class.""" return f"FixedType(length={self._len})" + def __getnewargs__(self) -> Tuple[int]: + """A magic function for pickling the FixedType class.""" + return (self._len,) + class DecimalType(PrimitiveType): """A fixed data type in Iceberg. + Example: >>> DecimalType(32, 3) DecimalType(precision=32, scale=3) @@ -188,8 +199,13 @@ def scale(self) -> int: return self._scale def __repr__(self) -> str: + """Returns the string representation of the DecimalType class.""" return f"DecimalType(precision={self._precision}, scale={self._scale})" + def __getnewargs__(self) -> Tuple[int, int]: + """A magic function for pickling the DecimalType class.""" + return (self._precision, self._scale) + class NestedField(IcebergType): """Represents a field of a struct, a map key, a map value, or a list element. @@ -242,17 +258,22 @@ def __init__( super().__init__(**data) def __str__(self) -> str: + """Returns the string representation of the NestedField class.""" doc = "" if not self.doc else f" ({self.doc})" req = "required" if self.required else "optional" return f"{self.field_id}: {self.name}: {req} {self.field_type}{doc}" + def __getnewargs__(self) -> Tuple[int, str, IcebergType, bool, Optional[str]]: + """A magic function for pickling the NestedField class.""" + return (self.field_id, self.name, self.field_type, self.required, self.doc) + @property def optional(self) -> bool: return not self.required class StructType(IcebergType): - """A struct type in Iceberg + """A struct type in Iceberg. Example: >>> str(StructType( @@ -264,12 +285,14 @@ class StructType(IcebergType): type: Literal["struct"] = "struct" fields: Tuple[NestedField, ...] = Field(default_factory=tuple) + _hash: int = PrivateAttr() def __init__(self, *fields: NestedField, **data: Any): # In case we use positional arguments, instead of keyword args if fields: data["fields"] = fields super().__init__(**data) + self._hash = hash(self.fields) def field(self, field_id: int) -> Optional[NestedField]: for field in self.fields: @@ -278,17 +301,28 @@ def field(self, field_id: int) -> Optional[NestedField]: return None def __str__(self) -> str: + """Returns the string representation of the StructType class.""" return f"struct<{', '.join(map(str, self.fields))}>" def __repr__(self) -> str: + """Returns the string representation of the StructType class.""" return f"StructType(fields=({', '.join(map(repr, self.fields))},))" def __len__(self) -> int: + """Returns the length of an instance of the StructType class.""" return len(self.fields) + def __getnewargs__(self) -> Tuple[NestedField, ...]: + """A magic function for pickling the StructType class.""" + return self.fields + + def __hash__(self) -> int: + """Used the cache hash value of the StructType class.""" + return self._hash + class ListType(IcebergType): - """A list type in Iceberg + """A list type in Iceberg. Example: >>> ListType(element_id=3, element_type=StringType(), element_required=True) @@ -319,11 +353,16 @@ def __init__( super().__init__(**data) def __str__(self) -> str: + """Returns the string representation of the ListType class.""" return f"list<{self.element_type}>" + def __getnewargs__(self) -> Tuple[int, IcebergType, bool]: + """A magic function for pickling the ListType class.""" + return (self.element_id, self.element_type, self.element_required) + class MapType(IcebergType): - """A map type in Iceberg + """A map type in Iceberg. Example: >>> MapType(key_id=1, key_type=StringType(), value_id=2, value_type=IntegerType(), value_required=True) @@ -364,8 +403,13 @@ def __init__( super().__init__(**data) def __str__(self) -> str: + """Returns the string representation of the MapType class.""" return f"map<{self.key_type}, {self.value_type}>" + def __getnewargs__(self) -> Tuple[int, IcebergType, int, IcebergType, bool]: + """A magic function for pickling the MapType class.""" + return (self.key_id, self.key_type, self.value_id, self.value_type, self.value_required) + class BooleanType(PrimitiveType): """A boolean data type in Iceberg can be represented using an instance of this class. @@ -382,8 +426,9 @@ class BooleanType(PrimitiveType): class IntegerType(PrimitiveType): - """An Integer data type in Iceberg can be represented using an instance of this class. Integers in Iceberg are - 32-bit signed and can be promoted to Longs. + """An Integer data type in Iceberg can be represented using an instance of this class. + + Integers in Iceberg are 32-bit signed and can be promoted to Longs. Example: >>> column_foo = IntegerType() @@ -404,8 +449,9 @@ class IntegerType(PrimitiveType): class LongType(PrimitiveType): - """A Long data type in Iceberg can be represented using an instance of this class. Longs in Iceberg are - 64-bit signed integers. + """A Long data type in Iceberg can be represented using an instance of this class. + + Longs in Iceberg are 64-bit signed integers. Example: >>> column_foo = LongType() @@ -430,8 +476,9 @@ class LongType(PrimitiveType): class FloatType(PrimitiveType): - """A Float data type in Iceberg can be represented using an instance of this class. Floats in Iceberg are - 32-bit IEEE 754 floating points and can be promoted to Doubles. + """A Float data type in Iceberg can be represented using an instance of this class. + + Floats in Iceberg are 32-bit IEEE 754 floating points and can be promoted to Doubles. Example: >>> column_foo = FloatType() @@ -454,8 +501,9 @@ class FloatType(PrimitiveType): class DoubleType(PrimitiveType): - """A Double data type in Iceberg can be represented using an instance of this class. Doubles in Iceberg are - 64-bit IEEE 754 floating points. + """A Double data type in Iceberg can be represented using an instance of this class. + + Doubles in Iceberg are 64-bit IEEE 754 floating points. Example: >>> column_foo = DoubleType() @@ -469,8 +517,9 @@ class DoubleType(PrimitiveType): class DateType(PrimitiveType): - """A Date data type in Iceberg can be represented using an instance of this class. Dates in Iceberg are - calendar dates without a timezone or time. + """A Date data type in Iceberg can be represented using an instance of this class. + + Dates in Iceberg are calendar dates without a timezone or time. Example: >>> column_foo = DateType() @@ -484,8 +533,9 @@ class DateType(PrimitiveType): class TimeType(PrimitiveType): - """A Time data type in Iceberg can be represented using an instance of this class. Times in Iceberg - have microsecond precision and are a time of day without a date or timezone. + """A Time data type in Iceberg can be represented using an instance of this class. + + Times in Iceberg have microsecond precision and are a time of day without a date or timezone. Example: >>> column_foo = TimeType() @@ -499,8 +549,9 @@ class TimeType(PrimitiveType): class TimestampType(PrimitiveType): - """A Timestamp data type in Iceberg can be represented using an instance of this class. Timestamps in - Iceberg have microsecond precision and include a date and a time of day without a timezone. + """A Timestamp data type in Iceberg can be represented using an instance of this class. + + Timestamps in Iceberg have microsecond precision and include a date and a time of day without a timezone. Example: >>> column_foo = TimestampType() @@ -514,8 +565,9 @@ class TimestampType(PrimitiveType): class TimestamptzType(PrimitiveType): - """A Timestamptz data type in Iceberg can be represented using an instance of this class. Timestamptzs in - Iceberg are stored as UTC and include a date and a time of day with a timezone. + """A Timestamptz data type in Iceberg can be represented using an instance of this class. + + Timestamptzs in Iceberg are stored as UTC and include a date and a time of day with a timezone. Example: >>> column_foo = TimestamptzType() @@ -529,8 +581,9 @@ class TimestamptzType(PrimitiveType): class StringType(PrimitiveType): - """A String data type in Iceberg can be represented using an instance of this class. Strings in - Iceberg are arbitrary-length character sequences and are encoded with UTF-8. + """A String data type in Iceberg can be represented using an instance of this class. + + Strings in Iceberg are arbitrary-length character sequences and are encoded with UTF-8. Example: >>> column_foo = StringType() @@ -544,8 +597,9 @@ class StringType(PrimitiveType): class UUIDType(PrimitiveType): - """A UUID data type in Iceberg can be represented using an instance of this class. UUIDs in - Iceberg are universally unique identifiers. + """A UUID data type in Iceberg can be represented using an instance of this class. + + UUIDs in Iceberg are universally unique identifiers. Example: >>> column_foo = UUIDType() @@ -559,8 +613,9 @@ class UUIDType(PrimitiveType): class BinaryType(PrimitiveType): - """A Binary data type in Iceberg can be represented using an instance of this class. Binaries in - Iceberg are arbitrary-length byte arrays. + """A Binary data type in Iceberg can be represented using an instance of this class. + + Binaries in Iceberg are arbitrary-length byte arrays. Example: >>> column_foo = BinaryType() diff --git a/python/pyiceberg/utils/bin_packing.py b/python/pyiceberg/utils/bin_packing.py index 10e21693ae5f..7f9d44117a6e 100644 --- a/python/pyiceberg/utils/bin_packing.py +++ b/python/pyiceberg/utils/bin_packing.py @@ -64,9 +64,11 @@ def __init__( self.bins = [] def __iter__(self) -> PackingIterator[T]: + """Returns an iterator for the PackingIterator class.""" return self def __next__(self) -> List[T]: + """Returns the next item when iterating over the PackingIterator class.""" while True: try: item = next(self.items) diff --git a/python/pyiceberg/utils/concurrent.py b/python/pyiceberg/utils/concurrent.py new file mode 100644 index 000000000000..a71ee3281e5b --- /dev/null +++ b/python/pyiceberg/utils/concurrent.py @@ -0,0 +1,67 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# pylint: disable=redefined-outer-name,arguments-renamed,fixme +"""Concurrency concepts that support multi-threading.""" +import threading +from concurrent.futures import Executor, ThreadPoolExecutor +from contextlib import AbstractContextManager +from typing import Any, Generic, TypeVar + +from typing_extensions import Self + +T = TypeVar("T") + + +class Synchronized(Generic[T], AbstractContextManager): # type: ignore + """A context manager that provides concurrency-safe access to a value.""" + + value: T + lock: threading.Lock + + def __init__(self, value: T, lock: threading.Lock): + super().__init__() + self.value = value + self.lock = lock + + def __enter__(self) -> T: + """Acquires a lock, allowing access to the wrapped value.""" + self.lock.acquire() + return self.value + + def __exit__(self, exc_type: Any, exc_value: Any, traceback: Any) -> None: + """Releases the lock, allowing other threads to access the value.""" + self.lock.release() + + +class ManagedExecutor(Executor): + """An executor that provides synchronization.""" + + def synchronized(self, value: T) -> Synchronized[T]: + raise NotImplementedError + + +class ManagedThreadPoolExecutor(ThreadPoolExecutor, ManagedExecutor): + """A thread pool executor that provides synchronization.""" + + def __enter__(self) -> Self: + """Returns the executor itself as a context manager.""" + super().__enter__() + return self + + def synchronized(self, value: T) -> Synchronized[T]: + lock = threading.Lock() + return Synchronized(value, lock) diff --git a/python/pyiceberg/utils/config.py b/python/pyiceberg/utils/config.py index 5c0409807458..e6cbeb78d616 100644 --- a/python/pyiceberg/utils/config.py +++ b/python/pyiceberg/utils/config.py @@ -33,7 +33,7 @@ def merge_config(lhs: RecursiveDict, rhs: RecursiveDict) -> RecursiveDict: - """merges right-hand side into the left-hand side""" + """Merges right-hand side into the left-hand side.""" new_config = lhs.copy() for rhs_key, rhs_value in rhs.items(): if rhs_key in new_config: @@ -52,7 +52,7 @@ def merge_config(lhs: RecursiveDict, rhs: RecursiveDict) -> RecursiveDict: def _lowercase_dictionary_keys(input_dict: RecursiveDict) -> RecursiveDict: - """Lowers all the keys of a dictionary in a recursive manner, to make the lookup case-insensitive""" + """Lowers all the keys of a dictionary in a recursive manner, to make the lookup case-insensitive.""" return {k.lower(): _lowercase_dictionary_keys(v) if isinstance(v, dict) else v for k, v in input_dict.items()} @@ -66,7 +66,7 @@ def __init__(self) -> None: @staticmethod def _from_configuration_files() -> Optional[RecursiveDict]: - """Loads the first configuration file that its finds + """Loads the first configuration file that its finds. Will first look in the PYICEBERG_HOME env variable, and then in the home directory. @@ -94,13 +94,13 @@ def _load_yaml(directory: Optional[str]) -> Optional[RecursiveDict]: @staticmethod def _from_environment_variables(config: RecursiveDict) -> RecursiveDict: - """Reads the environment variables, to check if there are any prepended by PYICEBERG_ + """Reads the environment variables, to check if there are any prepended by PYICEBERG_. Args: - config: Existing configuration that's being amended with configuration from environment variables + config: Existing configuration that's being amended with configuration from environment variables. Returns: - Amended configuration + Amended configuration. """ def set_property(_config: RecursiveDict, path: List[str], config_value: str) -> None: @@ -132,12 +132,10 @@ def set_property(_config: RecursiveDict, path: List[str], config_value: str) -> return config def get_default_catalog_name(self) -> str: - """ - Looks into the configuration file for `default-catalog` - and returns the name as the default catalog + """Returns the default catalog name. - Returns: The name of the default catalog in `default-catalog` - Returns `default` when the key cannot be found. + Returns: The name of the default catalog in `default-catalog`. + Returns `default` when the key cannot be found in the config file. """ if default_catalog_name := self.config.get(DEFAULT_CATALOG): if not isinstance(default_catalog_name, str): diff --git a/python/pyiceberg/utils/datetime.py b/python/pyiceberg/utils/datetime.py index 2d3664103f6b..71a62ca45607 100644 --- a/python/pyiceberg/utils/datetime.py +++ b/python/pyiceberg/utils/datetime.py @@ -14,8 +14,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -"""Helper methods for working with date/time representations -""" +"""Helper methods for working with date/time representations.""" from __future__ import annotations import re @@ -34,12 +33,12 @@ def micros_to_days(timestamp: int) -> int: - """Converts a timestamp in microseconds to a date in days""" + """Converts a timestamp in microseconds to a date in days.""" return timedelta(microseconds=timestamp).days def micros_to_time(micros: int) -> time: - """Converts a timestamp in microseconds to a time""" + """Converts a timestamp in microseconds to a time.""" micros, microseconds = divmod(micros, 1000000) micros, seconds = divmod(micros, 60) micros, minutes = divmod(micros, 60) @@ -48,28 +47,33 @@ def micros_to_time(micros: int) -> time: def date_str_to_days(date_str: str) -> int: - """Converts an ISO-8601 formatted date to days from 1970-01-01""" + """Converts an ISO-8601 formatted date to days from 1970-01-01.""" return (date.fromisoformat(date_str) - EPOCH_DATE).days def date_to_days(date_val: date) -> int: - """Converts a Python date object to days from 1970-01-01""" + """Converts a Python date object to days from 1970-01-01.""" return (date_val - EPOCH_DATE).days def days_to_date(days: int) -> date: - """Creates a date from the number of days from 1970-01-01""" + """Creates a date from the number of days from 1970-01-01.""" return EPOCH_DATE + timedelta(days) def time_to_micros(time_str: str) -> int: - """Converts an ISO-8601 formatted time to microseconds from midnight""" + """Converts an ISO-8601 formatted time to microseconds from midnight.""" t = time.fromisoformat(time_str) return (((t.hour * 60 + t.minute) * 60) + t.second) * 1_000_000 + t.microsecond +def time_object_to_micros(t: time) -> int: + """Converts an datetime.time object to microseconds from midnight.""" + return int(t.hour * 60 * 60 * 1e6 + t.minute * 60 * 1e6 + t.second * 1e6 + t.microsecond) + + def datetime_to_micros(dt: datetime) -> int: - """Converts a datetime to microseconds from 1970-01-01T00:00:00.000000""" + """Converts a datetime to microseconds from 1970-01-01T00:00:00.000000.""" if dt.tzinfo: delta = dt - EPOCH_TIMESTAMPTZ else: @@ -78,7 +82,7 @@ def datetime_to_micros(dt: datetime) -> int: def timestamp_to_micros(timestamp_str: str) -> int: - """Converts an ISO-9601 formatted timestamp without zone to microseconds from 1970-01-01T00:00:00.000000""" + """Converts an ISO-9601 formatted timestamp without zone to microseconds from 1970-01-01T00:00:00.000000.""" if ISO_TIMESTAMP.fullmatch(timestamp_str): return datetime_to_micros(datetime.fromisoformat(timestamp_str)) if ISO_TIMESTAMPTZ.fullmatch(timestamp_str): @@ -87,8 +91,17 @@ def timestamp_to_micros(timestamp_str: str) -> int: raise ValueError(f"Invalid timestamp without zone: {timestamp_str} (must be ISO-8601)") +def datetime_to_millis(dt: datetime) -> int: + """Converts a datetime to milliseconds from 1970-01-01T00:00:00.000000.""" + if dt.tzinfo: + delta = dt - EPOCH_TIMESTAMPTZ + else: + delta = dt - EPOCH_TIMESTAMP + return (delta.days * 86400 + delta.seconds) * 1_000 + delta.microseconds // 1_000 + + def timestamptz_to_micros(timestamptz_str: str) -> int: - """Converts an ISO-8601 formatted timestamp with zone to microseconds from 1970-01-01T00:00:00.000000+00:00""" + """Converts an ISO-8601 formatted timestamp with zone to microseconds from 1970-01-01T00:00:00.000000+00:00.""" if ISO_TIMESTAMPTZ.fullmatch(timestamptz_str): return datetime_to_micros(datetime.fromisoformat(timestamptz_str)) if ISO_TIMESTAMP.fullmatch(timestamptz_str): @@ -98,54 +111,54 @@ def timestamptz_to_micros(timestamptz_str: str) -> int: def micros_to_timestamp(micros: int) -> datetime: - """Converts microseconds from epoch to a timestamp""" + """Converts microseconds from epoch to a timestamp.""" dt = timedelta(microseconds=micros) return EPOCH_TIMESTAMP + dt def micros_to_timestamptz(micros: int) -> datetime: - """Converts microseconds from epoch to an utc timestamp""" + """Converts microseconds from epoch to an utc timestamp.""" dt = timedelta(microseconds=micros) return EPOCH_TIMESTAMPTZ + dt def to_human_year(year_ordinal: int) -> str: - """Converts a DateType value to human string""" + """Converts a DateType value to human string.""" return f"{EPOCH_TIMESTAMP.year + year_ordinal:0=4d}" def to_human_month(month_ordinal: int) -> str: - """Converts a DateType value to human string""" + """Converts a DateType value to human string.""" return f"{EPOCH_TIMESTAMP.year + month_ordinal // 12:0=4d}-{1 + month_ordinal % 12:0=2d}" def to_human_day(day_ordinal: int) -> str: - """Converts a DateType value to human string""" + """Converts a DateType value to human string.""" return (EPOCH_DATE + timedelta(days=day_ordinal)).isoformat() def to_human_hour(hour_ordinal: int) -> str: - """Converts a DateType value to human string""" + """Converts a DateType value to human string.""" return (EPOCH_TIMESTAMP + timedelta(hours=hour_ordinal)).isoformat("-", "hours") def to_human_time(micros_from_midnight: int) -> str: - """Converts a TimeType value to human string""" + """Converts a TimeType value to human string.""" return micros_to_time(micros_from_midnight).isoformat() def to_human_timestamptz(timestamp_micros: int) -> str: - """Converts a TimestamptzType value to human string""" + """Converts a TimestamptzType value to human string.""" return (EPOCH_TIMESTAMPTZ + timedelta(microseconds=timestamp_micros)).isoformat() def to_human_timestamp(timestamp_micros: int) -> str: - """Converts a TimestampType value to human string""" + """Converts a TimestampType value to human string.""" return (EPOCH_TIMESTAMP + timedelta(microseconds=timestamp_micros)).isoformat() def micros_to_hours(micros: int) -> int: - """Converts a timestamp in microseconds to hours from 1970-01-01T00:00""" + """Converts a timestamp in microseconds to hours from 1970-01-01T00:00.""" return micros // 3_600_000_000 diff --git a/python/pyiceberg/utils/decimal.py b/python/pyiceberg/utils/decimal.py index fb58034e68db..32f6db6aa949 100644 --- a/python/pyiceberg/utils/decimal.py +++ b/python/pyiceberg/utils/decimal.py @@ -15,47 +15,46 @@ # specific language governing permissions and limitations # under the License. -"""Helper methods for working with Python Decimals -""" +"""Helper methods for working with Python Decimals.""" from decimal import Decimal from typing import Union def decimal_to_unscaled(value: Decimal) -> int: - """Get an unscaled value given a Decimal value + """Get an unscaled value given a Decimal value. Args: - value (Decimal): A Decimal instance + value (Decimal): A Decimal instance. Returns: - int: The unscaled value + int: The unscaled value. """ sign, digits, _ = value.as_tuple() return int(Decimal((sign, digits, 0)).to_integral_value()) def unscaled_to_decimal(unscaled: int, scale: int) -> Decimal: - """Get a scaled Decimal value given an unscaled value and a scale + """Get a scaled Decimal value given an unscaled value and a scale. Args: - unscaled (int): An unscaled value - scale (int): A scale to set for the returned Decimal instance + unscaled (int): An unscaled value. + scale (int): A scale to set for the returned Decimal instance. Returns: - Decimal: A scaled Decimal instance + Decimal: A scaled Decimal instance. """ sign, digits, _ = Decimal(unscaled).as_tuple() return Decimal((sign, digits, -scale)) def bytes_required(value: Union[int, Decimal]) -> int: - """Returns the minimum number of bytes needed to serialize a decimal or unscaled value + """Returns the minimum number of bytes needed to serialize a decimal or unscaled value. Args: - value (int | Decimal): a Decimal value or unscaled int value + value (int | Decimal): a Decimal value or unscaled int value. Returns: - int: the minimum number of bytes needed to serialize the value + int: the minimum number of bytes needed to serialize the value. """ if isinstance(value, int): return (value.bit_length() + 7) // 8 @@ -66,24 +65,25 @@ def bytes_required(value: Union[int, Decimal]) -> int: def decimal_to_bytes(value: Decimal) -> bytes: - """Returns a byte representation of a decimal + """Returns a byte representation of a decimal. Args: - value (Decimal): a decimal value + value (Decimal): a decimal value. Returns: - bytes: the unscaled value of the Decimal as bytes + bytes: the unscaled value of the Decimal as bytes. """ unscaled_value = decimal_to_unscaled(value) return unscaled_value.to_bytes(bytes_required(unscaled_value), byteorder="big", signed=True) def truncate_decimal(value: Decimal, width: int) -> Decimal: - """Get a truncated Decimal value given a decimal value and a width + """Get a truncated Decimal value given a decimal value and a width. + Args: - value (Decimal): a decimal value - width (int): A width for the returned Decimal instance + value (Decimal): a decimal value. + width (int): A width for the returned Decimal instance. Returns: - Decimal: A truncated Decimal instance + Decimal: A truncated Decimal instance. """ unscaled_value = decimal_to_unscaled(value) applied_value = unscaled_value - (((unscaled_value % width) + width) % width) diff --git a/python/pyiceberg/utils/deprecated.py b/python/pyiceberg/utils/deprecated.py index 2fc4ec39ed32..d39498c5540a 100644 --- a/python/pyiceberg/utils/deprecated.py +++ b/python/pyiceberg/utils/deprecated.py @@ -20,10 +20,10 @@ def deprecated(deprecated_in: str, removed_in: str, help_message: Optional[str] = None) -> Callable: # type: ignore - """This is a decorator which can be used to mark functions - as deprecated. It will result in a warning being emitted - when the function is used.""" + """A decorator which can be used to mark functions as deprecated. + Adding this will result in a warning being emitted when the function is used. + """ if help_message is not None: help_message = f" {help_message}." diff --git a/python/pyiceberg/utils/parsing.py b/python/pyiceberg/utils/parsing.py index c2a254a2a9d1..22786a1d6809 100644 --- a/python/pyiceberg/utils/parsing.py +++ b/python/pyiceberg/utils/parsing.py @@ -19,7 +19,7 @@ class ParseNumberFromBrackets: - """Extracts the size from a string in the form of prefix[22]""" + """Extracts the size from a string in the form of prefix[22].""" regex: Pattern # type: ignore prefix: str diff --git a/python/pyiceberg/utils/schema_conversion.py b/python/pyiceberg/utils/schema_conversion.py index 2f9c321a1361..4f46668866da 100644 --- a/python/pyiceberg/utils/schema_conversion.py +++ b/python/pyiceberg/utils/schema_conversion.py @@ -14,19 +14,18 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -"""Utility class for converting between Avro and Iceberg schemas""" -from __future__ import annotations - +"""Utility class for converting between Avro and Iceberg schemas.""" import logging from typing import ( Any, Dict, List, + Optional, Tuple, Union, ) -from pyiceberg.schema import Schema +from pyiceberg.schema import Schema, SchemaVisitorPerPrimitiveType, visit from pyiceberg.types import ( BinaryType, BooleanType, @@ -45,13 +44,14 @@ StringType, StructType, TimestampType, + TimestamptzType, TimeType, UUIDType, ) logger = logging.getLogger(__name__) -PRIMITIVE_FIELD_TYPE_MAPPING: dict[str, PrimitiveType] = { +PRIMITIVE_FIELD_TYPE_MAPPING: Dict[str, PrimitiveType] = { "boolean": BooleanType(), "bytes": BinaryType(), "double": DoubleType(), @@ -62,7 +62,7 @@ "enum": StringType(), } -LOGICAL_FIELD_TYPE_MAPPING: dict[tuple[str, str], PrimitiveType] = { +LOGICAL_FIELD_TYPE_MAPPING: Dict[Tuple[str, str], PrimitiveType] = { ("date", "int"): DateType(), ("time-millis", "int"): TimeType(), ("timestamp-millis", "long"): TimestampType(), @@ -71,12 +71,14 @@ ("uuid", "fixed"): UUIDType(), } +AvroType = Union[str, Any] + class AvroSchemaConversion: - def avro_to_iceberg(self, avro_schema: dict[str, Any]) -> Schema: - """Converts an Apache Avro into an Apache Iceberg schema equivalent + def avro_to_iceberg(self, avro_schema: Dict[str, Any]) -> Schema: + """Converts an Apache Avro into an Apache Iceberg schema equivalent. - This expects to have field id's to be encoded in the Avro schema:: + This expects to have field id's to be encoded in the Avro schema: { "type": "record", @@ -111,18 +113,22 @@ def avro_to_iceberg(self, avro_schema: dict[str, Any]) -> Schema: True Args: - avro_schema (Dict[str, Any]): The JSON decoded Avro schema + avro_schema (Dict[str, Any]): The JSON decoded Avro schema. Returns: - Equivalent Iceberg schema + Equivalent Iceberg schema. """ return Schema(*[self._convert_field(field) for field in avro_schema["fields"]], schema_id=1) + def iceberg_to_avro(self, schema: Schema, schema_name: Optional[str] = None) -> AvroType: + """Converts an Iceberg schema into an Avro dictionary that can be serialized to JSON.""" + return visit(schema, ConvertSchemaToAvro(schema_name)) + def _resolve_union( self, type_union: Union[Dict[str, str], List[Union[str, Dict[str, str]]], str] ) -> Tuple[Union[str, Dict[str, Any]], bool]: """ - Converts Unions into their type and resolves if the field is required + Converts Unions into their type and resolves if the field is required. Examples: >>> AvroSchemaConversion()._resolve_union('str') @@ -135,13 +141,13 @@ def _resolve_union( ({'type': 'str'}, False) Args: - type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'} + type_union: The field, can be a string 'str', list ['null', 'str'], or dict {"type": 'str'}. Returns: - A tuple containing the type and if required + A tuple containing the type and if required. Raises: - TypeError: In the case non-optional union types are encountered + TypeError: In the case non-optional union types are encountered. """ avro_types: Union[Dict[str, str], List[Union[Dict[str, str], str]]] if isinstance(type_union, str): @@ -171,13 +177,13 @@ def _resolve_union( def _convert_schema(self, avro_type: Union[str, Dict[str, Any]]) -> IcebergType: """ - Resolves the Avro type + Resolves the Avro type. Args: - avro_type: The Avro type, can be simple or complex + avro_type: The Avro type, can be simple or complex. Returns: - The equivalent IcebergType + The equivalent IcebergType. Raises: ValueError: When there are unknown types @@ -207,14 +213,14 @@ def _convert_schema(self, avro_type: Union[str, Dict[str, Any]]) -> IcebergType: else: raise TypeError(f"Unknown type: {avro_type}") - def _convert_field(self, field: dict[str, Any]) -> NestedField: - """ - Converts an Avro field into an Iceberg equivalent field + def _convert_field(self, field: Dict[str, Any]) -> NestedField: + """Converts an Avro field into an Iceberg equivalent field. + Args: - field: The Avro field + field: The Avro field. Returns: - The Iceberg equivalent field + The Iceberg equivalent field. """ if "field-id" not in field: raise ValueError(f"Cannot convert field, missing field-id: {field}") @@ -229,9 +235,9 @@ def _convert_field(self, field: dict[str, Any]) -> NestedField: doc=field.get("doc"), ) - def _convert_record_type(self, record_type: dict[str, Any]) -> StructType: + def _convert_record_type(self, record_type: Dict[str, Any]) -> StructType: """ - Converts the fields from a record into an Iceberg struct + Converts the fields from a record into an Iceberg struct. Examples: >>> from pyiceberg.utils.schema_conversion import AvroSchemaConversion @@ -274,16 +280,16 @@ def _convert_record_type(self, record_type: dict[str, Any]) -> StructType: True Args: - record_type: The record type itself + record_type: The record type itself. - Returns: + Returns: A StructType. """ if record_type["type"] != "record": raise ValueError(f"Expected record type, got: {record_type}") return StructType(*[self._convert_field(field) for field in record_type["fields"]]) - def _convert_array_type(self, array_type: dict[str, Any]) -> ListType: + def _convert_array_type(self, array_type: Dict[str, Any]) -> ListType: if "element-id" not in array_type: raise ValueError(f"Cannot convert array-type, missing element-id: {array_type}") @@ -295,10 +301,11 @@ def _convert_array_type(self, array_type: dict[str, Any]) -> ListType: element_required=element_required, ) - def _convert_map_type(self, map_type: dict[str, Any]) -> MapType: - """ + def _convert_map_type(self, map_type: Dict[str, Any]) -> MapType: + """Converts an avro map type into an Iceberg MapType. + Args: - map_type: The dict that describes the Avro map type + map_type: The dict that describes the Avro map type. Examples: >>> from pyiceberg.utils.schema_conversion import AvroSchemaConversion @@ -319,7 +326,7 @@ def _convert_map_type(self, map_type: dict[str, Any]) -> MapType: >>> actual == expected True - Returns: A MapType + Returns: A MapType. """ value_type, value_required = self._resolve_union(map_type["values"]) return MapType( @@ -331,11 +338,11 @@ def _convert_map_type(self, map_type: dict[str, Any]) -> MapType: value_required=value_required, ) - def _convert_logical_type(self, avro_logical_type: dict[str, Any]) -> IcebergType: - """ - Convert a schema with a logical type annotation. For the decimal and map - we need to fetch more keys from the dict, and for the simple ones we can just - look it up in the mapping. + def _convert_logical_type(self, avro_logical_type: Dict[str, Any]) -> IcebergType: + """Convert a schema with a logical type annotation into an IcebergType. + + For the decimal and map we need to fetch more keys from the dict, and for + the simple ones we can just look it up in the mapping. Examples: >>> from pyiceberg.utils.schema_conversion import AvroSchemaConversion @@ -348,13 +355,13 @@ def _convert_logical_type(self, avro_logical_type: dict[str, Any]) -> IcebergTyp True Args: - avro_logical_type: The logical type + avro_logical_type: The logical type. Returns: - The converted logical type + The converted logical type. Raises: - ValueError: When the logical type is unknown + ValueError: When the logical type is unknown. """ logical_type = avro_logical_type["logicalType"] physical_type = avro_logical_type["type"] @@ -367,10 +374,11 @@ def _convert_logical_type(self, avro_logical_type: dict[str, Any]) -> IcebergTyp else: raise ValueError(f"Unknown logical/physical type combination: {avro_logical_type}") - def _convert_logical_decimal_type(self, avro_type: dict[str, Any]) -> DecimalType: - """ + def _convert_logical_decimal_type(self, avro_type: Dict[str, Any]) -> DecimalType: + """Converts an avro type to an Iceberg DecimalType. + Args: - avro_type: The Avro type + avro_type: The Avro type. Examples: >>> from pyiceberg.utils.schema_conversion import AvroSchemaConversion @@ -389,17 +397,17 @@ def _convert_logical_decimal_type(self, avro_type: dict[str, Any]) -> DecimalTyp True Returns: - A Iceberg DecimalType + A Iceberg DecimalType. """ return DecimalType(precision=avro_type["precision"], scale=avro_type["scale"]) - def _convert_logical_map_type(self, avro_type: dict[str, Any]) -> MapType: - """ - In the case where a map hasn't a key as a type you can use a logical map to - still encode this in Avro + def _convert_logical_map_type(self, avro_type: Dict[str, Any]) -> MapType: + """Converts an avro map type to an Iceberg MapType. + + In the case where a map hasn't a key as a type you can use a logical map to still encode this in Avro. Args: - avro_type: The Avro Type + avro_type: The Avro Type. Examples: >>> from pyiceberg.utils.schema_conversion import AvroSchemaConversion @@ -430,7 +438,7 @@ def _convert_logical_map_type(self, avro_type: dict[str, Any]) -> MapType: https://iceberg.apache.org/spec/#appendix-a-format-specific-requirements Returns: - The logical map + The logical map. """ fields = avro_type["items"]["fields"] if len(fields) != 2: @@ -445,12 +453,14 @@ def _convert_logical_map_type(self, avro_type: dict[str, Any]) -> MapType: value_required=value.required, ) - def _convert_fixed_type(self, avro_type: dict[str, Any]) -> FixedType: + def _convert_fixed_type(self, avro_type: Dict[str, Any]) -> FixedType: """ - https://avro.apache.org/docs/current/spec.html#Fixed + Converts Avro Type to the equivalent Iceberg fixed type. + + - https://avro.apache.org/docs/current/spec.html#Fixed Args: - avro_type: The Avro Type + avro_type: The Avro type. Examples: >>> from pyiceberg.utils.schema_conversion import AvroSchemaConversion @@ -463,6 +473,132 @@ def _convert_fixed_type(self, avro_type: dict[str, Any]) -> FixedType: True Returns: - An Iceberg equivalent fixed type + An Iceberg equivalent fixed type. """ return FixedType(length=avro_type["size"]) + + +class ConvertSchemaToAvro(SchemaVisitorPerPrimitiveType[AvroType]): + """Converts an Iceberg schema to an Avro schema.""" + + schema_name: Optional[str] + last_list_field_id: int + last_map_key_field_id: int + last_map_value_field_id: int + + def __init__(self, schema_name: Optional[str]) -> None: + """Converts an Iceberg schema to an Avro schema. + + Args: + schema_name: The name of the root record. + """ + self.schema_name = schema_name + + def schema(self, schema: Schema, struct_result: AvroType) -> AvroType: + if isinstance(struct_result, dict) and self.schema_name is not None: + struct_result["name"] = self.schema_name + return struct_result + + def before_list_element(self, element: NestedField) -> None: + self.last_list_field_id = element.field_id + + def before_map_key(self, key: NestedField) -> None: + self.last_map_key_field_id = key.field_id + + def before_map_value(self, value: NestedField) -> None: + self.last_map_value_field_id = value.field_id + + def struct(self, struct: StructType, field_results: List[AvroType]) -> AvroType: + return {"type": "record", "fields": field_results} + + def field(self, field: NestedField, field_result: AvroType) -> AvroType: + # Sets the schema name + if isinstance(field_result, dict) and field_result.get("type") == "record": + field_result["name"] = f"r{field.field_id}" + + result = { + "name": field.name, + "field-id": field.field_id, + "type": field_result if field.required else ["null", field_result], + } + + if field.optional: + result["default"] = None + + if field.doc is not None: + result["doc"] = field.doc + + return result + + def list(self, list_type: ListType, element_result: AvroType) -> AvroType: + # Sets the schema name in case of a record + if isinstance(element_result, dict) and element_result.get("type") == "record": + element_result["name"] = f"r{self.last_list_field_id}" + return {"type": "array", "element-id": self.last_list_field_id, "items": element_result} + + def map(self, map_type: MapType, key_result: AvroType, value_result: AvroType) -> AvroType: + if isinstance(key_result, StringType): + # Avro Maps does not support other keys than a String, + return { + "type": "map", + "values": value_result, + } + else: + # Creates a logical map that's a list of schema's + # binary compatible + return { + "type": "array", + "items": { + "type": "record", + "name": f"k{self.last_map_key_field_id}_v{self.last_map_value_field_id}", + "fields": [ + {"name": "key", "type": key_result, "field-id": self.last_map_key_field_id}, + {"name": "value", "type": value_result, "field-id": self.last_map_value_field_id}, + ], + }, + "logicalType": "map", + } + + def visit_fixed(self, fixed_type: FixedType) -> AvroType: + return {"type": "fixed", "size": len(fixed_type)} + + def visit_decimal(self, decimal_type: DecimalType) -> AvroType: + return {"type": "bytes", "logicalType": "decimal", "precision": decimal_type.precision, "scale": decimal_type.scale} + + def visit_boolean(self, boolean_type: BooleanType) -> AvroType: + return "boolean" + + def visit_integer(self, integer_type: IntegerType) -> AvroType: + return "int" + + def visit_long(self, long_type: LongType) -> AvroType: + return "long" + + def visit_float(self, float_type: FloatType) -> AvroType: + return "float" + + def visit_double(self, double_type: DoubleType) -> AvroType: + return "double" + + def visit_date(self, date_type: DateType) -> AvroType: + return {"type": "int", "logicalType": "date"} + + def visit_time(self, time_type: TimeType) -> AvroType: + return {"type": "long", "logicalType": "time-micros"} + + def visit_timestamp(self, timestamp_type: TimestampType) -> AvroType: + # Iceberg only supports micro's + return {"type": "long", "logicalType": "timestamp-micros"} + + def visit_timestamptz(self, timestamptz_type: TimestamptzType) -> AvroType: + # Iceberg only supports micro's + return {"type": "long", "logicalType": "timestamp-micros"} + + def visit_string(self, string_type: StringType) -> AvroType: + return "string" + + def visit_uuid(self, uuid_type: UUIDType) -> AvroType: + return {"type": "string", "logicalType": "uuid"} + + def visit_binary(self, binary_type: BinaryType) -> AvroType: + return "bytes" diff --git a/python/pyiceberg/utils/singleton.py b/python/pyiceberg/utils/singleton.py index 709f36a7050b..9380b89cbc91 100644 --- a/python/pyiceberg/utils/singleton.py +++ b/python/pyiceberg/utils/singleton.py @@ -15,7 +15,7 @@ # specific language governing permissions and limitations # under the License. """ -This is a singleton metaclass that can be used to cache and re-use existing objects +This is a singleton metaclass that can be used to cache and re-use existing objects. In the Iceberg codebase we have a lot of objects that are stateless (for example Types such as StringType, BooleanType etc). FixedTypes have arguments (eg. Fixed[22]) that we also make part of the key when caching diff --git a/python/pyproject.toml b/python/pyproject.toml index 9690bbe884cc..be0bfaf73a86 100644 --- a/python/pyproject.toml +++ b/python/pyproject.toml @@ -16,7 +16,7 @@ # under the License. [tool.poetry] name = "pyiceberg" -version = "0.3.0" +version = "0.4.0" readme = "README.md" homepage = "https://iceberg.apache.org/" repository = "https://github.com/apache/iceberg/" @@ -49,8 +49,9 @@ click = ">=7.1.1,<9.0.0" rich = ">=10.11.0,<14.0.0" strictyaml = ">=1.7.0,<2.0.0" # CVE-2020-14343 was fixed in 5.4. pydantic = ">=1.9.0,<2.0.0" +sortedcontainers = "2.4.0" fsspec = ">=2021.09.0,<2024.1.0" # `lexists()` was implemented in 2021.09.0. Upper bound set arbitrarily, to be reassessed in early 2024. -pyparsing = ">=3.0.7,<4.0.0" # The `min` keyword argument for `delimited_list()` was added in 3.0.7. +pyparsing = ">=3.1.0,<4.0.0" zstandard = ">=0.13.0,<1.0.0" pyarrow = { version = ">=9.0.0,<13.0.0", optional = true } pandas = { version = ">=1.0.0,<3.0.0", optional = true } @@ -61,16 +62,23 @@ thrift = { version = ">=0.13.0,<1.0.0", optional = true } boto3 = { version = ">=1.17.106", optional = true } s3fs = { version = ">=2021.08.0,<2024.1.0", optional = true } # Upper bound set arbitrarily, to be reassessed in early 2024. adlfs = { version = ">=2021.07.0,<2024.1.0", optional = true } # Upper bound set arbitrarily, to be reassessed in early 2024. +psycopg2-binary = { version = ">=2.9.6", optional = true } +sqlalchemy = { version = "^2.0.18", optional = true } [tool.poetry.dev-dependencies] -pytest = "7.3.1" -pytest-checkdocs = "2.9.0" -pre-commit = "3.3.2" -fastavro = "1.7.4" +pytest = "7.4.0" +pytest-checkdocs = "2.10.0" +pre-commit = "3.3.3" +fastavro = "1.8.2" coverage = { version = "^7.2.3", extras = ["toml"] } -requests-mock = "1.10.0" -moto = "^4.1.7" -typing-extensions = "^4.6.3" +requests-mock = "1.11.0" +moto = "^4.1.13" +typing-extensions = "4.7.1" +pytest-mock = "3.11.1" + +[[tool.mypy.overrides]] +module = "pytest_mock.*" +ignore_missing_imports = true [tool.poetry.scripts] pyiceberg = "pyiceberg.cli.console:run" @@ -91,6 +99,7 @@ glue = ["boto3"] adlfs = ["adlfs"] dynamodb = ["boto3"] zstandard = ["zstandard"] +sql-postgres = ["sqlalchemy", "psycopg2-binary"] [tool.pytest.ini_options] markers = [ @@ -238,5 +247,17 @@ ignore_missing_imports = true module = "strictyaml.*" ignore_missing_imports = true +[[tool.mypy.overrides]] +module = "sortedcontainers.*" +ignore_missing_imports = true + +[[tool.mypy.overrides]] +module = "numpy.*" +ignore_missing_imports = true + +[[tool.mypy.overrides]] +module = "sqlalchemy.*" +ignore_missing_imports = true + [tool.coverage.run] source = ['pyiceberg/'] diff --git a/python/tests/avro/test_decoder.py b/python/tests/avro/test_decoder.py index 35405c8badcd..b1ab97fb1d73 100644 --- a/python/tests/avro/test_decoder.py +++ b/python/tests/avro/test_decoder.py @@ -16,6 +16,7 @@ # under the License. from __future__ import annotations +import io from datetime import datetime, timezone from decimal import Decimal from io import SEEK_SET @@ -25,74 +26,84 @@ import pytest -from pyiceberg.avro.decoder import BinaryDecoder +from pyiceberg.avro.decoder import BinaryDecoder, InMemoryBinaryDecoder, StreamingBinaryDecoder from pyiceberg.avro.resolver import resolve from pyiceberg.io import InputStream -from pyiceberg.io.memory import MemoryInputStream from pyiceberg.types import DoubleType, FloatType +AVAILABLE_DECODERS = [StreamingBinaryDecoder, InMemoryBinaryDecoder] -def test_read_decimal_from_fixed() -> None: - mis = MemoryInputStream(b"\x00\x00\x00\x05\x6A\x48\x1C\xFB\x2C\x7C\x50\x00") - decoder = BinaryDecoder(mis) + +@pytest.mark.parametrize("decoder_class", AVAILABLE_DECODERS) +def test_read_decimal_from_fixed(decoder_class: Type[BinaryDecoder]) -> None: + mis = io.BytesIO(b"\x00\x00\x00\x05\x6A\x48\x1C\xFB\x2C\x7C\x50\x00") + decoder = decoder_class(mis) actual = decoder.read_decimal_from_fixed(28, 15, 12) expected = Decimal("99892.123400000000000") assert actual == expected -def test_read_boolean_true() -> None: - mis = MemoryInputStream(b"\x01") - decoder = BinaryDecoder(mis) +@pytest.mark.parametrize("decoder_class", AVAILABLE_DECODERS) +def test_read_boolean_true(decoder_class: Type[BinaryDecoder]) -> None: + mis = io.BytesIO(b"\x01") + decoder = decoder_class(mis) assert decoder.read_boolean() is True -def test_read_boolean_false() -> None: - mis = MemoryInputStream(b"\x00") - decoder = BinaryDecoder(mis) +@pytest.mark.parametrize("decoder_class", AVAILABLE_DECODERS) +def test_read_boolean_false(decoder_class: Type[BinaryDecoder]) -> None: + mis = io.BytesIO(b"\x00") + decoder = decoder_class(mis) assert decoder.read_boolean() is False -def test_skip_boolean() -> None: - mis = MemoryInputStream(b"\x00") - decoder = BinaryDecoder(mis) - assert mis.tell() == 0 +@pytest.mark.parametrize("decoder_class", AVAILABLE_DECODERS) +def test_skip_boolean(decoder_class: Type[BinaryDecoder]) -> None: + mis = io.BytesIO(b"\x00") + decoder = decoder_class(mis) + assert decoder.tell() == 0 decoder.skip_boolean() - assert mis.tell() == 1 + assert decoder.tell() == 1 -def test_read_int() -> None: - mis = MemoryInputStream(b"\x18") - decoder = BinaryDecoder(mis) +@pytest.mark.parametrize("decoder_class", AVAILABLE_DECODERS) +def test_read_int(decoder_class: Type[BinaryDecoder]) -> None: + mis = io.BytesIO(b"\x18") + decoder = decoder_class(mis) assert decoder.read_int() == 12 -def test_skip_int() -> None: - mis = MemoryInputStream(b"\x18") - decoder = BinaryDecoder(mis) - assert mis.tell() == 0 +@pytest.mark.parametrize("decoder_class", AVAILABLE_DECODERS) +def test_skip_int(decoder_class: Type[BinaryDecoder]) -> None: + mis = io.BytesIO(b"\x18") + decoder = decoder_class(mis) + assert decoder.tell() == 0 decoder.skip_int() - assert mis.tell() == 1 + assert decoder.tell() == 1 -def test_read_decimal() -> None: - mis = MemoryInputStream(b"\x18\x00\x00\x00\x05\x6A\x48\x1C\xFB\x2C\x7C\x50\x00") - decoder = BinaryDecoder(mis) +@pytest.mark.parametrize("decoder_class", AVAILABLE_DECODERS) +def test_read_decimal(decoder_class: Type[BinaryDecoder]) -> None: + mis = io.BytesIO(b"\x18\x00\x00\x00\x05\x6A\x48\x1C\xFB\x2C\x7C\x50\x00") + decoder = decoder_class(mis) actual = decoder.read_decimal_from_bytes(28, 15) expected = Decimal("99892.123400000000000") assert actual == expected -def test_decimal_from_fixed_big() -> None: - mis = MemoryInputStream(b"\x0E\xC2\x02\xE9\x06\x16\x33\x49\x77\x67\xA8\x00") - decoder = BinaryDecoder(mis) +@pytest.mark.parametrize("decoder_class", AVAILABLE_DECODERS) +def test_decimal_from_fixed_big(decoder_class: Type[BinaryDecoder]) -> None: + mis = io.BytesIO(b"\x0E\xC2\x02\xE9\x06\x16\x33\x49\x77\x67\xA8\x00") + decoder = decoder_class(mis) actual = decoder.read_decimal_from_fixed(28, 15, 12) expected = Decimal("4567335489766.998340000000000") assert actual == expected -def test_read_negative_bytes() -> None: - mis = MemoryInputStream(b"") - decoder = BinaryDecoder(mis) +@pytest.mark.parametrize("decoder_class", AVAILABLE_DECODERS) +def test_read_negative_bytes(decoder_class: Type[BinaryDecoder]) -> None: + mis = io.BytesIO(b"") + decoder = decoder_class(mis) with pytest.raises(ValueError) as exc_info: decoder.read(-1) @@ -130,92 +141,107 @@ def __exit__( self.close() -def test_read_single_byte_at_the_time() -> None: - decoder = BinaryDecoder(OneByteAtATimeInputStream()) +# InMemoryBinaryDecoder doesn't work for a byte at a time reading +@pytest.mark.parametrize("decoder_class", [StreamingBinaryDecoder]) +def test_read_single_byte_at_the_time(decoder_class: Type[BinaryDecoder]) -> None: + decoder = decoder_class(OneByteAtATimeInputStream()) assert decoder.read(2) == b"\x01\x02" -def test_read_float() -> None: - mis = MemoryInputStream(b"\x00\x00\x9A\x41") - decoder = BinaryDecoder(mis) +@pytest.mark.parametrize("decoder_class", AVAILABLE_DECODERS) +def test_read_float(decoder_class: Type[BinaryDecoder]) -> None: + mis = io.BytesIO(b"\x00\x00\x9A\x41") + decoder = decoder_class(mis) assert decoder.read_float() == 19.25 -def test_skip_float() -> None: - mis = MemoryInputStream(b"\x00\x00\x9A\x41") - decoder = BinaryDecoder(mis) - assert mis.tell() == 0 +@pytest.mark.parametrize("decoder_class", AVAILABLE_DECODERS) +def test_skip_float(decoder_class: Type[BinaryDecoder]) -> None: + mis = io.BytesIO(b"\x00\x00\x9A\x41") + decoder = decoder_class(mis) + assert decoder.tell() == 0 decoder.skip_float() - assert mis.tell() == 4 + assert decoder.tell() == 4 -def test_read_double() -> None: - mis = MemoryInputStream(b"\x00\x00\x00\x00\x00\x40\x33\x40") - decoder = BinaryDecoder(mis) +@pytest.mark.parametrize("decoder_class", AVAILABLE_DECODERS) +def test_read_double(decoder_class: Type[BinaryDecoder]) -> None: + mis = io.BytesIO(b"\x00\x00\x00\x00\x00\x40\x33\x40") + decoder = decoder_class(mis) assert decoder.read_double() == 19.25 -def test_skip_double() -> None: - mis = MemoryInputStream(b"\x00\x00\x00\x00\x00\x40\x33\x40") - decoder = BinaryDecoder(mis) - assert mis.tell() == 0 +@pytest.mark.parametrize("decoder_class", AVAILABLE_DECODERS) +def test_skip_double(decoder_class: Type[BinaryDecoder]) -> None: + mis = io.BytesIO(b"\x00\x00\x00\x00\x00\x40\x33\x40") + decoder = decoder_class(mis) + assert decoder.tell() == 0 decoder.skip_double() - assert mis.tell() == 8 + assert decoder.tell() == 8 -def test_read_uuid_from_fixed() -> None: - mis = MemoryInputStream(b"\x12\x34\x56\x78" * 4) - decoder = BinaryDecoder(mis) +@pytest.mark.parametrize("decoder_class", AVAILABLE_DECODERS) +def test_read_uuid_from_fixed(decoder_class: Type[BinaryDecoder]) -> None: + mis = io.BytesIO(b"\x12\x34\x56\x78" * 4) + decoder = decoder_class(mis) assert decoder.read_uuid_from_fixed() == UUID("{12345678-1234-5678-1234-567812345678}") -def test_read_time_millis() -> None: - mis = MemoryInputStream(b"\xBC\x7D") - decoder = BinaryDecoder(mis) +@pytest.mark.parametrize("decoder_class", AVAILABLE_DECODERS) +def test_read_time_millis(decoder_class: Type[BinaryDecoder]) -> None: + mis = io.BytesIO(b"\xBC\x7D") + decoder = decoder_class(mis) assert decoder.read_time_millis().microsecond == 30000 -def test_read_time_micros() -> None: - mis = MemoryInputStream(b"\xBC\x7D") - decoder = BinaryDecoder(mis) +@pytest.mark.parametrize("decoder_class", AVAILABLE_DECODERS) +def test_read_time_micros(decoder_class: Type[BinaryDecoder]) -> None: + mis = io.BytesIO(b"\xBC\x7D") + decoder = decoder_class(mis) assert decoder.read_time_micros().microsecond == 8030 -def test_read_timestamp_micros() -> None: - mis = MemoryInputStream(b"\xBC\x7D") - decoder = BinaryDecoder(mis) +@pytest.mark.parametrize("decoder_class", AVAILABLE_DECODERS) +def test_read_timestamp_micros(decoder_class: Type[BinaryDecoder]) -> None: + mis = io.BytesIO(b"\xBC\x7D") + decoder = decoder_class(mis) assert decoder.read_timestamp_micros() == datetime(1970, 1, 1, 0, 0, 0, 8030) -def test_read_timestamptz_micros() -> None: - mis = MemoryInputStream(b"\xBC\x7D") - decoder = BinaryDecoder(mis) +@pytest.mark.parametrize("decoder_class", AVAILABLE_DECODERS) +def test_read_timestamptz_micros(decoder_class: Type[BinaryDecoder]) -> None: + mis = io.BytesIO(b"\xBC\x7D") + decoder = decoder_class(mis) assert decoder.read_timestamptz_micros() == datetime(1970, 1, 1, 0, 0, 0, 8030, tzinfo=timezone.utc) -def test_read_bytes() -> None: - mis = MemoryInputStream(b"\x08\x01\x02\x03\x04") - decoder = BinaryDecoder(mis) +@pytest.mark.parametrize("decoder_class", AVAILABLE_DECODERS) +def test_read_bytes(decoder_class: Type[BinaryDecoder]) -> None: + mis = io.BytesIO(b"\x08\x01\x02\x03\x04") + decoder = decoder_class(mis) actual = decoder.read_bytes() assert actual == b"\x01\x02\x03\x04" -def test_read_utf8() -> None: - mis = MemoryInputStream(b"\x04\x76\x6F") - decoder = BinaryDecoder(mis) +@pytest.mark.parametrize("decoder_class", AVAILABLE_DECODERS) +def test_read_utf8(decoder_class: Type[BinaryDecoder]) -> None: + mis = io.BytesIO(b"\x04\x76\x6F") + decoder = decoder_class(mis) assert decoder.read_utf8() == "vo" -def test_skip_utf8() -> None: - mis = MemoryInputStream(b"\x04\x76\x6F") - decoder = BinaryDecoder(mis) - assert mis.tell() == 0 +@pytest.mark.parametrize("decoder_class", AVAILABLE_DECODERS) +def test_skip_utf8(decoder_class: Type[BinaryDecoder]) -> None: + mis = io.BytesIO(b"\x04\x76\x6F") + decoder = decoder_class(mis) + assert decoder.tell() == 0 decoder.skip_utf8() - assert mis.tell() == 3 + assert decoder.tell() == 3 -def test_read_int_as_float() -> None: - mis = MemoryInputStream(b"\x00\x00\x9A\x41") - decoder = BinaryDecoder(mis) +@pytest.mark.parametrize("decoder_class", AVAILABLE_DECODERS) +def test_read_int_as_float(decoder_class: Type[BinaryDecoder]) -> None: + mis = io.BytesIO(b"\x00\x00\x9A\x41") + decoder = decoder_class(mis) reader = resolve(FloatType(), DoubleType()) assert reader.read(decoder) == 19.25 diff --git a/python/tests/avro/test_encoder.py b/python/tests/avro/test_encoder.py new file mode 100644 index 000000000000..4646e65e6e61 --- /dev/null +++ b/python/tests/avro/test_encoder.py @@ -0,0 +1,207 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +from __future__ import annotations + +import datetime +import io +import struct +from decimal import Decimal + +from pyiceberg.avro.encoder import BinaryEncoder + + +def test_write() -> None: + output = io.BytesIO() + encoder = BinaryEncoder(output) + + _input = b"\x12\x34\x56" + + encoder.write(_input) + + assert output.getbuffer() == _input + + +def test_write_boolean() -> None: + output = io.BytesIO() + encoder = BinaryEncoder(output) + + encoder.write_boolean(True) + encoder.write_boolean(False) + + assert output.getbuffer() == struct.pack("??", True, False) + + +def test_write_int() -> None: + output = io.BytesIO() + encoder = BinaryEncoder(output) + + _1byte_input = 2 + _2byte_input = 7466 + _3byte_input = 523490 + _4byte_input = 86561570 + _5byte_input = 2510416930 + _6byte_input = 734929016866 + _7byte_input = 135081528772642 + _8byte_input = 35124861473277986 + + encoder.write_int(_1byte_input) + encoder.write_int(_2byte_input) + encoder.write_int(_3byte_input) + encoder.write_int(_4byte_input) + encoder.write_int(_5byte_input) + encoder.write_int(_6byte_input) + encoder.write_int(_7byte_input) + encoder.write_int(_8byte_input) + + buffer = output.getbuffer() + + assert buffer[0:1] == b"\x04" + assert buffer[1:3] == b"\xd4\x74" + assert buffer[3:6] == b"\xc4\xf3\x3f" + assert buffer[6:10] == b"\xc4\xcc\xc6\x52" + assert buffer[10:15] == b"\xc4\xb0\x8f\xda\x12" + assert buffer[15:21] == b"\xc4\xe0\xf6\xd2\xe3\x2a" + assert buffer[21:28] == b"\xc4\xa0\xce\xe8\xe3\xb6\x3d" + assert buffer[28:36] == b"\xc4\xa0\xb2\xae\x83\xf8\xe4\x7c" + + +def test_write_float() -> None: + output = io.BytesIO() + encoder = BinaryEncoder(output) + + _input = 3.14159265359 + + encoder.write_float(_input) + + assert output.getbuffer() == struct.pack(" None: + output = io.BytesIO() + encoder = BinaryEncoder(output) + + _input = 3.14159265359 + + encoder.write_double(_input) + + assert output.getbuffer() == struct.pack(" None: + output = io.BytesIO() + encoder = BinaryEncoder(output) + + _input = Decimal("3.14159265359") + + encoder.write_decimal_bytes(_input) + + assert output.getbuffer() == b"\x0a\x49\x25\x59\xf6\x4f" + + +def test_write_decimal_fixed() -> None: + output = io.BytesIO() + encoder = BinaryEncoder(output) + + _input = Decimal("3.14159265359") + + encoder.write_decimal_fixed(_input, 8) + + assert output.getbuffer() == b"\x00\x00\x00\x49\x25\x59\xf6\x4f" + + +def test_write_bytes() -> None: + output = io.BytesIO() + encoder = BinaryEncoder(output) + + _input = b"\x12\x34\x56" + + encoder.write_bytes(_input) + + assert output.getbuffer() == b"".join([b"\x06", _input]) + + +def test_write_bytes_fixed() -> None: + output = io.BytesIO() + encoder = BinaryEncoder(output) + + _input = b"\x12\x34\x56" + + encoder.write_bytes_fixed(_input) + + assert output.getbuffer() == _input + + +def test_write_utf8() -> None: + output = io.BytesIO() + encoder = BinaryEncoder(output) + + _input = "That, my liege, is how we know the Earth to be banana-shaped." + bin_input = _input.encode() + encoder.write_utf8(_input) + + assert output.getbuffer() == b"".join([b"\x7a", bin_input]) + + +def test_write_date_int() -> None: + output = io.BytesIO() + encoder = BinaryEncoder(output) + + _input = datetime.date(1970, 1, 2) + encoder.write_date_int(_input) + + assert output.getbuffer() == b"\x02" + + +def test_write_time_millis_int() -> None: + output = io.BytesIO() + encoder = BinaryEncoder(output) + + _input = datetime.time(1, 2, 3, 456000) + encoder.write_time_millis_int(_input) + + assert output.getbuffer() == b"\x80\xc3\xc6\x03" + + +def test_write_time_micros_long() -> None: + output = io.BytesIO() + encoder = BinaryEncoder(output) + + _input = datetime.time(1, 2, 3, 456000) + + encoder.write_time_micros_long(_input) + + assert output.getbuffer() == b"\x80\xb8\xfb\xde\x1b" + + +def test_write_timestamp_millis_long() -> None: + output = io.BytesIO() + encoder = BinaryEncoder(output) + + _input = datetime.datetime(2023, 1, 1, 1, 2, 3) + encoder.write_timestamp_millis_long(_input) + + assert output.getbuffer() == b"\xf0\xdb\xcc\xad\xad\x61" + + +def test_write_timestamp_micros_long() -> None: + output = io.BytesIO() + encoder = BinaryEncoder(output) + + _input = datetime.datetime(2023, 1, 1, 1, 2, 3) + encoder.write_timestamp_micros_long(_input) + + assert output.getbuffer() == b"\x80\xe3\xad\x9f\xac\xca\xf8\x05" diff --git a/python/tests/avro/test_file.py b/python/tests/avro/test_file.py index cdb973a39541..101e676c906c 100644 --- a/python/tests/avro/test_file.py +++ b/python/tests/avro/test_file.py @@ -14,10 +14,28 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +import inspect +from enum import Enum +from tempfile import TemporaryDirectory +from typing import Any + import pytest +from fastavro import reader, writer +import pyiceberg.avro.file as avro from pyiceberg.avro.codecs import DeflateCodec from pyiceberg.avro.file import META_SCHEMA, AvroFileHeader +from pyiceberg.io.pyarrow import PyArrowFileIO +from pyiceberg.manifest import ( + MANIFEST_ENTRY_SCHEMA, + DataFile, + DataFileContent, + FileFormat, + ManifestEntry, + ManifestEntryStatus, +) +from pyiceberg.typedef import Record +from pyiceberg.utils.schema_conversion import AvroSchemaConversion def get_deflate_compressor() -> None: @@ -58,3 +76,120 @@ def test_missing_schema() -> None: header.get_schema() assert "No schema found in Avro file headers" in str(exc_info.value) + + +# helper function to serialize our objects to dicts to enable +# direct comparison with the dicts returned by fastavro +def todict(obj: Any) -> Any: + if isinstance(obj, dict): + data = [] + for k, v in obj.items(): + data.append({"key": k, "value": v}) + return data + elif isinstance(obj, Enum): + return obj.value + elif hasattr(obj, "__iter__") and not isinstance(obj, str) and not isinstance(obj, bytes): + return [todict(v) for v in obj] + elif hasattr(obj, "__dict__"): + return {key: todict(value) for key, value in inspect.getmembers(obj) if not callable(value) and not key.startswith("_")} + else: + return obj + + +def test_write_manifest_entry_with_iceberg_read_with_fastavro() -> None: + data_file = DataFile( + content=DataFileContent.DATA, + file_path="s3://some-path/some-file.parquet", + file_format=FileFormat.PARQUET, + partition=Record(), + record_count=131327, + file_size_in_bytes=220669226, + column_sizes={1: 220661854}, + value_counts={1: 131327}, + null_value_counts={1: 0}, + nan_value_counts={}, + lower_bounds={1: b"aaaaaaaaaaaaaaaa"}, + upper_bounds={1: b"zzzzzzzzzzzzzzzz"}, + key_metadata=b"\xde\xad\xbe\xef", + split_offsets=[4, 133697593], + equality_ids=[], + sort_order_id=4, + spec_id=3, + ) + entry = ManifestEntry( + status=ManifestEntryStatus.ADDED, + snapshot_id=8638475580105682862, + data_sequence_number=0, + file_sequence_number=0, + data_file=data_file, + ) + + additional_metadata = {"foo": "bar"} + + with TemporaryDirectory() as tmpdir: + tmp_avro_file = tmpdir + "/manifest_entry.avro" + + with avro.AvroOutputFile[ManifestEntry]( + PyArrowFileIO().new_output(tmp_avro_file), MANIFEST_ENTRY_SCHEMA, "manifest_entry", additional_metadata + ) as out: + out.write_block([entry]) + + with open(tmp_avro_file, "rb") as fo: + r = reader(fo=fo) + + for k, v in additional_metadata.items(): + assert k in r.metadata + assert v == r.metadata[k] + + it = iter(r) + + fa_entry = next(it) + + assert todict(entry) == fa_entry + + +def test_write_manifest_entry_with_fastavro_read_with_iceberg() -> None: + data_file = DataFile( + content=DataFileContent.DATA, + file_path="s3://some-path/some-file.parquet", + file_format=FileFormat.PARQUET, + partition=Record(), + record_count=131327, + file_size_in_bytes=220669226, + column_sizes={1: 220661854}, + value_counts={1: 131327}, + null_value_counts={1: 0}, + nan_value_counts={}, + lower_bounds={1: b"aaaaaaaaaaaaaaaa"}, + upper_bounds={1: b"zzzzzzzzzzzzzzzz"}, + key_metadata=b"\xde\xad\xbe\xef", + split_offsets=[4, 133697593], + equality_ids=[], + sort_order_id=4, + spec_id=3, + ) + entry = ManifestEntry( + status=ManifestEntryStatus.ADDED, + snapshot_id=8638475580105682862, + data_sequence_number=0, + file_sequence_number=0, + data_file=data_file, + ) + + with TemporaryDirectory() as tmpdir: + tmp_avro_file = tmpdir + "/manifest_entry.avro" + + schema = AvroSchemaConversion().iceberg_to_avro(MANIFEST_ENTRY_SCHEMA, schema_name="manifest_entry") + + with open(tmp_avro_file, "wb") as out: + writer(out, schema, [todict(entry)]) + + with avro.AvroFile[ManifestEntry]( + PyArrowFileIO().new_input(tmp_avro_file), + MANIFEST_ENTRY_SCHEMA, + {-1: ManifestEntry, 2: DataFile}, + ) as avro_reader: + it = iter(avro_reader) + avro_entry = next(it) + + assert entry == avro_entry diff --git a/python/tests/avro/test_reader.py b/python/tests/avro/test_reader.py index b8736dd69b4f..416a11abb253 100644 --- a/python/tests/avro/test_reader.py +++ b/python/tests/avro/test_reader.py @@ -15,11 +15,13 @@ # specific language governing permissions and limitations # under the License. # pylint:disable=protected-access +import io import json +from typing import Type import pytest -from pyiceberg.avro.decoder import BinaryDecoder +from pyiceberg.avro.decoder import BinaryDecoder, InMemoryBinaryDecoder, StreamingBinaryDecoder from pyiceberg.avro.file import AvroFile from pyiceberg.avro.reader import ( BinaryReader, @@ -38,7 +40,6 @@ UUIDReader, ) from pyiceberg.avro.resolver import construct_reader -from pyiceberg.io.memory import MemoryInputStream from pyiceberg.io.pyarrow import PyArrowFileIO from pyiceberg.manifest import MANIFEST_ENTRY_SCHEMA, DataFile, ManifestEntry from pyiceberg.schema import Schema @@ -63,6 +64,8 @@ UUIDType, ) +AVAILABLE_DECODERS = [StreamingBinaryDecoder, InMemoryBinaryDecoder] + def test_read_header(generated_manifest_entry_file: str, iceberg_manifest_entry_schema: Schema) -> None: with AvroFile[ManifestEntry]( @@ -335,18 +338,19 @@ def test_uuid_reader() -> None: assert construct_reader(UUIDType()) == UUIDReader() -def test_read_struct() -> None: - mis = MemoryInputStream(b"\x18") - decoder = BinaryDecoder(mis) - +@pytest.mark.parametrize("decoder_class", AVAILABLE_DECODERS) +def test_read_struct(decoder_class: Type[BinaryDecoder]) -> None: + mis = io.BytesIO(b"\x18") + decoder = decoder_class(mis) struct = StructType(NestedField(1, "id", IntegerType(), required=True)) result = StructReader(((0, IntegerReader()),), Record, struct).read(decoder) assert repr(result) == "Record[id=12]" -def test_read_struct_lambda() -> None: - mis = MemoryInputStream(b"\x18") - decoder = BinaryDecoder(mis) +@pytest.mark.parametrize("decoder_class", AVAILABLE_DECODERS) +def test_read_struct_lambda(decoder_class: Type[BinaryDecoder]) -> None: + mis = io.BytesIO(b"\x18") + decoder = decoder_class(mis) struct = StructType(NestedField(1, "id", IntegerType(), required=True)) # You can also pass in an arbitrary function that returns a struct @@ -356,9 +360,10 @@ def test_read_struct_lambda() -> None: assert repr(result) == "Record[id=12]" -def test_read_not_struct_type() -> None: - mis = MemoryInputStream(b"\x18") - decoder = BinaryDecoder(mis) +@pytest.mark.parametrize("decoder_class", AVAILABLE_DECODERS) +def test_read_not_struct_type(decoder_class: Type[BinaryDecoder]) -> None: + mis = io.BytesIO(b"\x18") + decoder = decoder_class(mis) struct = StructType(NestedField(1, "id", IntegerType(), required=True)) with pytest.raises(ValueError) as exc_info: @@ -367,9 +372,10 @@ def test_read_not_struct_type() -> None: assert "Incompatible with StructProtocol: " in str(exc_info.value) -def test_read_struct_exception_handling() -> None: - mis = MemoryInputStream(b"\x18") - decoder = BinaryDecoder(mis) +@pytest.mark.parametrize("decoder_class", AVAILABLE_DECODERS) +def test_read_struct_exception_handling(decoder_class: Type[BinaryDecoder]) -> None: + mis = io.BytesIO(b"\x18") + decoder = decoder_class(mis) def raise_err(struct: StructType) -> None: raise TypeError("boom") diff --git a/python/tests/avro/test_writer.py b/python/tests/avro/test_writer.py new file mode 100644 index 000000000000..c517a0cd1c4d --- /dev/null +++ b/python/tests/avro/test_writer.py @@ -0,0 +1,220 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# pylint:disable=protected-access + +import io +import struct +from typing import Dict, List + +import pytest + +from pyiceberg.avro.encoder import BinaryEncoder +from pyiceberg.avro.resolver import construct_writer +from pyiceberg.avro.writer import ( + BinaryWriter, + BooleanWriter, + DateWriter, + DecimalWriter, + DoubleWriter, + FixedWriter, + FloatWriter, + IntegerWriter, + StringWriter, + TimestamptzWriter, + TimestampWriter, + TimeWriter, + UUIDWriter, +) +from pyiceberg.typedef import Record +from pyiceberg.types import ( + BinaryType, + BooleanType, + DateType, + DecimalType, + DoubleType, + FixedType, + FloatType, + IntegerType, + ListType, + LongType, + MapType, + NestedField, + PrimitiveType, + StringType, + StructType, + TimestampType, + TimestamptzType, + TimeType, + UUIDType, +) + + +def zigzag_encode(datum: int) -> bytes: + result = [] + datum = (datum << 1) ^ (datum >> 63) + while (datum & ~0x7F) != 0: + result.append(struct.pack("B", (datum & 0x7F) | 0x80)) + datum >>= 7 + result.append(struct.pack("B", datum)) + return b"".join(result) + + +def test_fixed_writer() -> None: + assert construct_writer(FixedType(22)) == FixedWriter(22) + + +def test_decimal_writer() -> None: + assert construct_writer(DecimalType(19, 25)) == DecimalWriter(19, 25) + + +def test_boolean_writer() -> None: + assert construct_writer(BooleanType()) == BooleanWriter() + + +def test_integer_writer() -> None: + assert construct_writer(IntegerType()) == IntegerWriter() + + +def test_long_writer() -> None: + assert construct_writer(LongType()) == IntegerWriter() + + +def test_float_writer() -> None: + assert construct_writer(FloatType()) == FloatWriter() + + +def test_double_writer() -> None: + assert construct_writer(DoubleType()) == DoubleWriter() + + +def test_date_writer() -> None: + assert construct_writer(DateType()) == DateWriter() + + +def test_time_writer() -> None: + assert construct_writer(TimeType()) == TimeWriter() + + +def test_timestamp_writer() -> None: + assert construct_writer(TimestampType()) == TimestampWriter() + + +def test_timestamptz_writer() -> None: + assert construct_writer(TimestamptzType()) == TimestamptzWriter() + + +def test_string_writer() -> None: + assert construct_writer(StringType()) == StringWriter() + + +def test_binary_writer() -> None: + assert construct_writer(BinaryType()) == BinaryWriter() + + +def test_unknown_type() -> None: + class UnknownType(PrimitiveType): + __root__ = "UnknownType" + + with pytest.raises(ValueError) as exc_info: + construct_writer(UnknownType()) + + assert "Unknown type:" in str(exc_info.value) + + +def test_uuid_writer() -> None: + assert construct_writer(UUIDType()) == UUIDWriter() + + +def test_write_simple_struct() -> None: + output = io.BytesIO() + encoder = BinaryEncoder(output) + + schema = StructType( + NestedField(1, "id", IntegerType(), required=True), NestedField(2, "property", StringType(), required=True) + ) + + class MyStruct(Record): + id: int + property: str + + my_struct = MyStruct(id=12, property="awesome") + + enc_str = b"awesome" + + construct_writer(schema).write(encoder, my_struct) + + assert output.getbuffer() == b"".join([b"\x18", zigzag_encode(len(enc_str)), enc_str]) + + +def test_write_struct_with_dict() -> None: + output = io.BytesIO() + encoder = BinaryEncoder(output) + + schema = StructType( + NestedField(1, "id", IntegerType(), required=True), + NestedField(2, "properties", MapType(3, IntegerType(), 4, IntegerType()), required=True), + ) + + class MyStruct(Record): + id: int + properties: Dict[int, int] + + my_struct = MyStruct(id=12, properties={1: 2, 3: 4}) + + construct_writer(schema).write(encoder, my_struct) + + assert output.getbuffer() == b"".join( + [ + b"\x18", + zigzag_encode(len(my_struct.properties)), + zigzag_encode(1), + zigzag_encode(2), + zigzag_encode(3), + zigzag_encode(4), + b"\x00", + ] + ) + + +def test_write_struct_with_list() -> None: + output = io.BytesIO() + encoder = BinaryEncoder(output) + + schema = StructType( + NestedField(1, "id", IntegerType(), required=True), + NestedField(2, "properties", ListType(3, IntegerType()), required=True), + ) + + class MyStruct(Record): + id: int + properties: List[int] + + my_struct = MyStruct(id=12, properties=[1, 2, 3, 4]) + + construct_writer(schema).write(encoder, my_struct) + + assert output.getbuffer() == b"".join( + [ + b"\x18", + zigzag_encode(len(my_struct.properties)), + zigzag_encode(1), + zigzag_encode(2), + zigzag_encode(3), + zigzag_encode(4), + b"\x00", + ] + ) diff --git a/python/tests/catalog/integration_test_dynamodb.py b/python/tests/catalog/integration_test_dynamodb.py index a430175c5a9c..36035a245c81 100644 --- a/python/tests/catalog/integration_test_dynamodb.py +++ b/python/tests/catalog/integration_test_dynamodb.py @@ -44,7 +44,7 @@ def fixture_dynamodb_client() -> boto3.client: @pytest.fixture(name="test_catalog", scope="module") def fixture_test_catalog() -> Generator[Catalog, None, None]: - """The pre- and post-setting of aws integration test""" + """The pre- and post-setting of aws integration test.""" test_catalog = DynamoDbCatalog("test_ddb_catalog", warehouse=get_s3_path(get_bucket_name())) yield test_catalog clean_up(test_catalog) diff --git a/python/tests/catalog/integration_test_glue.py b/python/tests/catalog/integration_test_glue.py index 62870390b3ee..bd025f2a3d20 100644 --- a/python/tests/catalog/integration_test_glue.py +++ b/python/tests/catalog/integration_test_glue.py @@ -35,6 +35,7 @@ # The number of tables/databases used in list_table/namespace test LIST_TEST_NUMBER = 2 +CATALOG_NAME = "glue" @pytest.fixture(name="glue", scope="module") @@ -44,8 +45,8 @@ def fixture_glue_client() -> boto3.client: @pytest.fixture(name="test_catalog", scope="module") def fixture_test_catalog() -> Generator[Catalog, None, None]: - """The pre- and post-setting of aws integration test""" - test_catalog = GlueCatalog("glue", warehouse=get_s3_path(get_bucket_name())) + """The pre- and post-setting of aws integration test.""" + test_catalog = GlueCatalog(CATALOG_NAME, warehouse=get_s3_path(get_bucket_name())) yield test_catalog clean_up(test_catalog) @@ -57,7 +58,7 @@ def test_create_table( test_catalog.create_namespace(database_name) test_catalog.create_table(identifier, table_schema_nested, get_s3_path(get_bucket_name(), database_name, table_name)) table = test_catalog.load_table(identifier) - assert table.identifier == identifier + assert table.identifier == (CATALOG_NAME,) + identifier metadata_location = table.metadata_location.split(get_bucket_name())[1][1:] s3.head_object(Bucket=get_bucket_name(), Key=metadata_location) @@ -78,7 +79,7 @@ def test_create_table_with_default_location( test_catalog.create_namespace(database_name) test_catalog.create_table(identifier, table_schema_nested) table = test_catalog.load_table(identifier) - assert table.identifier == identifier + assert table.identifier == (CATALOG_NAME,) + identifier metadata_location = table.metadata_location.split(get_bucket_name())[1][1:] s3.head_object(Bucket=get_bucket_name(), Key=metadata_location) @@ -125,11 +126,11 @@ def test_rename_table( new_table_name = f"rename-{table_name}" identifier = (database_name, table_name) table = test_catalog.create_table(identifier, table_schema_nested) - assert table.identifier == identifier + assert table.identifier == (CATALOG_NAME,) + identifier new_identifier = (new_database_name, new_table_name) test_catalog.rename_table(identifier, new_identifier) new_table = test_catalog.load_table(new_identifier) - assert new_table.identifier == new_identifier + assert new_table.identifier == (CATALOG_NAME,) + new_identifier assert new_table.metadata_location == table.metadata_location metadata_location = new_table.metadata_location.split(get_bucket_name())[1][1:] s3.head_object(Bucket=get_bucket_name(), Key=metadata_location) @@ -141,7 +142,7 @@ def test_drop_table(test_catalog: Catalog, table_schema_nested: Schema, table_na identifier = (database_name, table_name) test_catalog.create_namespace(database_name) table = test_catalog.create_table(identifier, table_schema_nested) - assert table.identifier == identifier + assert table.identifier == (CATALOG_NAME,) + identifier test_catalog.drop_table(identifier) with pytest.raises(NoSuchTableError): test_catalog.load_table(identifier) @@ -154,7 +155,7 @@ def test_purge_table( test_catalog.create_namespace(database_name) test_catalog.create_table(identifier, table_schema_nested) table = test_catalog.load_table(identifier) - assert table.identifier == identifier + assert table.identifier == (CATALOG_NAME,) + identifier metadata_location = table.metadata_location.split(get_bucket_name())[1][1:] s3.head_object(Bucket=get_bucket_name(), Key=metadata_location) test_catalog.purge_table(identifier) diff --git a/python/tests/catalog/test_base.py b/python/tests/catalog/test_base.py index 742549a6543c..b47aa5f5f799 100644 --- a/python/tests/catalog/test_base.py +++ b/python/tests/catalog/test_base.py @@ -42,7 +42,7 @@ from pyiceberg.io import load_file_io from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionField, PartitionSpec from pyiceberg.schema import Schema -from pyiceberg.table import Table +from pyiceberg.table import CommitTableRequest, CommitTableResponse, Table from pyiceberg.table.metadata import TableMetadataV1 from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder from pyiceberg.transforms import IdentityTransform @@ -96,17 +96,21 @@ def create_table( ], }, "partition-spec": [{"name": "x", "transform": "identity", "source-id": 1, "field-id": 1000}], - "properties": {}, + "properties": properties, "current-snapshot-id": -1, "snapshots": [{"snapshot-id": 1925, "timestamp-ms": 1602638573822}], } ), metadata_location=f's3://warehouse/{"/".join(identifier)}/metadata/metadata.json', io=load_file_io(), + catalog=self, ) self.__tables[identifier] = table return table + def _commit_table(self, table_request: CommitTableRequest) -> CommitTableResponse: + raise NotImplementedError + def load_table(self, identifier: Union[str, Identifier]) -> Table: identifier = Catalog.identifier_to_tuple(identifier) try: @@ -141,6 +145,7 @@ def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: U metadata=table.metadata, metadata_location=table.metadata_location, io=load_file_io(), + catalog=self, ) return self.__tables[to_identifier] @@ -170,6 +175,10 @@ def list_tables(self, namespace: Optional[Union[str, Identifier]] = None) -> Lis return list_tables def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identifier]: + # Hierarchical namespace is not supported. Return an empty list + if namespace: + return [] + return list(self.__namespaces.keys()) def load_namespace_properties(self, namespace: Union[str, Identifier]) -> Properties: diff --git a/python/tests/catalog/test_glue.py b/python/tests/catalog/test_glue.py index 0fef1fad5fdd..d21741669768 100644 --- a/python/tests/catalog/test_glue.py +++ b/python/tests/catalog/test_glue.py @@ -15,6 +15,7 @@ # specific language governing permissions and limitations # under the License. from typing import List +from unittest import mock import pytest from moto import mock_glue @@ -37,11 +38,12 @@ def test_create_table_with_database_location( _bucket_initialize: None, _patch_aiobotocore: None, table_schema_nested: Schema, database_name: str, table_name: str ) -> None: + catalog_name = "glue" identifier = (database_name, table_name) - test_catalog = GlueCatalog("glue", **{"py-io-impl": "pyiceberg.io.fsspec.FsspecFileIO"}) + test_catalog = GlueCatalog(catalog_name, **{"py-io-impl": "pyiceberg.io.fsspec.FsspecFileIO"}) test_catalog.create_namespace(namespace=database_name, properties={"location": f"s3://{BUCKET_NAME}/{database_name}.db"}) table = test_catalog.create_table(identifier, table_schema_nested) - assert table.identifier == identifier + assert table.identifier == (catalog_name,) + identifier assert TABLE_METADATA_LOCATION_REGEX.match(table.metadata_location) @@ -49,11 +51,14 @@ def test_create_table_with_database_location( def test_create_table_with_default_warehouse( _bucket_initialize: None, _patch_aiobotocore: None, table_schema_nested: Schema, database_name: str, table_name: str ) -> None: + catalog_name = "glue" identifier = (database_name, table_name) - test_catalog = GlueCatalog("glue", **{"py-io-impl": "pyiceberg.io.fsspec.FsspecFileIO", "warehouse": f"s3://{BUCKET_NAME}"}) + test_catalog = GlueCatalog( + catalog_name, **{"py-io-impl": "pyiceberg.io.fsspec.FsspecFileIO", "warehouse": f"s3://{BUCKET_NAME}"} + ) test_catalog.create_namespace(namespace=database_name) table = test_catalog.create_table(identifier, table_schema_nested) - assert table.identifier == identifier + assert table.identifier == (catalog_name,) + identifier assert TABLE_METADATA_LOCATION_REGEX.match(table.metadata_location) @@ -61,13 +66,14 @@ def test_create_table_with_default_warehouse( def test_create_table_with_given_location( _bucket_initialize: None, _patch_aiobotocore: None, table_schema_nested: Schema, database_name: str, table_name: str ) -> None: + catalog_name = "glue" identifier = (database_name, table_name) - test_catalog = GlueCatalog("glue", **{"py-io-impl": "pyiceberg.io.fsspec.FsspecFileIO"}) + test_catalog = GlueCatalog(catalog_name, **{"py-io-impl": "pyiceberg.io.fsspec.FsspecFileIO"}) test_catalog.create_namespace(namespace=database_name) table = test_catalog.create_table( identifier=identifier, schema=table_schema_nested, location=f"s3://{BUCKET_NAME}/{database_name}.db/{table_name}" ) - assert table.identifier == identifier + assert table.identifier == (catalog_name,) + identifier assert TABLE_METADATA_LOCATION_REGEX.match(table.metadata_location) @@ -75,8 +81,9 @@ def test_create_table_with_given_location( def test_create_table_with_no_location( _bucket_initialize: None, _patch_aiobotocore: None, table_schema_nested: Schema, database_name: str, table_name: str ) -> None: + catalog_name = "glue" identifier = (database_name, table_name) - test_catalog = GlueCatalog("glue", **{"py-io-impl": "pyiceberg.io.fsspec.FsspecFileIO"}) + test_catalog = GlueCatalog(catalog_name, **{"py-io-impl": "pyiceberg.io.fsspec.FsspecFileIO"}) test_catalog.create_namespace(namespace=database_name) with pytest.raises(ValueError): test_catalog.create_table(identifier=identifier, schema=table_schema_nested) @@ -86,11 +93,12 @@ def test_create_table_with_no_location( def test_create_table_with_strips( _bucket_initialize: None, _patch_aiobotocore: None, table_schema_nested: Schema, database_name: str, table_name: str ) -> None: + catalog_name = "glue" identifier = (database_name, table_name) - test_catalog = GlueCatalog("glue", **{"py-io-impl": "pyiceberg.io.fsspec.FsspecFileIO"}) + test_catalog = GlueCatalog(catalog_name, **{"py-io-impl": "pyiceberg.io.fsspec.FsspecFileIO"}) test_catalog.create_namespace(namespace=database_name, properties={"location": f"s3://{BUCKET_NAME}/{database_name}.db/"}) table = test_catalog.create_table(identifier, table_schema_nested) - assert table.identifier == identifier + assert table.identifier == (catalog_name,) + identifier assert TABLE_METADATA_LOCATION_REGEX.match(table.metadata_location) @@ -98,11 +106,12 @@ def test_create_table_with_strips( def test_create_table_with_strips_bucket_root( _bucket_initialize: None, _patch_aiobotocore: None, table_schema_nested: Schema, database_name: str, table_name: str ) -> None: + catalog_name = "glue" identifier = (database_name, table_name) test_catalog = GlueCatalog("glue", **{"py-io-impl": "pyiceberg.io.fsspec.FsspecFileIO", "warehouse": f"s3://{BUCKET_NAME}/"}) test_catalog.create_namespace(namespace=database_name) table_strip = test_catalog.create_table(identifier, table_schema_nested) - assert table_strip.identifier == identifier + assert table_strip.identifier == (catalog_name,) + identifier assert TABLE_METADATA_LOCATION_REGEX.match(table_strip.metadata_location) @@ -132,12 +141,15 @@ def test_create_duplicated_table( def test_load_table( _bucket_initialize: None, _patch_aiobotocore: None, table_schema_nested: Schema, database_name: str, table_name: str ) -> None: + catalog_name = "glue" identifier = (database_name, table_name) - test_catalog = GlueCatalog("glue", **{"py-io-impl": "pyiceberg.io.fsspec.FsspecFileIO", "warehouse": f"s3://{BUCKET_NAME}/"}) + test_catalog = GlueCatalog( + catalog_name, **{"py-io-impl": "pyiceberg.io.fsspec.FsspecFileIO", "warehouse": f"s3://{BUCKET_NAME}/"} + ) test_catalog.create_namespace(namespace=database_name) test_catalog.create_table(identifier, table_schema_nested) table = test_catalog.load_table(identifier) - assert table.identifier == identifier + assert table.identifier == (catalog_name,) + identifier assert TABLE_METADATA_LOCATION_REGEX.match(table.metadata_location) @@ -154,12 +166,15 @@ def test_load_non_exist_table(_bucket_initialize: None, _patch_aiobotocore: None def test_drop_table( _bucket_initialize: None, _patch_aiobotocore: None, table_schema_nested: Schema, database_name: str, table_name: str ) -> None: + catalog_name = "glue" identifier = (database_name, table_name) - test_catalog = GlueCatalog("glue", **{"py-io-impl": "pyiceberg.io.fsspec.FsspecFileIO", "warehouse": f"s3://{BUCKET_NAME}/"}) + test_catalog = GlueCatalog( + catalog_name, **{"py-io-impl": "pyiceberg.io.fsspec.FsspecFileIO", "warehouse": f"s3://{BUCKET_NAME}/"} + ) test_catalog.create_namespace(namespace=database_name) test_catalog.create_table(identifier, table_schema_nested) table = test_catalog.load_table(identifier) - assert table.identifier == identifier + assert table.identifier == (catalog_name,) + identifier assert TABLE_METADATA_LOCATION_REGEX.match(table.metadata_location) test_catalog.drop_table(identifier) with pytest.raises(NoSuchTableError): @@ -178,17 +193,18 @@ def test_drop_non_exist_table(_bucket_initialize: None, _patch_aiobotocore: None def test_rename_table( _bucket_initialize: None, _patch_aiobotocore: None, table_schema_nested: Schema, database_name: str, table_name: str ) -> None: + catalog_name = "glue" new_table_name = f"{table_name}_new" identifier = (database_name, table_name) new_identifier = (database_name, new_table_name) test_catalog = GlueCatalog("glue", **{"py-io-impl": "pyiceberg.io.fsspec.FsspecFileIO", "warehouse": f"s3://{BUCKET_NAME}/"}) test_catalog.create_namespace(namespace=database_name) table = test_catalog.create_table(identifier, table_schema_nested) - assert table.identifier == identifier + assert table.identifier == (catalog_name,) + identifier assert TABLE_METADATA_LOCATION_REGEX.match(table.metadata_location) test_catalog.rename_table(identifier, new_identifier) new_table = test_catalog.load_table(new_identifier) - assert new_table.identifier == new_identifier + assert new_table.identifier == (catalog_name,) + new_identifier # the metadata_location should not change assert new_table.metadata_location == table.metadata_location # old table should be dropped @@ -421,3 +437,22 @@ def test_update_namespace_properties_overlap_update_removal( test_catalog.update_namespace_properties(database_name, removals, updates) # should not modify the properties assert test_catalog.load_namespace_properties(database_name) == test_properties + + +@mock_glue +def test_passing_profile_name() -> None: + session_properties = { + "aws_secret_key_id": "abc", + "aws_secret_access_key": "def", + "aws_session_token": "ghi", + "region_name": "eu-central-1", + "profile_name": "sandbox", + } + test_properties = {"type": "glue"} + test_properties.update(session_properties) + + with mock.patch("boto3.Session") as mock_session: + test_catalog = GlueCatalog("glue", **test_properties) + + mock_session.assert_called_with(**session_properties) + assert test_catalog.glue is mock_session().client() diff --git a/python/tests/catalog/test_hive.py b/python/tests/catalog/test_hive.py index fee3a4f731aa..f353f795df3b 100644 --- a/python/tests/catalog/test_hive.py +++ b/python/tests/catalog/test_hive.py @@ -351,8 +351,8 @@ def test_load_table(hive_table: HiveTable) -> None: ), ], snapshot_log=[ - SnapshotLogEntry(snapshot_id="3051729675574597004", timestamp_ms=1515100955770), - SnapshotLogEntry(snapshot_id="3055729675574597004", timestamp_ms=1555100955770), + SnapshotLogEntry(snapshot_id=3051729675574597004, timestamp_ms=1515100955770), + SnapshotLogEntry(snapshot_id=3055729675574597004, timestamp_ms=1555100955770), ], metadata_log=[MetadataLogEntry(metadata_file="s3://bucket/.../v1.json", timestamp_ms=1515100)], sort_orders=[ @@ -390,7 +390,7 @@ def test_load_table(hive_table: HiveTable) -> None: last_sequence_number=34, ) - assert table.identifier == ("default", "new_tabl2e") + assert table.identifier == (HIVE_CATALOG_NAME, "default", "new_tabl2e") assert expected == table.metadata diff --git a/python/tests/catalog/test_rest.py b/python/tests/catalog/test_rest.py index 6f3cafffc175..a7663ac5117f 100644 --- a/python/tests/catalog/test_rest.py +++ b/python/tests/catalog/test_rest.py @@ -408,7 +408,8 @@ def test_load_table_200(rest_mock: Mocker) -> None: status_code=200, request_headers=TEST_HEADERS, ) - actual = RestCatalog("rest", uri=TEST_URI, token=TEST_TOKEN).load_table(("fokko", "table")) + catalog = RestCatalog("rest", uri=TEST_URI, token=TEST_TOKEN) + actual = catalog.load_table(("fokko", "table")) expected = Table( identifier=("rest", "fokko", "table"), metadata_location="s3://warehouse/database/table/metadata/00001-5f2f8166-244c-4eae-ac36-384ecdec81fc.gz.metadata.json", @@ -484,6 +485,7 @@ def test_load_table_200(rest_mock: Mocker) -> None: partition_spec=[], ), io=load_file_io(), + catalog=catalog, ) assert actual == expected @@ -585,7 +587,8 @@ def test_create_table_200(rest_mock: Mocker, table_schema_simple: Schema) -> Non status_code=200, request_headers=TEST_HEADERS, ) - table = RestCatalog("rest", uri=TEST_URI, token=TEST_TOKEN).create_table( + catalog = RestCatalog("rest", uri=TEST_URI, token=TEST_TOKEN) + table = catalog.create_table( identifier=("fokko", "fokko2"), schema=table_schema_simple, location=None, @@ -639,6 +642,7 @@ def test_create_table_200(rest_mock: Mocker, table_schema_simple: Schema) -> Non partition_spec=[], ), io=load_file_io(), + catalog=catalog, ) diff --git a/python/tests/catalog/test_sql.py b/python/tests/catalog/test_sql.py new file mode 100644 index 000000000000..2b903093a491 --- /dev/null +++ b/python/tests/catalog/test_sql.py @@ -0,0 +1,355 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import os +from pathlib import Path +from typing import Generator, List + +import pytest +from pytest import TempPathFactory +from sqlalchemy.exc import ArgumentError, IntegrityError + +from pyiceberg.catalog import Identifier +from pyiceberg.catalog.sql import SqlCatalog +from pyiceberg.exceptions import ( + NamespaceAlreadyExistsError, + NamespaceNotEmptyError, + NoSuchNamespaceError, + NoSuchPropertyException, + NoSuchTableError, + TableAlreadyExistsError, +) +from pyiceberg.schema import Schema +from pyiceberg.table.sorting import ( + NullOrder, + SortDirection, + SortField, + SortOrder, +) +from pyiceberg.transforms import IdentityTransform + + +@pytest.fixture(name="warehouse", scope="session") +def fixture_warehouse(tmp_path_factory: TempPathFactory) -> Path: + return tmp_path_factory.mktemp("test_sql") + + +@pytest.fixture(name="random_identifier") +def fixture_random_identifier(warehouse: Path, database_name: str, table_name: str) -> Identifier: + os.makedirs(f"{warehouse}/{database_name}.db/{table_name}/metadata/", exist_ok=True) + return database_name, table_name + + +@pytest.fixture(name="another_random_identifier") +def fixture_another_random_identifier(warehouse: Path, database_name: str, table_name: str) -> Identifier: + database_name = database_name + "_new" + table_name = table_name + "_new" + os.makedirs(f"{warehouse}/{database_name}.db/{table_name}/metadata/", exist_ok=True) + return database_name, table_name + + +@pytest.fixture(name="test_catalog", scope="module") +def fixture_test_catalog(warehouse: Path) -> Generator[SqlCatalog, None, None]: + props = { + "uri": "sqlite+pysqlite:///:memory:", + "warehouse": f"file://{warehouse}", + } + test_catalog = SqlCatalog("test_sql_catalog", **props) + test_catalog.create_tables() + yield test_catalog + test_catalog.destroy_tables() + + +def test_creation_with_no_uri() -> None: + with pytest.raises(NoSuchPropertyException): + SqlCatalog("test_ddb_catalog", not_uri="unused") + + +def test_creation_with_unsupported_uri() -> None: + with pytest.raises(ArgumentError): + SqlCatalog("test_ddb_catalog", uri="unsupported:xxx") + + +def test_create_tables_idempotency(test_catalog: SqlCatalog) -> None: + # Second initialization should not fail even if tables are already created + test_catalog.create_tables() + test_catalog.create_tables() + + +def test_create_table_default_sort_order( + test_catalog: SqlCatalog, table_schema_nested: Schema, random_identifier: Identifier +) -> None: + database_name, _table_name = random_identifier + test_catalog.create_namespace(database_name) + table = test_catalog.create_table(random_identifier, table_schema_nested) + assert table.sort_order().order_id == 0, "Order ID must match" + assert table.sort_order().is_unsorted is True, "Order must be unsorted" + test_catalog.drop_table(random_identifier) + + +def test_create_table_custom_sort_order( + test_catalog: SqlCatalog, table_schema_nested: Schema, random_identifier: Identifier +) -> None: + database_name, _table_name = random_identifier + test_catalog.create_namespace(database_name) + order = SortOrder(SortField(source_id=2, transform=IdentityTransform(), null_order=NullOrder.NULLS_FIRST)) + table = test_catalog.create_table(random_identifier, table_schema_nested, sort_order=order) + given_sort_order = table.sort_order() + assert given_sort_order.order_id == 1, "Order ID must match" + assert len(given_sort_order.fields) == 1, "Order must have 1 field" + assert given_sort_order.fields[0].direction == SortDirection.ASC, "Direction must match" + assert given_sort_order.fields[0].null_order == NullOrder.NULLS_FIRST, "Null order must match" + assert isinstance(given_sort_order.fields[0].transform, IdentityTransform), "Transform must match" + test_catalog.drop_table(random_identifier) + + +def test_create_table_with_default_warehouse_location( + warehouse: Path, test_catalog: SqlCatalog, table_schema_nested: Schema, random_identifier: Identifier +) -> None: + database_name, _table_name = random_identifier + test_catalog.create_namespace(database_name) + test_catalog.create_table(random_identifier, table_schema_nested) + table = test_catalog.load_table(random_identifier) + assert table.identifier == (test_catalog.name,) + random_identifier + assert table.metadata_location.startswith(f"file://{warehouse}") + assert os.path.exists(table.metadata_location[len("file://") :]) + test_catalog.drop_table(random_identifier) + + +def test_create_duplicated_table(test_catalog: SqlCatalog, table_schema_nested: Schema, random_identifier: Identifier) -> None: + database_name, _table_name = random_identifier + test_catalog.create_namespace(database_name) + test_catalog.create_table(random_identifier, table_schema_nested) + with pytest.raises(TableAlreadyExistsError): + test_catalog.create_table(random_identifier, table_schema_nested) + + +def test_create_table_with_non_existing_namespace(test_catalog: SqlCatalog, table_schema_nested: Schema, table_name: str) -> None: + identifier = ("invalid", table_name) + with pytest.raises(NoSuchNamespaceError): + test_catalog.create_table(identifier, table_schema_nested) + + +def test_create_table_without_namespace(test_catalog: SqlCatalog, table_schema_nested: Schema, table_name: str) -> None: + with pytest.raises(ValueError): + test_catalog.create_table(table_name, table_schema_nested) + + +def test_load_table(test_catalog: SqlCatalog, table_schema_nested: Schema, random_identifier: Identifier) -> None: + database_name, _table_name = random_identifier + test_catalog.create_namespace(database_name) + table = test_catalog.create_table(random_identifier, table_schema_nested) + loaded_table = test_catalog.load_table(random_identifier) + assert table.identifier == loaded_table.identifier + assert table.metadata_location == loaded_table.metadata_location + assert table.metadata == loaded_table.metadata + + +def test_drop_table(test_catalog: SqlCatalog, table_schema_nested: Schema, random_identifier: Identifier) -> None: + database_name, _table_name = random_identifier + test_catalog.create_namespace(database_name) + table = test_catalog.create_table(random_identifier, table_schema_nested) + assert table.identifier == (test_catalog.name,) + random_identifier + test_catalog.drop_table(random_identifier) + with pytest.raises(NoSuchTableError): + test_catalog.load_table(random_identifier) + + +def test_drop_table_that_does_not_exist(test_catalog: SqlCatalog, random_identifier: Identifier) -> None: + with pytest.raises(NoSuchTableError): + test_catalog.drop_table(random_identifier) + + +def test_rename_table( + test_catalog: SqlCatalog, table_schema_nested: Schema, random_identifier: Identifier, another_random_identifier: Identifier +) -> None: + from_database_name, _from_table_name = random_identifier + to_database_name, _to_table_name = another_random_identifier + test_catalog.create_namespace(from_database_name) + test_catalog.create_namespace(to_database_name) + table = test_catalog.create_table(random_identifier, table_schema_nested) + assert table.identifier == (test_catalog.name,) + random_identifier + test_catalog.rename_table(random_identifier, another_random_identifier) + new_table = test_catalog.load_table(another_random_identifier) + assert new_table.identifier == (test_catalog.name,) + another_random_identifier + assert new_table.metadata_location == table.metadata_location + with pytest.raises(NoSuchTableError): + test_catalog.load_table(random_identifier) + + +def test_rename_table_to_existing_one( + test_catalog: SqlCatalog, table_schema_nested: Schema, random_identifier: Identifier, another_random_identifier: Identifier +) -> None: + from_database_name, _from_table_name = random_identifier + to_database_name, _to_table_name = another_random_identifier + test_catalog.create_namespace(from_database_name) + test_catalog.create_namespace(to_database_name) + table = test_catalog.create_table(random_identifier, table_schema_nested) + assert table.identifier == (test_catalog.name,) + random_identifier + new_table = test_catalog.create_table(another_random_identifier, table_schema_nested) + assert new_table.identifier == (test_catalog.name,) + another_random_identifier + with pytest.raises(TableAlreadyExistsError): + test_catalog.rename_table(random_identifier, another_random_identifier) + + +def test_rename_missing_table( + test_catalog: SqlCatalog, random_identifier: Identifier, another_random_identifier: Identifier +) -> None: + to_database_name, _to_table_name = another_random_identifier + test_catalog.create_namespace(to_database_name) + with pytest.raises(NoSuchTableError): + test_catalog.rename_table(random_identifier, another_random_identifier) + + +def test_rename_table_to_missing_namespace( + test_catalog: SqlCatalog, table_schema_nested: Schema, random_identifier: Identifier, another_random_identifier: Identifier +) -> None: + from_database_name, _from_table_name = random_identifier + test_catalog.create_namespace(from_database_name) + table = test_catalog.create_table(random_identifier, table_schema_nested) + assert table.identifier == (test_catalog.name,) + random_identifier + with pytest.raises(NoSuchNamespaceError): + test_catalog.rename_table(random_identifier, another_random_identifier) + + +def test_list_tables( + test_catalog: SqlCatalog, table_schema_nested: Schema, random_identifier: Identifier, another_random_identifier: Identifier +) -> None: + database_name_1, _table_name_1 = random_identifier + database_name_2, _table_name_2 = another_random_identifier + test_catalog.create_namespace(database_name_1) + test_catalog.create_namespace(database_name_2) + test_catalog.create_table(random_identifier, table_schema_nested) + test_catalog.create_table(another_random_identifier, table_schema_nested) + identifier_list = test_catalog.list_tables(database_name_1) + assert len(identifier_list) == 1 + assert random_identifier in identifier_list + + identifier_list = test_catalog.list_tables(database_name_2) + assert len(identifier_list) == 1 + assert another_random_identifier in identifier_list + + +def test_create_namespace(test_catalog: SqlCatalog, database_name: str) -> None: + test_catalog.create_namespace(database_name) + assert (database_name,) in test_catalog.list_namespaces() + + +def test_create_duplicate_namespace(test_catalog: SqlCatalog, database_name: str) -> None: + test_catalog.create_namespace(database_name) + with pytest.raises(NamespaceAlreadyExistsError): + test_catalog.create_namespace(database_name) + + +def test_create_namespaces_sharing_same_prefix(test_catalog: SqlCatalog, database_name: str) -> None: + test_catalog.create_namespace(database_name + "_1") + # Second namespace is a prefix of the first one, make sure it can be added. + test_catalog.create_namespace(database_name) + + +def test_create_namespace_with_comment_and_location(test_catalog: SqlCatalog, database_name: str) -> None: + test_location = "/test/location" + test_properties = { + "comment": "this is a test description", + "location": test_location, + } + test_catalog.create_namespace(namespace=database_name, properties=test_properties) + loaded_database_list = test_catalog.list_namespaces() + assert (database_name,) in loaded_database_list + properties = test_catalog.load_namespace_properties(database_name) + assert properties["comment"] == "this is a test description" + assert properties["location"] == test_location + + +def test_create_namespace_with_null_properties(test_catalog: SqlCatalog, database_name: str) -> None: + with pytest.raises(IntegrityError): + test_catalog.create_namespace(namespace=database_name, properties={None: "value"}) # type: ignore + + with pytest.raises(IntegrityError): + test_catalog.create_namespace(namespace=database_name, properties={"key": None}) # type: ignore + + +def test_list_namespaces(test_catalog: SqlCatalog, database_list: List[str]) -> None: + for database_name in database_list: + test_catalog.create_namespace(database_name) + db_list = test_catalog.list_namespaces() + for database_name in database_list: + assert (database_name,) in db_list + assert len(test_catalog.list_namespaces(database_name)) == 1 + + +def test_list_non_existing_namespaces(test_catalog: SqlCatalog) -> None: + with pytest.raises(NoSuchNamespaceError): + test_catalog.list_namespaces("does_not_exist") + + +def test_drop_namespace(test_catalog: SqlCatalog, table_schema_nested: Schema, random_identifier: Identifier) -> None: + database_name, table_name = random_identifier + test_catalog.create_namespace(database_name) + assert (database_name,) in test_catalog.list_namespaces() + test_catalog.create_table((database_name, table_name), table_schema_nested) + with pytest.raises(NamespaceNotEmptyError): + test_catalog.drop_namespace(database_name) + test_catalog.drop_table((database_name, table_name)) + test_catalog.drop_namespace(database_name) + assert (database_name,) not in test_catalog.list_namespaces() + + +def test_load_namespace_properties(test_catalog: SqlCatalog, database_name: str) -> None: + warehouse_location = "/test/location" + test_properties = { + "comment": "this is a test description", + "location": f"{warehouse_location}/{database_name}.db", + "test_property1": "1", + "test_property2": "2", + "test_property3": "3", + } + + test_catalog.create_namespace(database_name, test_properties) + listed_properties = test_catalog.load_namespace_properties(database_name) + for k, v in listed_properties.items(): + assert k in test_properties + assert v == test_properties[k] + + +def test_load_empty_namespace_properties(test_catalog: SqlCatalog, database_name: str) -> None: + test_catalog.create_namespace(database_name) + listed_properties = test_catalog.load_namespace_properties(database_name) + assert listed_properties == {"exists": "true"} + + +def test_update_namespace_properties(test_catalog: SqlCatalog, database_name: str) -> None: + warehouse_location = "/test/location" + test_properties = { + "comment": "this is a test description", + "location": f"{warehouse_location}/{database_name}.db", + "test_property1": "1", + "test_property2": "2", + "test_property3": "3", + } + removals = {"test_property1", "test_property2", "test_property3", "should_not_removed"} + updates = {"test_property4": "4", "test_property5": "5", "comment": "updated test description"} + test_catalog.create_namespace(database_name, test_properties) + update_report = test_catalog.update_namespace_properties(database_name, removals, updates) + for k in updates.keys(): + assert k in update_report.updated + for k in removals: + if k == "should_not_removed": + assert k in update_report.missing + else: + assert k in update_report.removed + assert "updated test description" == test_catalog.load_namespace_properties(database_name)["comment"] diff --git a/python/tests/cli/test_console.py b/python/tests/cli/test_console.py index 555071dd0dc2..31c062a22803 100644 --- a/python/tests/cli/test_console.py +++ b/python/tests/cli/test_console.py @@ -15,286 +15,159 @@ # specific language governing permissions and limitations # under the License. import os -from typing import ( - List, - Optional, - Set, - Union, -) -from unittest import mock +import pytest from click.testing import CliRunner +from pytest_mock import MockFixture -from pyiceberg.catalog import Catalog, PropertiesUpdateSummary from pyiceberg.cli.console import run -from pyiceberg.exceptions import NoSuchNamespaceError, NoSuchTableError -from pyiceberg.io import load_file_io -from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec +from pyiceberg.partitioning import PartitionField, PartitionSpec from pyiceberg.schema import Schema -from pyiceberg.table import Table -from pyiceberg.table.metadata import TableMetadataV2 -from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder -from pyiceberg.typedef import EMPTY_DICT, Identifier, Properties +from pyiceberg.transforms import IdentityTransform +from pyiceberg.typedef import Properties from pyiceberg.utils.config import Config - -EXAMPLE_TABLE_METADATA_V2 = { - "format-version": 2, - "table-uuid": "9c12d441-03fe-4693-9a96-a0705ddf69c1", - "location": "s3://bucket/test/location", - "last-sequence-number": 34, - "last-updated-ms": 1602638573590, - "last-column-id": 3, - "current-schema-id": 1, - "schemas": [ - {"type": "struct", "schema-id": 0, "fields": [{"id": 1, "name": "x", "required": True, "type": "long"}]}, - { - "type": "struct", - "schema-id": 1, - "identifier-field-ids": [1, 2], - "fields": [ - {"id": 1, "name": "x", "required": True, "type": "long"}, - {"id": 2, "name": "y", "required": True, "type": "long", "doc": "comment"}, - {"id": 3, "name": "z", "required": True, "type": "long"}, - ], - }, - ], - "default-spec-id": 0, - "partition-specs": [{"spec-id": 0, "fields": [{"name": "x", "transform": "identity", "source-id": 1, "field-id": 1000}]}], - "last-partition-id": 1000, - "default-sort-order-id": 3, - "sort-orders": [ - { - "order-id": 3, - "fields": [ - {"transform": "identity", "source-id": 2, "direction": "asc", "null-order": "nulls-first"}, - {"transform": "bucket[4]", "source-id": 3, "direction": "desc", "null-order": "nulls-last"}, - ], - } - ], - "properties": {"read.split.target.size": 134217728}, - "current-snapshot-id": 3055729675574597004, - "snapshots": [ - { - "snapshot-id": 3051729675574597004, - "timestamp-ms": 1515100955770, - "sequence-number": 0, - "summary": {"operation": "append"}, - "manifest-list": "s3://a/b/1.avro", - }, - { - "snapshot-id": 3055729675574597004, - "parent-snapshot-id": 3051729675574597004, - "timestamp-ms": 1555100955770, - "sequence-number": 1, - "summary": {"operation": "append"}, - "manifest-list": "s3://a/b/2.avro", - "schema-id": 1, - }, - ], - "snapshot-log": [ - {"snapshot-id": 3051729675574597004, "timestamp-ms": 1515100955770}, - {"snapshot-id": 3055729675574597004, "timestamp-ms": 1555100955770}, - ], - "metadata-log": [{"metadata-file": "s3://bucket/.../v1.json", "timestamp-ms": 1515100}], - "refs": {"test": {"snapshot-id": 3051729675574597004, "type": "tag", "max-ref-age-ms": 10000000}}, -} - - -class MockCatalog(Catalog): - def create_table( - self, - identifier: Union[str, Identifier], - schema: Schema, - location: Optional[str] = None, - partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC, - sort_order: SortOrder = UNSORTED_SORT_ORDER, - properties: Properties = EMPTY_DICT, - ) -> Table: - return Table( - identifier=Catalog.identifier_to_tuple(identifier), - metadata_location="s3://tmp/", - metadata=TableMetadataV2(**EXAMPLE_TABLE_METADATA_V2), - io=load_file_io(), - ) - - def load_table(self, identifier: Union[str, Identifier]) -> Table: - tuple_identifier = Catalog.identifier_to_tuple(identifier) - if tuple_identifier == ("default", "foo"): - return Table( - identifier=tuple_identifier, - metadata_location="s3://tmp/", - metadata=TableMetadataV2(**EXAMPLE_TABLE_METADATA_V2), - io=load_file_io(), - ) - else: - raise NoSuchTableError(f"Table does not exist: {'.'.join(tuple_identifier)}") - - def drop_table(self, identifier: Union[str, Identifier]) -> None: - tuple_identifier = Catalog.identifier_to_tuple(identifier) - if tuple_identifier == ("default", "foo"): - return None - else: - raise NoSuchTableError(f"Table does not exist: {identifier}") - - def purge_table(self, identifier: Union[str, Identifier]) -> None: - self.drop_table(identifier) - - def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table: - tuple_identifier = Catalog.identifier_to_tuple(from_identifier) - if tuple_identifier == ("default", "foo"): - return Table( - identifier=tuple_identifier, - metadata_location="s3://tmp/", - metadata=TableMetadataV2(**EXAMPLE_TABLE_METADATA_V2), - io=load_file_io(), - ) - else: - raise NoSuchTableError(f"Table does not exist: {from_identifier}") - - def create_namespace(self, namespace: Union[str, Identifier], properties: Properties = EMPTY_DICT) -> None: - return None - - def drop_namespace(self, namespace: Union[str, Identifier]) -> None: - tuple_identifier = Catalog.identifier_to_tuple(namespace) - if tuple_identifier == ("default",): - return None - else: - raise NoSuchNamespaceError(f"Namespace does not exist: {namespace}") - - def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]: - return [ - ("default", "foo"), - ("default", "bar"), - ] - - def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identifier]: - # No hierarchical namespaces for now - if namespace == (): - return [("default",), ("personal",)] - else: - return [] - - def load_namespace_properties(self, namespace: Union[str, Identifier]) -> Properties: - identifier = Catalog.identifier_to_tuple(namespace) - if identifier == ("default",): - return {"location": "s3://warehouse/database/location"} - else: - raise NoSuchNamespaceError(f"Namespace does not exist: {'.'.join(namespace)}") - - def update_namespace_properties( - self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Properties = EMPTY_DICT - ) -> PropertiesUpdateSummary: - identifier = Catalog.identifier_to_tuple(namespace) - - if identifier == ("default",): - return PropertiesUpdateSummary(removed=["location"], updated=[], missing=[]) - else: - raise NoSuchNamespaceError(f"Namespace does not exist: {namespace}") - - -MOCK_CATALOG = MockCatalog("production", uri="http://somewhere") -MOCK_ENVIRONMENT = {"PYICEBERG_CATALOG__PRODUCTION__URI": "test://doesnotexist"} +from tests.catalog.test_base import InMemoryCatalog -def test_missing_uri(empty_home_dir_path: str) -> None: +def test_missing_uri(mocker: MockFixture, empty_home_dir_path: str) -> None: # mock to prevent parsing ~/.pyiceberg.yaml or {PYICEBERG_HOME}/.pyiceberg.yaml - with mock.patch.dict(os.environ, {"HOME": empty_home_dir_path, "PYICEBERG_HOME": empty_home_dir_path}): - with mock.patch("pyiceberg.catalog._ENV_CONFIG", Config()): - runner = CliRunner() - result = runner.invoke(run, ["list"]) - assert result.exit_code == 1 - assert ( - result.output - == "URI missing, please provide using --uri, the config or environment variable \nPYICEBERG_CATALOG__DEFAULT__URI\n" - ) + mocker.patch.dict(os.environ, values={"HOME": empty_home_dir_path, "PYICEBERG_HOME": empty_home_dir_path}) + mocker.patch("pyiceberg.catalog._ENV_CONFIG", return_value=Config()) + + runner = CliRunner() + result = runner.invoke(run, ["list"]) + + assert result.exit_code == 1 + assert result.output == "Could not initialize catalog with the following properties: {}\n" + + +@pytest.fixture(autouse=True) +def env_vars(mocker: MockFixture) -> None: + mocker.patch.dict(os.environ, MOCK_ENVIRONMENT) + + +@pytest.fixture(name="catalog") +def fixture_catalog(mocker: MockFixture) -> InMemoryCatalog: + in_memory_catalog = InMemoryCatalog("test.in.memory.catalog", **{"test.key": "test.value"}) + mocker.patch("pyiceberg.cli.console.load_catalog", return_value=in_memory_catalog) + return in_memory_catalog + + +@pytest.fixture(name="namespace_properties") +def fixture_namespace_properties() -> Properties: + return TEST_NAMESPACE_PROPERTIES.copy() + + +TEST_TABLE_IDENTIFIER = ("default", "my_table") +TEST_TABLE_NAMESPACE = "default" +TEST_NAMESPACE_PROPERTIES = {"location": "s3://warehouse/database/location"} +TEST_TABLE_NAME = "my_table" +TEST_TABLE_SCHEMA = Schema(schema_id=0) +TEST_TABLE_LOCATION = "protocol://some/location" +TEST_TABLE_PARTITION_SPEC = PartitionSpec(PartitionField(name="x", transform=IdentityTransform(), source_id=1, field_id=1000)) +TEST_TABLE_PROPERTIES = {"read.split.target.size": "134217728"} +MOCK_ENVIRONMENT = {"PYICEBERG_CATALOG__PRODUCTION__URI": "test://doesnotexist"} -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_list_root(_: MockCatalog) -> None: +def test_list_root(catalog: InMemoryCatalog) -> None: + catalog.create_namespace(TEST_TABLE_NAMESPACE) + runner = CliRunner() result = runner.invoke(run, ["list"]) + assert result.exit_code == 0 - assert result.output == "default \npersonal\n" + assert TEST_TABLE_NAMESPACE in result.output -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_list_namespace(_: MockCatalog) -> None: +def test_list_namespace(catalog: InMemoryCatalog) -> None: + catalog.create_table( + identifier=TEST_TABLE_IDENTIFIER, + schema=TEST_TABLE_SCHEMA, + location=TEST_TABLE_LOCATION, + partition_spec=TEST_TABLE_PARTITION_SPEC, + properties=TEST_TABLE_PROPERTIES, + ) + runner = CliRunner() result = runner.invoke(run, ["list", "default"]) + assert result.exit_code == 0 - assert result.output == "default.foo\ndefault.bar\n" + assert result.output == "default.my_table\n" + +def test_describe_namespace(catalog: InMemoryCatalog, namespace_properties: Properties) -> None: + catalog.create_namespace(TEST_TABLE_NAMESPACE, namespace_properties) -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_describe_namespace(_: MockCatalog) -> None: runner = CliRunner() result = runner.invoke(run, ["describe", "default"]) + assert result.exit_code == 0 assert result.output == "location s3://warehouse/database/location\n" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_describe_namespace_does_not_exists(_: MockCatalog) -> None: +def test_describe_namespace_does_not_exists(catalog: InMemoryCatalog) -> None: + # pylint: disable=unused-argument + runner = CliRunner() result = runner.invoke(run, ["describe", "doesnotexist"]) + assert result.exit_code == 1 - assert result.output == "Namespace does not exist: doesnotexist\n" + assert result.output == "Namespace does not exist: ('doesnotexist',)\n" + +def test_describe_table(catalog: InMemoryCatalog) -> None: + catalog.create_table( + identifier=TEST_TABLE_IDENTIFIER, + schema=TEST_TABLE_SCHEMA, + location=TEST_TABLE_LOCATION, + partition_spec=TEST_TABLE_PARTITION_SPEC, + ) -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_describe_table(_: MockCatalog) -> None: runner = CliRunner() - result = runner.invoke(run, ["describe", "default.foo"]) + result = runner.invoke(run, ["describe", "default.my_table"]) assert result.exit_code == 0 assert ( # Strip the whitespace on the end "\n".join([line.rstrip() for line in result.output.split("\n")]) - == """Table format version 2 -Metadata location s3://tmp/ -Table UUID 9c12d441-03fe-4693-9a96-a0705ddf69c1 -Last Updated 1602638573590 + == """Table format version 1 +Metadata location s3://warehouse/default/my_table/metadata/metadata.json +Table UUID d20125c8-7284-442c-9aea-15fee620737c +Last Updated 1602638573874 Partition spec [ 1000: x: identity(1) ] -Sort order [ - 2 ASC NULLS FIRST - bucket[4](3) DESC NULLS LAST - ] -Current schema Schema, id=1 +Sort order [] +Current schema Schema, id=0 ├── 1: x: required long ├── 2: y: required long (comment) └── 3: z: required long -Current snapshot Operation.APPEND: id=3055729675574597004, - parent_id=3051729675574597004, schema_id=1 +Current snapshot None Snapshots Snapshots - ├── Snapshot 3051729675574597004, schema None: - │ s3://a/b/1.avro - └── Snapshot 3055729675574597004, schema 1: - s3://a/b/2.avro -Properties read.split.target.size 134217728 + └── Snapshot 1925, schema None +Properties """ ) -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_describe_table_does_not_exists(_: MockCatalog) -> None: +def test_describe_table_does_not_exists(catalog: InMemoryCatalog) -> None: + # pylint: disable=unused-argument + runner = CliRunner() - result = runner.invoke(run, ["describe", "default.doesnotexit"]) + result = runner.invoke(run, ["describe", "default.doesnotexist"]) assert result.exit_code == 1 - assert result.output == "Table or namespace does not exist: default.doesnotexit\n" + assert result.output == "Table or namespace does not exist: default.doesnotexist\n" + +def test_schema(catalog: InMemoryCatalog) -> None: + catalog.create_table( + identifier=TEST_TABLE_IDENTIFIER, + schema=TEST_TABLE_SCHEMA, + location=TEST_TABLE_LOCATION, + partition_spec=TEST_TABLE_PARTITION_SPEC, + ) -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_schema(_: MockCatalog) -> None: runner = CliRunner() - result = runner.invoke(run, ["schema", "default.foo"]) + result = runner.invoke(run, ["schema", "default.my_table"]) assert result.exit_code == 0 assert ( "\n".join([line.rstrip() for line in result.output.split("\n")]) @@ -305,20 +178,25 @@ def test_schema(_: MockCatalog) -> None: ) -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_schema_does_not_exists(_: MockCatalog) -> None: +def test_schema_does_not_exists(catalog: InMemoryCatalog) -> None: + # pylint: disable=unused-argument + runner = CliRunner() - result = runner.invoke(run, ["schema", "default.doesnotexit"]) + result = runner.invoke(run, ["schema", "default.doesnotexist"]) assert result.exit_code == 1 - assert result.output == "Table does not exist: default.doesnotexit\n" + assert result.output == "Table does not exist: ('default', 'doesnotexist')\n" + +def test_spec(catalog: InMemoryCatalog) -> None: + catalog.create_table( + identifier=TEST_TABLE_IDENTIFIER, + schema=TEST_TABLE_SCHEMA, + location=TEST_TABLE_LOCATION, + partition_spec=TEST_TABLE_PARTITION_SPEC, + ) -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_spec(_: MockCatalog) -> None: runner = CliRunner() - result = runner.invoke(run, ["spec", "default.foo"]) + result = runner.invoke(run, ["spec", "default.my_table"]) assert result.exit_code == 0 assert ( result.output @@ -329,334 +207,409 @@ def test_spec(_: MockCatalog) -> None: ) -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_spec_does_not_exists(_: MockCatalog) -> None: +def test_spec_does_not_exists(catalog: InMemoryCatalog) -> None: + # pylint: disable=unused-argument + runner = CliRunner() - result = runner.invoke(run, ["spec", "default.doesnotexit"]) + result = runner.invoke(run, ["spec", "default.doesnotexist"]) assert result.exit_code == 1 - assert result.output == "Table does not exist: default.doesnotexit\n" + assert result.output == "Table does not exist: ('default', 'doesnotexist')\n" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_uuid(_: MockCatalog) -> None: +def test_uuid(catalog: InMemoryCatalog) -> None: + catalog.create_table( + identifier=TEST_TABLE_IDENTIFIER, + schema=TEST_TABLE_SCHEMA, + location=TEST_TABLE_LOCATION, + partition_spec=TEST_TABLE_PARTITION_SPEC, + ) + runner = CliRunner() - result = runner.invoke(run, ["uuid", "default.foo"]) + result = runner.invoke(run, ["uuid", "default.my_table"]) assert result.exit_code == 0 - assert result.output == """9c12d441-03fe-4693-9a96-a0705ddf69c1\n""" + assert result.output == """d20125c8-7284-442c-9aea-15fee620737c\n""" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_uuid_does_not_exists(_: MockCatalog) -> None: +def test_uuid_does_not_exists(catalog: InMemoryCatalog) -> None: + # pylint: disable=unused-argument + runner = CliRunner() - result = runner.invoke(run, ["uuid", "default.doesnotexit"]) + result = runner.invoke(run, ["uuid", "default.doesnotexist"]) assert result.exit_code == 1 - assert result.output == "Table does not exist: default.doesnotexit\n" + assert result.output == "Table does not exist: ('default', 'doesnotexist')\n" + +def test_location(catalog: InMemoryCatalog) -> None: + catalog.create_table( + identifier=TEST_TABLE_IDENTIFIER, + schema=TEST_TABLE_SCHEMA, + location=TEST_TABLE_LOCATION, + partition_spec=TEST_TABLE_PARTITION_SPEC, + ) -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_location(_: MockCatalog) -> None: runner = CliRunner() - result = runner.invoke(run, ["location", "default.foo"]) + result = runner.invoke(run, ["location", "default.my_table"]) assert result.exit_code == 0 assert result.output == """s3://bucket/test/location\n""" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_location_does_not_exists(_: MockCatalog) -> None: +def test_location_does_not_exists(catalog: InMemoryCatalog) -> None: + # pylint: disable=unused-argument + runner = CliRunner() - result = runner.invoke(run, ["location", "default.doesnotexit"]) + result = runner.invoke(run, ["location", "default.doesnotexist"]) assert result.exit_code == 1 - assert result.output == "Table does not exist: default.doesnotexit\n" + assert result.output == "Table does not exist: ('default', 'doesnotexist')\n" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_drop_table(_: MockCatalog) -> None: +def test_drop_table(catalog: InMemoryCatalog) -> None: + catalog.create_table( + identifier=TEST_TABLE_IDENTIFIER, + schema=TEST_TABLE_SCHEMA, + location=TEST_TABLE_LOCATION, + partition_spec=TEST_TABLE_PARTITION_SPEC, + ) + runner = CliRunner() - result = runner.invoke(run, ["drop", "table", "default.foo"]) + result = runner.invoke(run, ["drop", "table", "default.my_table"]) assert result.exit_code == 0 - assert result.output == """Dropped table: default.foo\n""" + assert result.output == """Dropped table: default.my_table\n""" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_drop_table_does_not_exists(_: MockCatalog) -> None: +def test_drop_table_does_not_exists(catalog: InMemoryCatalog) -> None: + # pylint: disable=unused-argument + runner = CliRunner() - result = runner.invoke(run, ["drop", "table", "default.doesnotexit"]) + result = runner.invoke(run, ["drop", "table", "default.doesnotexist"]) assert result.exit_code == 1 - assert result.output == "Table does not exist: default.doesnotexit\n" + assert result.output == "Table does not exist: ('default', 'doesnotexist')\n" + +def test_drop_namespace(catalog: InMemoryCatalog) -> None: + catalog.create_namespace(TEST_TABLE_NAMESPACE) -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_drop_namespace(_: MockCatalog) -> None: runner = CliRunner() result = runner.invoke(run, ["drop", "namespace", "default"]) assert result.exit_code == 0 assert result.output == """Dropped namespace: default\n""" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_drop_namespace_does_not_exists(_: MockCatalog) -> None: +def test_drop_namespace_does_not_exists(catalog: InMemoryCatalog) -> None: + # pylint: disable=unused-argument + runner = CliRunner() - result = runner.invoke(run, ["drop", "namespace", "doesnotexit"]) + result = runner.invoke(run, ["drop", "namespace", "doesnotexist"]) assert result.exit_code == 1 - assert result.output == "Namespace does not exist: doesnotexit\n" + assert result.output == "Namespace does not exist: ('doesnotexist',)\n" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_rename_table(_: MockCatalog) -> None: +def test_rename_table(catalog: InMemoryCatalog) -> None: + catalog.create_table( + identifier=TEST_TABLE_IDENTIFIER, + schema=TEST_TABLE_SCHEMA, + location=TEST_TABLE_LOCATION, + partition_spec=TEST_TABLE_PARTITION_SPEC, + ) + runner = CliRunner() - result = runner.invoke(run, ["rename", "default.foo", "default.bar"]) + result = runner.invoke(run, ["rename", "default.my_table", "default.my_new_table"]) assert result.exit_code == 0 - assert result.output == """Renamed table from default.foo to default.bar\n""" + assert result.output == """Renamed table from default.my_table to default.my_new_table\n""" + +def test_rename_table_does_not_exists(catalog: InMemoryCatalog) -> None: + # pylint: disable=unused-argument -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_rename_table_does_not_exists(_: MockCatalog) -> None: runner = CliRunner() - result = runner.invoke(run, ["rename", "default.doesnotexit", "default.bar"]) + result = runner.invoke(run, ["rename", "default.doesnotexist", "default.bar"]) assert result.exit_code == 1 - assert result.output == "Table does not exist: default.doesnotexit\n" + assert result.output == "Table does not exist: ('default', 'doesnotexist')\n" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_properties_get_table(_: MockCatalog) -> None: +def test_properties_get_table(catalog: InMemoryCatalog) -> None: + catalog.create_table( + identifier=TEST_TABLE_IDENTIFIER, + schema=TEST_TABLE_SCHEMA, + location=TEST_TABLE_LOCATION, + partition_spec=TEST_TABLE_PARTITION_SPEC, + properties=TEST_TABLE_PROPERTIES, + ) + runner = CliRunner() - result = runner.invoke(run, ["properties", "get", "table", "default.foo"]) + result = runner.invoke(run, ["properties", "get", "table", "default.my_table"]) assert result.exit_code == 0 assert result.output == "read.split.target.size 134217728\n" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_properties_get_table_specific_property(_: MockCatalog) -> None: +def test_properties_get_table_specific_property(catalog: InMemoryCatalog) -> None: + catalog.create_table( + identifier=TEST_TABLE_IDENTIFIER, + schema=TEST_TABLE_SCHEMA, + location=TEST_TABLE_LOCATION, + partition_spec=TEST_TABLE_PARTITION_SPEC, + properties=TEST_TABLE_PROPERTIES, + ) + runner = CliRunner() - result = runner.invoke(run, ["properties", "get", "table", "default.foo", "read.split.target.size"]) + result = runner.invoke(run, ["properties", "get", "table", "default.my_table", "read.split.target.size"]) assert result.exit_code == 0 assert result.output == "134217728\n" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_properties_get_table_specific_property_that_doesnt_exist(_: MockCatalog) -> None: +def test_properties_get_table_specific_property_that_doesnt_exist(catalog: InMemoryCatalog) -> None: + catalog.create_table( + identifier=TEST_TABLE_IDENTIFIER, + schema=TEST_TABLE_SCHEMA, + location=TEST_TABLE_LOCATION, + partition_spec=TEST_TABLE_PARTITION_SPEC, + properties=TEST_TABLE_PROPERTIES, + ) + runner = CliRunner() - result = runner.invoke(run, ["properties", "get", "table", "default.foo", "doesnotexist"]) + result = runner.invoke(run, ["properties", "get", "table", "default.my_table", "doesnotexist"]) assert result.exit_code == 1 - assert result.output == "Could not find property doesnotexist on table default.foo\n" + assert result.output == "Could not find property doesnotexist on table default.my_table\n" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_properties_get_table_does_not_exist(_: MockCatalog) -> None: +def test_properties_get_table_does_not_exist(catalog: InMemoryCatalog) -> None: + # pylint: disable=unused-argument + runner = CliRunner() result = runner.invoke(run, ["properties", "get", "table", "doesnotexist"]) assert result.exit_code == 1 - assert result.output == "Table does not exist: doesnotexist\n" + assert result.output == "Table does not exist: ('doesnotexist',)\n" + +def test_properties_get_namespace(catalog: InMemoryCatalog, namespace_properties: Properties) -> None: + catalog.create_namespace(TEST_TABLE_NAMESPACE, namespace_properties) -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_properties_get_namespace(_: MockCatalog) -> None: runner = CliRunner() result = runner.invoke(run, ["properties", "get", "namespace", "default"]) assert result.exit_code == 0 assert result.output == "location s3://warehouse/database/location\n" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_properties_get_namespace_specific_property(_: MockCatalog) -> None: +def test_properties_get_namespace_specific_property(catalog: InMemoryCatalog, namespace_properties: Properties) -> None: + catalog.create_namespace(TEST_TABLE_NAMESPACE, namespace_properties) + runner = CliRunner() result = runner.invoke(run, ["properties", "get", "namespace", "default", "location"]) assert result.exit_code == 0 assert result.output == "s3://warehouse/database/location\n" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_properties_get_namespace_does_not_exist(_: MockCatalog) -> None: +def test_properties_get_namespace_does_not_exist(catalog: InMemoryCatalog, namespace_properties: Properties) -> None: + catalog.create_namespace(TEST_TABLE_NAMESPACE, namespace_properties) + runner = CliRunner() result = runner.invoke(run, ["properties", "get", "namespace", "doesnotexist"]) assert result.exit_code == 1 - assert result.output == "Namespace does not exist: doesnotexist\n" + assert result.output == "Namespace does not exist: ('doesnotexist',)\n" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_properties_set_namespace(_: MockCatalog) -> None: +def test_properties_set_namespace(catalog: InMemoryCatalog, namespace_properties: Properties) -> None: + catalog.create_namespace(TEST_TABLE_NAMESPACE, namespace_properties) + runner = CliRunner() result = runner.invoke(run, ["properties", "set", "namespace", "default", "location", "s3://new_location"]) assert result.exit_code == 0 assert result.output == "Updated location on default\n" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_properties_set_namespace_that_doesnt_exist(_: MockCatalog) -> None: +def test_properties_set_namespace_that_doesnt_exist(catalog: InMemoryCatalog) -> None: + # pylint: disable=unused-argument + runner = CliRunner() result = runner.invoke(run, ["properties", "set", "namespace", "doesnotexist", "location", "s3://new_location"]) assert result.exit_code == 1 - assert result.output == "Namespace does not exist: doesnotexist\n" + assert result.output == "Namespace does not exist: ('doesnotexist',)\n" + +def test_properties_set_table(catalog: InMemoryCatalog) -> None: + catalog.create_table( + identifier=TEST_TABLE_IDENTIFIER, + schema=TEST_TABLE_SCHEMA, + location=TEST_TABLE_LOCATION, + partition_spec=TEST_TABLE_PARTITION_SPEC, + ) -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_properties_set_table(_: MockCatalog) -> None: runner = CliRunner() - result = runner.invoke(run, ["properties", "set", "table", "default.foo", "location", "s3://new_location"]) + result = runner.invoke(run, ["properties", "set", "table", "default.my_table", "location", "s3://new_location"]) assert result.exit_code == 1 assert "Writing is WIP" in result.output -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_properties_set_table_does_not_exist(_: MockCatalog) -> None: +def test_properties_set_table_does_not_exist(catalog: InMemoryCatalog) -> None: + # pylint: disable=unused-argument + runner = CliRunner() result = runner.invoke(run, ["properties", "set", "table", "default.doesnotexist", "location", "s3://new_location"]) assert result.exit_code == 1 - assert result.output == "Table does not exist: default.doesnotexist\n" + assert result.output == "Table does not exist: ('default', 'doesnotexist')\n" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_properties_remove_namespace(_: MockCatalog) -> None: +def test_properties_remove_namespace(catalog: InMemoryCatalog, namespace_properties: Properties) -> None: + catalog.create_namespace(TEST_TABLE_NAMESPACE, namespace_properties) + runner = CliRunner() result = runner.invoke(run, ["properties", "remove", "namespace", "default", "location"]) assert result.exit_code == 0 assert result.output == "Property location removed from default\n" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_properties_remove_namespace_that_doesnt_exist(_: MockCatalog) -> None: +def test_properties_remove_namespace_that_doesnt_exist(catalog: InMemoryCatalog) -> None: + # pylint: disable=unused-argument + runner = CliRunner() result = runner.invoke(run, ["properties", "remove", "namespace", "doesnotexist", "location"]) assert result.exit_code == 1 - assert result.output == "Namespace does not exist: doesnotexist\n" + assert result.output == "Namespace does not exist: ('doesnotexist',)\n" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_properties_remove_table(_: MockCatalog) -> None: +def test_properties_remove_table(catalog: InMemoryCatalog) -> None: + catalog.create_table( + identifier=TEST_TABLE_IDENTIFIER, + schema=TEST_TABLE_SCHEMA, + location=TEST_TABLE_LOCATION, + partition_spec=TEST_TABLE_PARTITION_SPEC, + properties=TEST_TABLE_PROPERTIES, + ) + runner = CliRunner() - result = runner.invoke(run, ["properties", "remove", "table", "default.foo", "read.split.target.size"]) + result = runner.invoke(run, ["properties", "remove", "table", "default.my_table", "read.split.target.size"]) assert result.exit_code == 1 assert result.output == "Writing is WIP\n1\n" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_properties_remove_table_property_does_not_exists(_: MockCatalog) -> None: +def test_properties_remove_table_property_does_not_exists(catalog: InMemoryCatalog) -> None: + catalog.create_table( + identifier=TEST_TABLE_IDENTIFIER, + schema=TEST_TABLE_SCHEMA, + location=TEST_TABLE_LOCATION, + partition_spec=TEST_TABLE_PARTITION_SPEC, + ) + runner = CliRunner() - result = runner.invoke(run, ["properties", "remove", "table", "default.foo", "doesnotexist"]) + result = runner.invoke(run, ["properties", "remove", "table", "default.my_table", "doesnotexist"]) assert result.exit_code == 1 - assert result.output == "Property doesnotexist does not exist on default.foo\n" + assert result.output == "Property doesnotexist does not exist on default.my_table\n" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_properties_remove_table_does_not_exist(_: MockCatalog) -> None: +def test_properties_remove_table_does_not_exist(catalog: InMemoryCatalog) -> None: + # pylint: disable=unused-argument + runner = CliRunner() result = runner.invoke(run, ["properties", "remove", "table", "default.doesnotexist", "location"]) assert result.exit_code == 1 - assert result.output == "Table does not exist: default.doesnotexist\n" + assert result.output == "Table does not exist: ('default', 'doesnotexist')\n" + +def test_json_list_root(catalog: InMemoryCatalog) -> None: + catalog.create_namespace(TEST_TABLE_NAMESPACE) -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_list_root(_: MockCatalog) -> None: runner = CliRunner() result = runner.invoke(run, ["--output=json", "list"]) assert result.exit_code == 0 - assert result.output == """["default", "personal"]\n""" + assert result.output == """["default"]\n""" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_list_namespace(_: MockCatalog) -> None: +def test_json_list_namespace(catalog: InMemoryCatalog) -> None: + catalog.create_table( + identifier=TEST_TABLE_IDENTIFIER, + schema=TEST_TABLE_SCHEMA, + location=TEST_TABLE_LOCATION, + partition_spec=TEST_TABLE_PARTITION_SPEC, + ) + runner = CliRunner() result = runner.invoke(run, ["--output=json", "list", "default"]) assert result.exit_code == 0 - assert result.output == """["default.foo", "default.bar"]\n""" + assert result.output == """["default.my_table"]\n""" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_describe_namespace(_: MockCatalog) -> None: +def test_json_describe_namespace(catalog: InMemoryCatalog, namespace_properties: Properties) -> None: + catalog.create_namespace(TEST_TABLE_NAMESPACE, namespace_properties) + runner = CliRunner() result = runner.invoke(run, ["--output=json", "describe", "default"]) assert result.exit_code == 0 assert result.output == """{"location": "s3://warehouse/database/location"}\n""" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_describe_namespace_does_not_exists(_: MockCatalog) -> None: +def test_json_describe_namespace_does_not_exists(catalog: InMemoryCatalog) -> None: + # pylint: disable=unused-argument + runner = CliRunner() result = runner.invoke(run, ["--output=json", "describe", "doesnotexist"]) assert result.exit_code == 1 - assert result.output == """{"type": "NoSuchNamespaceError", "message": "Namespace does not exist: doesnotexist"}\n""" + assert result.output == """{"type": "NoSuchNamespaceError", "message": "Namespace does not exist: ('doesnotexist',)"}\n""" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_describe_table(_: MockCatalog) -> None: +def test_json_describe_table(catalog: InMemoryCatalog) -> None: + catalog.create_table( + identifier=TEST_TABLE_IDENTIFIER, + schema=TEST_TABLE_SCHEMA, + location=TEST_TABLE_LOCATION, + partition_spec=TEST_TABLE_PARTITION_SPEC, + ) + runner = CliRunner() - result = runner.invoke(run, ["--output=json", "describe", "default.foo"]) + result = runner.invoke(run, ["--output=json", "describe", "default.my_table"]) assert result.exit_code == 0 assert ( result.output - == """{"identifier": ["default", "foo"], "metadata_location": "s3://tmp/", "metadata": {"location": "s3://bucket/test/location", "table-uuid": "9c12d441-03fe-4693-9a96-a0705ddf69c1", "last-updated-ms": 1602638573590, "last-column-id": 3, "schemas": [{"type": "struct", "fields": [{"id": 1, "name": "x", "type": "long", "required": true}], "schema-id": 0, "identifier-field-ids": []}, {"type": "struct", "fields": [{"id": 1, "name": "x", "type": "long", "required": true}, {"id": 2, "name": "y", "type": "long", "required": true, "doc": "comment"}, {"id": 3, "name": "z", "type": "long", "required": true}], "schema-id": 1, "identifier-field-ids": [1, 2]}], "current-schema-id": 1, "partition-specs": [{"spec-id": 0, "fields": [{"source-id": 1, "field-id": 1000, "transform": "identity", "name": "x"}]}], "default-spec-id": 0, "last-partition-id": 1000, "properties": {"read.split.target.size": "134217728"}, "current-snapshot-id": 3055729675574597004, "snapshots": [{"snapshot-id": 3051729675574597004, "sequence-number": 0, "timestamp-ms": 1515100955770, "manifest-list": "s3://a/b/1.avro", "summary": {"operation": "append"}}, {"snapshot-id": 3055729675574597004, "parent-snapshot-id": 3051729675574597004, "sequence-number": 1, "timestamp-ms": 1555100955770, "manifest-list": "s3://a/b/2.avro", "summary": {"operation": "append"}, "schema-id": 1}], "snapshot-log": [{"snapshot-id": "3051729675574597004", "timestamp-ms": 1515100955770}, {"snapshot-id": "3055729675574597004", "timestamp-ms": 1555100955770}], "metadata-log": [{"metadata-file": "s3://bucket/.../v1.json", "timestamp-ms": 1515100}], "sort-orders": [{"order-id": 3, "fields": [{"source-id": 2, "transform": "identity", "direction": "asc", "null-order": "nulls-first"}, {"source-id": 3, "transform": "bucket[4]", "direction": "desc", "null-order": "nulls-last"}]}], "default-sort-order-id": 3, "refs": {"test": {"snapshot-id": 3051729675574597004, "type": "tag", "max-ref-age-ms": 10000000}, "main": {"snapshot-id": 3055729675574597004, "type": "branch"}}, "format-version": 2, "last-sequence-number": 34}}\n""" + == """{"identifier": ["default", "my_table"], "metadata_location": "s3://warehouse/default/my_table/metadata/metadata.json", "metadata": {"location": "s3://bucket/test/location", "table-uuid": "d20125c8-7284-442c-9aea-15fee620737c", "last-updated-ms": 1602638573874, "last-column-id": 3, "schemas": [{"type": "struct", "fields": [{"id": 1, "name": "x", "type": "long", "required": true}, {"id": 2, "name": "y", "type": "long", "required": true, "doc": "comment"}, {"id": 3, "name": "z", "type": "long", "required": true}], "schema-id": 0, "identifier-field-ids": []}], "current-schema-id": 0, "partition-specs": [{"spec-id": 0, "fields": [{"source-id": 1, "field-id": 1000, "transform": "identity", "name": "x"}]}], "default-spec-id": 0, "last-partition-id": 1000, "properties": {}, "snapshots": [{"snapshot-id": 1925, "timestamp-ms": 1602638573822}], "snapshot-log": [], "metadata-log": [], "sort-orders": [{"order-id": 0, "fields": []}], "default-sort-order-id": 0, "refs": {}, "format-version": 1, "schema": {"type": "struct", "fields": [{"id": 1, "name": "x", "type": "long", "required": true}, {"id": 2, "name": "y", "type": "long", "required": true, "doc": "comment"}, {"id": 3, "name": "z", "type": "long", "required": true}], "schema-id": 0, "identifier-field-ids": []}, "partition-spec": [{"name": "x", "transform": "identity", "source-id": 1, "field-id": 1000}]}}\n""" ) -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_describe_table_does_not_exists(_: MockCatalog) -> None: +def test_json_describe_table_does_not_exists(catalog: InMemoryCatalog) -> None: + # pylint: disable=unused-argument + runner = CliRunner() - result = runner.invoke(run, ["--output=json", "describe", "default.doesnotexit"]) + result = runner.invoke(run, ["--output=json", "describe", "default.doesnotexist"]) assert result.exit_code == 1 assert ( - result.output == """{"type": "NoSuchTableError", "message": "Table or namespace does not exist: default.doesnotexit"}\n""" + result.output + == """{"type": "NoSuchTableError", "message": "Table or namespace does not exist: default.doesnotexist"}\n""" ) -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_schema(_: MockCatalog) -> None: +def test_json_schema(catalog: InMemoryCatalog) -> None: + catalog.create_table( + identifier=TEST_TABLE_IDENTIFIER, + schema=TEST_TABLE_SCHEMA, + location=TEST_TABLE_LOCATION, + partition_spec=TEST_TABLE_PARTITION_SPEC, + ) + runner = CliRunner() - result = runner.invoke(run, ["--output=json", "schema", "default.foo"]) + result = runner.invoke(run, ["--output=json", "schema", "default.my_table"]) assert result.exit_code == 0 assert ( result.output - == """{"type": "struct", "fields": [{"id": 1, "name": "x", "type": "long", "required": true}, {"id": 2, "name": "y", "type": "long", "required": true, "doc": "comment"}, {"id": 3, "name": "z", "type": "long", "required": true}], "schema-id": 1, "identifier-field-ids": [1, 2]}\n""" + == """{"type": "struct", "fields": [{"id": 1, "name": "x", "type": "long", "required": true}, {"id": 2, "name": "y", "type": "long", "required": true, "doc": "comment"}, {"id": 3, "name": "z", "type": "long", "required": true}], "schema-id": 0, "identifier-field-ids": []}\n""" ) -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_schema_does_not_exists(_: MockCatalog) -> None: +def test_json_schema_does_not_exists(catalog: InMemoryCatalog) -> None: + # pylint: disable=unused-argument + runner = CliRunner() - result = runner.invoke(run, ["--output=json", "schema", "default.doesnotexit"]) + result = runner.invoke(run, ["--output=json", "schema", "default.doesnotexist"]) assert result.exit_code == 1 - assert result.output == """{"type": "NoSuchTableError", "message": "Table does not exist: default.doesnotexit"}\n""" + assert result.output == """{"type": "NoSuchTableError", "message": "Table does not exist: ('default', 'doesnotexist')"}\n""" + +def test_json_spec(catalog: InMemoryCatalog) -> None: + catalog.create_table( + identifier=TEST_TABLE_IDENTIFIER, + schema=TEST_TABLE_SCHEMA, + location=TEST_TABLE_LOCATION, + partition_spec=TEST_TABLE_PARTITION_SPEC, + ) -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_spec(_: MockCatalog) -> None: runner = CliRunner() - result = runner.invoke(run, ["--output=json", "spec", "default.foo"]) + result = runner.invoke(run, ["--output=json", "spec", "default.my_table"]) assert result.exit_code == 0 assert ( result.output @@ -664,254 +617,312 @@ def test_json_spec(_: MockCatalog) -> None: ) -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_spec_does_not_exists(_: MockCatalog) -> None: +def test_json_spec_does_not_exists(catalog: InMemoryCatalog) -> None: + # pylint: disable=unused-argument + runner = CliRunner() - result = runner.invoke(run, ["--output=json", "spec", "default.doesnotexit"]) + result = runner.invoke(run, ["--output=json", "spec", "default.doesnotexist"]) assert result.exit_code == 1 - assert result.output == """{"type": "NoSuchTableError", "message": "Table does not exist: default.doesnotexit"}\n""" + assert result.output == """{"type": "NoSuchTableError", "message": "Table does not exist: ('default', 'doesnotexist')"}\n""" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_uuid(_: MockCatalog) -> None: +def test_json_uuid(catalog: InMemoryCatalog) -> None: + catalog.create_table( + identifier=TEST_TABLE_IDENTIFIER, + schema=TEST_TABLE_SCHEMA, + location=TEST_TABLE_LOCATION, + partition_spec=TEST_TABLE_PARTITION_SPEC, + ) + runner = CliRunner() - result = runner.invoke(run, ["--output=json", "uuid", "default.foo"]) + result = runner.invoke(run, ["--output=json", "uuid", "default.my_table"]) assert result.exit_code == 0 - assert result.output == """{"uuid": "9c12d441-03fe-4693-9a96-a0705ddf69c1"}\n""" + assert result.output == """{"uuid": "d20125c8-7284-442c-9aea-15fee620737c"}\n""" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_uuid_does_not_exists(_: MockCatalog) -> None: +def test_json_uuid_does_not_exists(catalog: InMemoryCatalog) -> None: + # pylint: disable=unused-argument + runner = CliRunner() - result = runner.invoke(run, ["--output=json", "uuid", "default.doesnotexit"]) + result = runner.invoke(run, ["--output=json", "uuid", "default.doesnotexist"]) assert result.exit_code == 1 - assert result.output == """{"type": "NoSuchTableError", "message": "Table does not exist: default.doesnotexit"}\n""" + assert result.output == """{"type": "NoSuchTableError", "message": "Table does not exist: ('default', 'doesnotexist')"}\n""" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_location(_: MockCatalog) -> None: +def test_json_location(catalog: InMemoryCatalog) -> None: + catalog.create_table( + identifier=TEST_TABLE_IDENTIFIER, + schema=TEST_TABLE_SCHEMA, + location=TEST_TABLE_LOCATION, + partition_spec=TEST_TABLE_PARTITION_SPEC, + ) + runner = CliRunner() - result = runner.invoke(run, ["--output=json", "location", "default.foo"]) + result = runner.invoke(run, ["--output=json", "location", "default.my_table"]) assert result.exit_code == 0 assert result.output == """"s3://bucket/test/location"\n""" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_location_does_not_exists(_: MockCatalog) -> None: +def test_json_location_does_not_exists(catalog: InMemoryCatalog) -> None: + # pylint: disable=unused-argument + runner = CliRunner() result = runner.invoke(run, ["--output=json", "location", "default.doesnotexist"]) assert result.exit_code == 1 - assert result.output == """{"type": "NoSuchTableError", "message": "Table does not exist: default.doesnotexist"}\n""" + assert result.output == """{"type": "NoSuchTableError", "message": "Table does not exist: ('default', 'doesnotexist')"}\n""" + +def test_json_drop_table(catalog: InMemoryCatalog) -> None: + catalog.create_table( + identifier=TEST_TABLE_IDENTIFIER, + schema=TEST_TABLE_SCHEMA, + location=TEST_TABLE_LOCATION, + partition_spec=TEST_TABLE_PARTITION_SPEC, + ) -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_drop_table(_: MockCatalog) -> None: runner = CliRunner() - result = runner.invoke(run, ["--output=json", "drop", "table", "default.foo"]) + result = runner.invoke(run, ["--output=json", "drop", "table", "default.my_table"]) assert result.exit_code == 0 - assert result.output == """"Dropped table: default.foo"\n""" + assert result.output == """"Dropped table: default.my_table"\n""" + +def test_json_drop_table_does_not_exists(catalog: InMemoryCatalog) -> None: + # pylint: disable=unused-argument -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_drop_table_does_not_exists(_: MockCatalog) -> None: runner = CliRunner() result = runner.invoke(run, ["--output=json", "drop", "table", "default.doesnotexist"]) assert result.exit_code == 1 - assert result.output == """{"type": "NoSuchTableError", "message": "Table does not exist: default.doesnotexist"}\n""" + assert result.output == """{"type": "NoSuchTableError", "message": "Table does not exist: ('default', 'doesnotexist')"}\n""" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_drop_namespace(_: MockCatalog) -> None: +def test_json_drop_namespace(catalog: InMemoryCatalog) -> None: + catalog.create_namespace(TEST_TABLE_NAMESPACE) + runner = CliRunner() result = runner.invoke(run, ["--output=json", "drop", "namespace", "default"]) assert result.exit_code == 0 assert result.output == """"Dropped namespace: default"\n""" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_drop_namespace_does_not_exists(_: MockCatalog) -> None: +def test_json_drop_namespace_does_not_exists(catalog: InMemoryCatalog) -> None: + # pylint: disable=unused-argument + runner = CliRunner() result = runner.invoke(run, ["--output=json", "drop", "namespace", "doesnotexist"]) assert result.exit_code == 1 - assert result.output == """{"type": "NoSuchNamespaceError", "message": "Namespace does not exist: doesnotexist"}\n""" + assert result.output == """{"type": "NoSuchNamespaceError", "message": "Namespace does not exist: ('doesnotexist',)"}\n""" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_rename_table(_: MockCatalog) -> None: +def test_json_rename_table(catalog: InMemoryCatalog) -> None: + catalog.create_table( + identifier=TEST_TABLE_IDENTIFIER, + schema=TEST_TABLE_SCHEMA, + location=TEST_TABLE_LOCATION, + partition_spec=TEST_TABLE_PARTITION_SPEC, + ) + runner = CliRunner() - result = runner.invoke(run, ["--output=json", "rename", "default.foo", "default.bar"]) + result = runner.invoke(run, ["--output=json", "rename", "default.my_table", "default.my_new_table"]) assert result.exit_code == 0 - assert result.output == """"Renamed table from default.foo to default.bar"\n""" + assert result.output == """"Renamed table from default.my_table to default.my_new_table"\n""" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_rename_table_does_not_exists(_: MockCatalog) -> None: +def test_json_rename_table_does_not_exists(catalog: InMemoryCatalog) -> None: + # pylint: disable=unused-argument + runner = CliRunner() - result = runner.invoke(run, ["--output=json", "rename", "default.doesnotexit", "default.bar"]) + result = runner.invoke(run, ["--output=json", "rename", "default.doesnotexist", "default.bar"]) assert result.exit_code == 1 - assert result.output == """{"type": "NoSuchTableError", "message": "Table does not exist: default.doesnotexit"}\n""" + assert result.output == """{"type": "NoSuchTableError", "message": "Table does not exist: ('default', 'doesnotexist')"}\n""" + +def test_json_properties_get_table(catalog: InMemoryCatalog) -> None: + catalog.create_table( + identifier=TEST_TABLE_IDENTIFIER, + schema=TEST_TABLE_SCHEMA, + location=TEST_TABLE_LOCATION, + partition_spec=TEST_TABLE_PARTITION_SPEC, + properties=TEST_TABLE_PROPERTIES, + ) -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_properties_get_table(_: MockCatalog) -> None: runner = CliRunner() - result = runner.invoke(run, ["--output=json", "properties", "get", "table", "default.foo"]) + result = runner.invoke(run, ["--output=json", "properties", "get", "table", "default.my_table"]) assert result.exit_code == 0 assert result.output == """{"read.split.target.size": "134217728"}\n""" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_properties_get_table_specific_property(_: MockCatalog) -> None: +def test_json_properties_get_table_specific_property(catalog: InMemoryCatalog) -> None: + catalog.create_table( + identifier=TEST_TABLE_IDENTIFIER, + schema=TEST_TABLE_SCHEMA, + location=TEST_TABLE_LOCATION, + partition_spec=TEST_TABLE_PARTITION_SPEC, + properties=TEST_TABLE_PROPERTIES, + ) + runner = CliRunner() - result = runner.invoke(run, ["--output=json", "properties", "get", "table", "default.foo", "read.split.target.size"]) + result = runner.invoke(run, ["--output=json", "properties", "get", "table", "default.my_table", "read.split.target.size"]) assert result.exit_code == 0 assert result.output == """"134217728"\n""" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_properties_get_table_specific_property_that_doesnt_exist(_: MockCatalog) -> None: +def test_json_properties_get_table_specific_property_that_doesnt_exist(catalog: InMemoryCatalog) -> None: + catalog.create_table( + identifier=TEST_TABLE_IDENTIFIER, + schema=TEST_TABLE_SCHEMA, + location=TEST_TABLE_LOCATION, + partition_spec=TEST_TABLE_PARTITION_SPEC, + properties=TEST_TABLE_PROPERTIES, + ) + runner = CliRunner() - result = runner.invoke(run, ["--output=json", "properties", "get", "table", "default.foo", "doesnotexist"]) + result = runner.invoke(run, ["--output=json", "properties", "get", "table", "default.my_table", "doesnotexist"]) assert result.exit_code == 1 assert ( result.output - == """{"type": "NoSuchPropertyException", "message": "Could not find property doesnotexist on table default.foo"}\n""" + == """{"type": "NoSuchPropertyException", "message": "Could not find property doesnotexist on table default.my_table"}\n""" ) -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_properties_get_table_does_not_exist(_: MockCatalog) -> None: +def test_json_properties_get_table_does_not_exist(catalog: InMemoryCatalog) -> None: + # pylint: disable=unused-argument + runner = CliRunner() result = runner.invoke(run, ["--output=json", "properties", "get", "table", "doesnotexist"]) assert result.exit_code == 1 - assert result.output == """{"type": "NoSuchTableError", "message": "Table does not exist: doesnotexist"}\n""" + assert result.output == """{"type": "NoSuchTableError", "message": "Table does not exist: ('doesnotexist',)"}\n""" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_properties_get_namespace(_: MockCatalog) -> None: +def test_json_properties_get_namespace(catalog: InMemoryCatalog, namespace_properties: Properties) -> None: + catalog.create_namespace(TEST_TABLE_NAMESPACE, namespace_properties) + runner = CliRunner() result = runner.invoke(run, ["--output=json", "properties", "get", "namespace", "default"]) assert result.exit_code == 0 assert result.output == """{"location": "s3://warehouse/database/location"}\n""" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_properties_get_namespace_specific_property(_: MockCatalog) -> None: +def test_json_properties_get_namespace_specific_property(catalog: InMemoryCatalog, namespace_properties: Properties) -> None: + catalog.create_namespace(TEST_TABLE_NAMESPACE, namespace_properties) + runner = CliRunner() result = runner.invoke(run, ["--output=json", "properties", "get", "namespace", "default", "location"]) assert result.exit_code == 0 assert result.output == """"s3://warehouse/database/location"\n""" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_properties_get_namespace_does_not_exist(_: MockCatalog) -> None: +def test_json_properties_get_namespace_does_not_exist(catalog: InMemoryCatalog) -> None: + # pylint: disable=unused-argument + runner = CliRunner() result = runner.invoke(run, ["--output=json", "properties", "get", "namespace", "doesnotexist"]) assert result.exit_code == 1 - assert result.output == """{"type": "NoSuchNamespaceError", "message": "Namespace does not exist: doesnotexist"}\n""" + assert result.output == """{"type": "NoSuchNamespaceError", "message": "Namespace does not exist: ('doesnotexist',)"}\n""" + +def test_json_properties_set_namespace(catalog: InMemoryCatalog, namespace_properties: Properties) -> None: + catalog.create_namespace(TEST_TABLE_NAMESPACE, namespace_properties) -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_properties_set_namespace(_: MockCatalog) -> None: runner = CliRunner() result = runner.invoke(run, ["--output=json", "properties", "set", "namespace", "default", "location", "s3://new_location"]) assert result.exit_code == 0 assert result.output == """"Updated location on default"\n""" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_properties_set_namespace_that_doesnt_exist(_: MockCatalog) -> None: +def test_json_properties_set_namespace_that_doesnt_exist(catalog: InMemoryCatalog) -> None: + # pylint: disable=unused-argument + runner = CliRunner() result = runner.invoke( run, ["--output=json", "properties", "set", "namespace", "doesnotexist", "location", "s3://new_location"] ) assert result.exit_code == 1 - assert result.output == """{"type": "NoSuchNamespaceError", "message": "Namespace does not exist: doesnotexist"}\n""" + assert result.output == """{"type": "NoSuchNamespaceError", "message": "Namespace does not exist: ('doesnotexist',)"}\n""" + +def test_json_properties_set_table(catalog: InMemoryCatalog) -> None: + catalog.create_table( + identifier=TEST_TABLE_IDENTIFIER, + schema=TEST_TABLE_SCHEMA, + location=TEST_TABLE_LOCATION, + partition_spec=TEST_TABLE_PARTITION_SPEC, + properties=TEST_TABLE_PROPERTIES, + ) -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_properties_set_table(_: MockCatalog) -> None: runner = CliRunner() - result = runner.invoke(run, ["--output=json", "properties", "set", "table", "default.foo", "location", "s3://new_location"]) + result = runner.invoke( + run, ["--output=json", "properties", "set", "table", "default.my_table", "location", "s3://new_location"] + ) assert result.exit_code == 1 assert "Writing is WIP" in result.output -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_properties_set_table_does_not_exist(_: MockCatalog) -> None: +def test_json_properties_set_table_does_not_exist(catalog: InMemoryCatalog) -> None: + # pylint: disable=unused-argument + runner = CliRunner() result = runner.invoke( run, ["--output=json", "properties", "set", "table", "default.doesnotexist", "location", "s3://new_location"] ) assert result.exit_code == 1 - assert result.output == """{"type": "NoSuchTableError", "message": "Table does not exist: default.doesnotexist"}\n""" + assert result.output == """{"type": "NoSuchTableError", "message": "Table does not exist: ('default', 'doesnotexist')"}\n""" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_properties_remove_namespace(_: MockCatalog) -> None: +def test_json_properties_remove_namespace(catalog: InMemoryCatalog, namespace_properties: Properties) -> None: + catalog.create_namespace(TEST_TABLE_NAMESPACE, namespace_properties) + runner = CliRunner() result = runner.invoke(run, ["--output=json", "properties", "remove", "namespace", "default", "location"]) assert result.exit_code == 0 assert result.output == """"Property location removed from default"\n""" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_properties_remove_namespace_that_doesnt_exist(_: MockCatalog) -> None: +def test_json_properties_remove_namespace_that_doesnt_exist(catalog: InMemoryCatalog) -> None: + # pylint: disable=unused-argument + runner = CliRunner() result = runner.invoke(run, ["--output=json", "properties", "remove", "namespace", "doesnotexist", "location"]) assert result.exit_code == 1 - assert result.output == """{"type": "NoSuchNamespaceError", "message": "Namespace does not exist: doesnotexist"}\n""" + assert result.output == """{"type": "NoSuchNamespaceError", "message": "Namespace does not exist: ('doesnotexist',)"}\n""" -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_properties_remove_table(_: MockCatalog) -> None: +def test_json_properties_remove_table(catalog: InMemoryCatalog) -> None: + catalog.create_table( + identifier=TEST_TABLE_IDENTIFIER, + schema=TEST_TABLE_SCHEMA, + location=TEST_TABLE_LOCATION, + partition_spec=TEST_TABLE_PARTITION_SPEC, + properties=TEST_TABLE_PROPERTIES, + ) + runner = CliRunner() - result = runner.invoke(run, ["--output=json", "properties", "remove", "table", "default.foo", "read.split.target.size"]) + result = runner.invoke(run, ["--output=json", "properties", "remove", "table", "default.my_table", "read.split.target.size"]) assert result.exit_code == 1 assert "Writing is WIP" in result.output -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_properties_remove_table_property_does_not_exists(_: MockCatalog) -> None: +def test_json_properties_remove_table_property_does_not_exists(catalog: InMemoryCatalog) -> None: + catalog.create_table( + identifier=TEST_TABLE_IDENTIFIER, + schema=TEST_TABLE_SCHEMA, + location=TEST_TABLE_LOCATION, + partition_spec=TEST_TABLE_PARTITION_SPEC, + properties=TEST_TABLE_PROPERTIES, + ) + runner = CliRunner() - result = runner.invoke(run, ["--output=json", "properties", "remove", "table", "default.foo", "doesnotexist"]) + result = runner.invoke(run, ["--output=json", "properties", "remove", "table", "default.my_table", "doesnotexist"]) assert result.exit_code == 1 assert ( result.output - == """{"type": "NoSuchPropertyException", "message": "Property doesnotexist does not exist on default.foo"}\n""" + == """{"type": "NoSuchPropertyException", "message": "Property doesnotexist does not exist on default.my_table"}\n""" ) -@mock.patch.dict(os.environ, MOCK_ENVIRONMENT) -@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG) -def test_json_properties_remove_table_does_not_exist(_: MockCatalog) -> None: +def test_json_properties_remove_table_does_not_exist(catalog: InMemoryCatalog) -> None: + # pylint: disable=unused-argument + runner = CliRunner() result = runner.invoke(run, ["--output=json", "properties", "remove", "table", "default.doesnotexist", "location"]) assert result.exit_code == 1 - assert result.output == """{"type": "NoSuchTableError", "message": "Table does not exist: default.doesnotexist"}\n""" + assert result.output == """{"type": "NoSuchTableError", "message": "Table does not exist: ('default', 'doesnotexist')"}\n""" diff --git a/python/tests/conftest.py b/python/tests/conftest.py index 748d8fa76e48..e5d08604897e 100644 --- a/python/tests/conftest.py +++ b/python/tests/conftest.py @@ -336,6 +336,14 @@ def metadata_location(tmp_path_factory: pytest.TempPathFactory) -> str: return metadata_location +@pytest.fixture(scope="session") +def metadata_location_gz(tmp_path_factory: pytest.TempPathFactory) -> str: + metadata_location = str(tmp_path_factory.mktemp("metadata") / f"{uuid.uuid4()}.gz.metadata.json") + metadata = TableMetadataV2(**EXAMPLE_TABLE_METADATA_V2) + ToOutputFile.table_metadata(metadata, PyArrowFileIO().new_output(location=metadata_location), overwrite=True) + return metadata_location + + manifest_entry_records = [ { "status": 1, @@ -613,8 +621,8 @@ def metadata_location(tmp_path_factory: pytest.TempPathFactory) -> str: "manifest_length": 7989, "partition_spec_id": 0, "content": 1, - "sequence_number": None, # To be inherited - "min_sequence_number": None, # To be inherited + "sequence_number": 3, + "min_sequence_number": 3, "added_snapshot_id": 9182715666859759686, "added_files_count": 3, "existing_files_count": 0, @@ -1026,7 +1034,7 @@ def simple_map() -> MapType: class LocalOutputFile(OutputFile): - """An OutputFile implementation for local files (for test use only)""" + """An OutputFile implementation for local files (for test use only).""" def __init__(self, location: str) -> None: parsed_location = urlparse(location) # Create a ParseResult from the uri @@ -1041,6 +1049,7 @@ def __init__(self, location: str) -> None: self._path = parsed_location.path def __len__(self) -> int: + """Returns the length of an instance of the LocalOutputFile class.""" return os.path.getsize(self._path) def exists(self) -> bool: @@ -1289,9 +1298,9 @@ def fsspec_fileio(request: pytest.FixtureRequest) -> FsspecFileIO: class MockAWSResponse(aiobotocore.awsrequest.AioAWSResponse): - """ - A mocked aws response implementation (for test use only) - See https://github.com/aio-libs/aiobotocore/issues/755 + """A mocked aws response implementation (for test use only). + + See https://github.com/aio-libs/aiobotocore/issues/755. """ def __init__(self, response: botocore.awsrequest.AWSResponse) -> None: @@ -1308,9 +1317,9 @@ async def _text_prop(self) -> str: class MockHttpClientResponse(aiohttp.client_reqrep.ClientResponse): - """ - A mocked http client response implementation (for test use only) - See https://github.com/aio-libs/aiobotocore/issues/755 + """A mocked http client response implementation (for test use only). + + See https://github.com/aio-libs/aiobotocore/issues/755. """ def __init__(self, response: botocore.awsrequest.AWSResponse) -> None: @@ -1329,9 +1338,9 @@ def raw_headers(self) -> aiohttp.typedefs.RawHeaders: def patch_aiobotocore() -> None: - """ - Patch aiobotocore to work with moto - See https://github.com/aio-libs/aiobotocore/issues/755 + """Patch aiobotocore to work with moto. + + See https://github.com/aio-libs/aiobotocore/issues/755. """ def factory(original: Callable) -> Callable: # type: ignore @@ -1347,9 +1356,9 @@ def patched_convert_to_response_dict( # type: ignore @pytest.fixture(name="_patch_aiobotocore") def fixture_aiobotocore(): # type: ignore - """ - Patch aiobotocore to work with moto - pending close of this issue: https://github.com/aio-libs/aiobotocore/issues/755 + """Patch aiobotocore to work with moto. + + pending close of this issue: https://github.com/aio-libs/aiobotocore/issues/755. """ stored_method = aiobotocore.endpoint.convert_to_response_dict yield patch_aiobotocore() @@ -1378,21 +1387,21 @@ def fixture_aws_credentials() -> Generator[None, None, None]: @pytest.fixture(name="_s3") def fixture_s3(_aws_credentials: None) -> Generator[boto3.client, None, None]: - """Mocked S3 client""" + """Mocked S3 client.""" with mock_s3(): yield boto3.client("s3", region_name="us-east-1") @pytest.fixture(name="_glue") def fixture_glue(_aws_credentials: None) -> Generator[boto3.client, None, None]: - """Mocked glue client""" + """Mocked glue client.""" with mock_glue(): yield boto3.client("glue", region_name="us-east-1") @pytest.fixture(name="_dynamodb") def fixture_dynamodb(_aws_credentials: None) -> Generator[boto3.client, None, None]: - """Mocked DynamoDB client""" + """Mocked DynamoDB client.""" with mock_dynamodb(): yield boto3.client("dynamodb", region_name="us-east-1") @@ -1465,9 +1474,7 @@ def fixture_s3_bucket(_s3) -> None: # type: ignore def get_bucket_name() -> str: - """ - Set the environment variable AWS_TEST_BUCKET for a default bucket to test - """ + """Set the environment variable AWS_TEST_BUCKET for a default bucket to test.""" bucket_name = os.getenv("AWS_TEST_BUCKET") if bucket_name is None: raise ValueError("Please specify a bucket to run the test by setting environment variable AWS_TEST_BUCKET") @@ -1490,7 +1497,7 @@ def fixture_s3_client() -> boto3.client: def clean_up(test_catalog: Catalog) -> None: - """Clean all databases and tables created during the integration test""" + """Clean all databases and tables created during the integration test.""" for database_tuple in test_catalog.list_namespaces(): database_name = database_tuple[0] if "my_iceberg_database-" in database_name: diff --git a/python/tests/expressions/test_expressions.py b/python/tests/expressions/test_expressions.py index 5b82c14744cc..bd3a14165edb 100644 --- a/python/tests/expressions/test_expressions.py +++ b/python/tests/expressions/test_expressions.py @@ -16,6 +16,7 @@ # under the License. # pylint:disable=redefined-outer-name,eval-used +import pickle import uuid from decimal import Decimal from typing import Any @@ -679,6 +680,7 @@ def test_bound_reference(field: NestedField, accessor: Accessor) -> None: assert str(bound_ref) == f"BoundReference(field={repr(field)}, accessor={repr(accessor)})" assert repr(bound_ref) == f"BoundReference(field={repr(field)}, accessor={repr(accessor)})" assert bound_ref == eval(repr(bound_ref)) + assert bound_ref == pickle.loads(pickle.dumps(bound_ref)) def test_reference() -> None: @@ -687,6 +689,7 @@ def test_reference() -> None: assert str(ref) == "Reference(name='abc')" assert repr(ref) == "Reference(name='abc')" assert ref == eval(repr(ref)) + assert ref == pickle.loads(pickle.dumps(ref)) def test_and() -> None: @@ -696,6 +699,7 @@ def test_and() -> None: assert str(and_) == f"And(left={str(null)}, right={str(nan)})" assert repr(and_) == f"And(left={repr(null)}, right={repr(nan)})" assert and_ == eval(repr(and_)) + assert and_ == pickle.loads(pickle.dumps(and_)) def test_or() -> None: @@ -705,14 +709,16 @@ def test_or() -> None: assert str(or_) == f"Or(left={str(null)}, right={str(nan)})" assert repr(or_) == f"Or(left={repr(null)}, right={repr(nan)})" assert or_ == eval(repr(or_)) + assert or_ == pickle.loads(pickle.dumps(or_)) def test_not() -> None: null = IsNull(Reference("a")) - or_ = Not(null) - assert str(or_) == f"Not(child={str(null)})" - assert repr(or_) == f"Not(child={repr(null)})" - assert or_ == eval(repr(or_)) + not_ = Not(null) + assert str(not_) == f"Not(child={str(null)})" + assert repr(not_) == f"Not(child={repr(null)})" + assert not_ == eval(repr(not_)) + assert not_ == pickle.loads(pickle.dumps(not_)) def test_always_true() -> None: @@ -720,6 +726,7 @@ def test_always_true() -> None: assert str(always_true) == "AlwaysTrue()" assert repr(always_true) == "AlwaysTrue()" assert always_true == eval(repr(always_true)) + assert always_true == pickle.loads(pickle.dumps(always_true)) def test_always_false() -> None: @@ -727,6 +734,7 @@ def test_always_false() -> None: assert str(always_false) == "AlwaysFalse()" assert repr(always_false) == "AlwaysFalse()" assert always_false == eval(repr(always_false)) + assert always_false == pickle.loads(pickle.dumps(always_false)) def test_bound_reference_field_property() -> None: @@ -756,6 +764,7 @@ def test_is_null() -> None: assert str(is_null) == f"IsNull(term={str(ref)})" assert repr(is_null) == f"IsNull(term={repr(ref)})" assert is_null == eval(repr(is_null)) + assert is_null == pickle.loads(pickle.dumps(is_null)) def test_not_null() -> None: @@ -764,6 +773,7 @@ def test_not_null() -> None: assert str(non_null) == f"NotNull(term={str(ref)})" assert repr(non_null) == f"NotNull(term={repr(ref)})" assert non_null == eval(repr(non_null)) + assert non_null == pickle.loads(pickle.dumps(non_null)) def test_bound_is_nan(accessor: Accessor) -> None: @@ -776,6 +786,7 @@ def test_bound_is_nan(accessor: Accessor) -> None: assert str(bound_is_nan) == f"BoundIsNaN(term={str(term)})" assert repr(bound_is_nan) == f"BoundIsNaN(term={repr(term)})" assert bound_is_nan == eval(repr(bound_is_nan)) + assert bound_is_nan == pickle.loads(pickle.dumps(bound_is_nan)) def test_bound_is_not_nan(accessor: Accessor) -> None: @@ -788,6 +799,7 @@ def test_bound_is_not_nan(accessor: Accessor) -> None: assert str(bound_not_nan) == f"BoundNotNaN(term={str(term)})" assert repr(bound_not_nan) == f"BoundNotNaN(term={repr(term)})" assert bound_not_nan == eval(repr(bound_not_nan)) + assert bound_not_nan == pickle.loads(pickle.dumps(bound_not_nan)) def test_is_nan() -> None: @@ -796,6 +808,7 @@ def test_is_nan() -> None: assert str(is_nan) == f"IsNaN(term={str(ref)})" assert repr(is_nan) == f"IsNaN(term={repr(ref)})" assert is_nan == eval(repr(is_nan)) + assert is_nan == pickle.loads(pickle.dumps(is_nan)) def test_not_nan() -> None: @@ -804,6 +817,7 @@ def test_not_nan() -> None: assert str(not_nan) == f"NotNaN(term={str(ref)})" assert repr(not_nan) == f"NotNaN(term={repr(ref)})" assert not_nan == eval(repr(not_nan)) + assert not_nan == pickle.loads(pickle.dumps(not_nan)) def test_bound_in(term: BoundReference[Any]) -> None: @@ -811,6 +825,7 @@ def test_bound_in(term: BoundReference[Any]) -> None: assert str(bound_in) == f"BoundIn({str(term)}, {{a, b, c}})" assert repr(bound_in) == f"BoundIn({repr(term)}, {{literal('a'), literal('b'), literal('c')}})" assert bound_in == eval(repr(bound_in)) + assert bound_in == pickle.loads(pickle.dumps(bound_in)) def test_bound_not_in(term: BoundReference[Any]) -> None: @@ -818,6 +833,7 @@ def test_bound_not_in(term: BoundReference[Any]) -> None: assert str(bound_not_in) == f"BoundNotIn({str(term)}, {{a, b, c}})" assert repr(bound_not_in) == f"BoundNotIn({repr(term)}, {{literal('a'), literal('b'), literal('c')}})" assert bound_not_in == eval(repr(bound_not_in)) + assert bound_not_in == pickle.loads(pickle.dumps(bound_not_in)) def test_in() -> None: @@ -826,6 +842,7 @@ def test_in() -> None: assert str(unbound_in) == f"In({str(ref)}, {{a, b, c}})" assert repr(unbound_in) == f"In({repr(ref)}, {{literal('a'), literal('b'), literal('c')}})" assert unbound_in == eval(repr(unbound_in)) + assert unbound_in == pickle.loads(pickle.dumps(unbound_in)) def test_not_in() -> None: @@ -834,6 +851,7 @@ def test_not_in() -> None: assert str(not_in) == f"NotIn({str(ref)}, {{a, b, c}})" assert repr(not_in) == f"NotIn({repr(ref)}, {{literal('a'), literal('b'), literal('c')}})" assert not_in == eval(repr(not_in)) + assert not_in == pickle.loads(pickle.dumps(not_in)) def test_bound_equal_to(term: BoundReference[Any]) -> None: @@ -841,6 +859,7 @@ def test_bound_equal_to(term: BoundReference[Any]) -> None: assert str(bound_equal_to) == f"BoundEqualTo(term={str(term)}, literal=literal('a'))" assert repr(bound_equal_to) == f"BoundEqualTo(term={repr(term)}, literal=literal('a'))" assert bound_equal_to == eval(repr(bound_equal_to)) + assert bound_equal_to == pickle.loads(pickle.dumps(bound_equal_to)) def test_bound_not_equal_to(term: BoundReference[Any]) -> None: @@ -848,6 +867,7 @@ def test_bound_not_equal_to(term: BoundReference[Any]) -> None: assert str(bound_not_equal_to) == f"BoundNotEqualTo(term={str(term)}, literal=literal('a'))" assert repr(bound_not_equal_to) == f"BoundNotEqualTo(term={repr(term)}, literal=literal('a'))" assert bound_not_equal_to == eval(repr(bound_not_equal_to)) + assert bound_not_equal_to == pickle.loads(pickle.dumps(bound_not_equal_to)) def test_bound_greater_than_or_equal_to(term: BoundReference[Any]) -> None: @@ -855,6 +875,7 @@ def test_bound_greater_than_or_equal_to(term: BoundReference[Any]) -> None: assert str(bound_greater_than_or_equal_to) == f"BoundGreaterThanOrEqual(term={str(term)}, literal=literal('a'))" assert repr(bound_greater_than_or_equal_to) == f"BoundGreaterThanOrEqual(term={repr(term)}, literal=literal('a'))" assert bound_greater_than_or_equal_to == eval(repr(bound_greater_than_or_equal_to)) + assert bound_greater_than_or_equal_to == pickle.loads(pickle.dumps(bound_greater_than_or_equal_to)) def test_bound_greater_than(term: BoundReference[Any]) -> None: @@ -862,6 +883,7 @@ def test_bound_greater_than(term: BoundReference[Any]) -> None: assert str(bound_greater_than) == f"BoundGreaterThan(term={str(term)}, literal=literal('a'))" assert repr(bound_greater_than) == f"BoundGreaterThan(term={repr(term)}, literal=literal('a'))" assert bound_greater_than == eval(repr(bound_greater_than)) + assert bound_greater_than == pickle.loads(pickle.dumps(bound_greater_than)) def test_bound_less_than(term: BoundReference[Any]) -> None: @@ -869,6 +891,7 @@ def test_bound_less_than(term: BoundReference[Any]) -> None: assert str(bound_less_than) == f"BoundLessThan(term={str(term)}, literal=literal('a'))" assert repr(bound_less_than) == f"BoundLessThan(term={repr(term)}, literal=literal('a'))" assert bound_less_than == eval(repr(bound_less_than)) + assert bound_less_than == pickle.loads(pickle.dumps(bound_less_than)) def test_bound_less_than_or_equal(term: BoundReference[Any]) -> None: @@ -876,6 +899,7 @@ def test_bound_less_than_or_equal(term: BoundReference[Any]) -> None: assert str(bound_less_than_or_equal) == f"BoundLessThanOrEqual(term={str(term)}, literal=literal('a'))" assert repr(bound_less_than_or_equal) == f"BoundLessThanOrEqual(term={repr(term)}, literal=literal('a'))" assert bound_less_than_or_equal == eval(repr(bound_less_than_or_equal)) + assert bound_less_than_or_equal == pickle.loads(pickle.dumps(bound_less_than_or_equal)) def test_equal_to() -> None: @@ -883,6 +907,7 @@ def test_equal_to() -> None: assert str(equal_to) == "EqualTo(term=Reference(name='a'), literal=literal('a'))" assert repr(equal_to) == "EqualTo(term=Reference(name='a'), literal=literal('a'))" assert equal_to == eval(repr(equal_to)) + assert equal_to == pickle.loads(pickle.dumps(equal_to)) def test_not_equal_to() -> None: @@ -890,6 +915,7 @@ def test_not_equal_to() -> None: assert str(not_equal_to) == "NotEqualTo(term=Reference(name='a'), literal=literal('a'))" assert repr(not_equal_to) == "NotEqualTo(term=Reference(name='a'), literal=literal('a'))" assert not_equal_to == eval(repr(not_equal_to)) + assert not_equal_to == pickle.loads(pickle.dumps(not_equal_to)) def test_greater_than_or_equal_to() -> None: @@ -897,6 +923,7 @@ def test_greater_than_or_equal_to() -> None: assert str(greater_than_or_equal_to) == "GreaterThanOrEqual(term=Reference(name='a'), literal=literal('a'))" assert repr(greater_than_or_equal_to) == "GreaterThanOrEqual(term=Reference(name='a'), literal=literal('a'))" assert greater_than_or_equal_to == eval(repr(greater_than_or_equal_to)) + assert greater_than_or_equal_to == pickle.loads(pickle.dumps(greater_than_or_equal_to)) def test_greater_than() -> None: @@ -904,6 +931,7 @@ def test_greater_than() -> None: assert str(greater_than) == "GreaterThan(term=Reference(name='a'), literal=literal('a'))" assert repr(greater_than) == "GreaterThan(term=Reference(name='a'), literal=literal('a'))" assert greater_than == eval(repr(greater_than)) + assert greater_than == pickle.loads(pickle.dumps(greater_than)) def test_less_than() -> None: @@ -911,6 +939,7 @@ def test_less_than() -> None: assert str(less_than) == "LessThan(term=Reference(name='a'), literal=literal('a'))" assert repr(less_than) == "LessThan(term=Reference(name='a'), literal=literal('a'))" assert less_than == eval(repr(less_than)) + assert less_than == pickle.loads(pickle.dumps(less_than)) def test_less_than_or_equal() -> None: @@ -918,6 +947,7 @@ def test_less_than_or_equal() -> None: assert str(less_than_or_equal) == "LessThanOrEqual(term=Reference(name='a'), literal=literal('a'))" assert repr(less_than_or_equal) == "LessThanOrEqual(term=Reference(name='a'), literal=literal('a'))" assert less_than_or_equal == eval(repr(less_than_or_equal)) + assert less_than_or_equal == pickle.loads(pickle.dumps(less_than_or_equal)) def test_bound_reference_eval(table_schema_simple: Schema) -> None: diff --git a/python/tests/expressions/test_parser.py b/python/tests/expressions/test_parser.py index 80b6c8d0ed72..f4bebca066f0 100644 --- a/python/tests/expressions/test_parser.py +++ b/python/tests/expressions/test_parser.py @@ -50,112 +50,119 @@ def test_false() -> None: def test_is_null() -> None: - assert IsNull("x") == parser.parse("x is null") - assert IsNull("x") == parser.parse("x IS NULL") + assert IsNull("foo") == parser.parse("foo is null") + assert IsNull("foo") == parser.parse("foo IS NULL") + assert IsNull("foo") == parser.parse("table.foo IS NULL") def test_not_null() -> None: - assert NotNull("x") == parser.parse("x is not null") - assert NotNull("x") == parser.parse("x IS NOT NULL") + assert NotNull("foo") == parser.parse("foo is not null") + assert NotNull("foo") == parser.parse("foo IS NOT NULL") def test_is_nan() -> None: - assert IsNaN("x") == parser.parse("x is nan") - assert IsNaN("x") == parser.parse("x IS NAN") + assert IsNaN("foo") == parser.parse("foo is nan") + assert IsNaN("foo") == parser.parse("foo IS NAN") def test_not_nan() -> None: - assert NotNaN("x") == parser.parse("x is not nan") - assert NotNaN("x") == parser.parse("x IS NOT NaN") + assert NotNaN("foo") == parser.parse("foo is not nan") + assert NotNaN("foo") == parser.parse("foo IS NOT NaN") def test_less_than() -> None: - assert LessThan("x", 5) == parser.parse("x < 5") - assert LessThan("x", "a") == parser.parse("'a' > x") + assert LessThan("foo", 5) == parser.parse("foo < 5") + assert LessThan("foo", "a") == parser.parse("'a' > foo") def test_less_than_or_equal() -> None: - assert LessThanOrEqual("x", 5) == parser.parse("x <= 5") - assert LessThanOrEqual("x", "a") == parser.parse("'a' >= x") + assert LessThanOrEqual("foo", 5) == parser.parse("foo <= 5") + assert LessThanOrEqual("foo", "a") == parser.parse("'a' >= foo") def test_greater_than() -> None: - assert GreaterThan("x", 5) == parser.parse("x > 5") - assert GreaterThan("x", "a") == parser.parse("'a' < x") + assert GreaterThan("foo", 5) == parser.parse("foo > 5") + assert GreaterThan("foo", "a") == parser.parse("'a' < foo") def test_greater_than_or_equal() -> None: - assert GreaterThanOrEqual("x", 5) == parser.parse("x <= 5") - assert GreaterThanOrEqual("x", "a") == parser.parse("'a' >= x") + assert GreaterThanOrEqual("foo", 5) == parser.parse("foo <= 5") + assert GreaterThanOrEqual("foo", "a") == parser.parse("'a' >= foo") def test_equal_to() -> None: - assert EqualTo("x", 5) == parser.parse("x = 5") - assert EqualTo("x", "a") == parser.parse("'a' = x") - assert EqualTo("x", "a") == parser.parse("x == 'a'") - assert EqualTo("x", 5) == parser.parse("5 == x") + assert EqualTo("foo", 5) == parser.parse("foo = 5") + assert EqualTo("foo", "a") == parser.parse("'a' = foo") + assert EqualTo("foo", "a") == parser.parse("foo == 'a'") + assert EqualTo("foo", 5) == parser.parse("5 == foo") def test_not_equal_to() -> None: - assert NotEqualTo("x", 5) == parser.parse("x != 5") - assert NotEqualTo("x", "a") == parser.parse("'a' != x") - assert NotEqualTo("x", "a") == parser.parse("x <> 'a'") - assert NotEqualTo("x", 5) == parser.parse("5 <> x") + assert NotEqualTo("foo", 5) == parser.parse("foo != 5") + assert NotEqualTo("foo", "a") == parser.parse("'a' != foo") + assert NotEqualTo("foo", "a") == parser.parse("foo <> 'a'") + assert NotEqualTo("foo", 5) == parser.parse("5 <> foo") def test_in() -> None: - assert In("x", {5, 6, 7}) == parser.parse("x in (5, 6, 7)") - assert In("x", {"a", "b", "c"}) == parser.parse("x IN ('a', 'b', 'c')") + assert In("foo", {5, 6, 7}) == parser.parse("foo in (5, 6, 7)") + assert In("foo", {"a", "b", "c"}) == parser.parse("foo IN ('a', 'b', 'c')") def test_in_different_types() -> None: with pytest.raises(ParseException): - parser.parse("x in (5, 'a')") + parser.parse("foo in (5, 'a')") def test_not_in() -> None: - assert NotIn("x", {5, 6, 7}) == parser.parse("x not in (5, 6, 7)") - assert NotIn("x", {"a", "b", "c"}) == parser.parse("x NOT IN ('a', 'b', 'c')") + assert NotIn("foo", {5, 6, 7}) == parser.parse("foo not in (5, 6, 7)") + assert NotIn("foo", {"a", "b", "c"}) == parser.parse("foo NOT IN ('a', 'b', 'c')") def test_not_in_different_types() -> None: with pytest.raises(ParseException): - parser.parse("x not in (5, 'a')") + parser.parse("foo not in (5, 'a')") def test_simple_and() -> None: - assert And(GreaterThanOrEqual("x", 5), LessThan("x", 10)) == parser.parse("5 <= x and x < 10") + assert And(GreaterThanOrEqual("foo", 5), LessThan("foo", 10)) == parser.parse("5 <= foo and foo < 10") def test_and_with_not() -> None: - assert And(Not(GreaterThanOrEqual("x", 5)), LessThan("x", 10)) == parser.parse("not 5 <= x and x < 10") - assert And(GreaterThanOrEqual("x", 5), Not(LessThan("x", 10))) == parser.parse("5 <= x and not x < 10") + assert And(Not(GreaterThanOrEqual("foo", 5)), LessThan("foo", 10)) == parser.parse("not 5 <= foo and foo < 10") + assert And(GreaterThanOrEqual("foo", 5), Not(LessThan("foo", 10))) == parser.parse("5 <= foo and not foo < 10") def test_or_with_not() -> None: - assert Or(Not(LessThan("x", 5)), GreaterThan("x", 10)) == parser.parse("not x < 5 or 10 < x") - assert Or(LessThan("x", 5), Not(GreaterThan("x", 10))) == parser.parse("x < 5 or not 10 < x") + assert Or(Not(LessThan("foo", 5)), GreaterThan("foo", 10)) == parser.parse("not foo < 5 or 10 < foo") + assert Or(LessThan("foo", 5), Not(GreaterThan("foo", 10))) == parser.parse("foo < 5 or not 10 < foo") def test_simple_or() -> None: - assert Or(LessThan("x", 5), GreaterThan("x", 10)) == parser.parse("x < 5 or 10 < x") + assert Or(LessThan("foo", 5), GreaterThan("foo", 10)) == parser.parse("foo < 5 or 10 < foo") def test_and_or_without_parens() -> None: - assert Or(And(NotNull("x"), LessThan("x", 5)), GreaterThan("x", 10)) == parser.parse("x is not null and x < 5 or 10 < x") - assert Or(IsNull("x"), And(GreaterThanOrEqual("x", 5), LessThan("x", 10))) == parser.parse("x is null or 5 <= x and x < 10") + assert Or(And(NotNull("foo"), LessThan("foo", 5)), GreaterThan("foo", 10)) == parser.parse( + "foo is not null and foo < 5 or 10 < foo" + ) + assert Or(IsNull("foo"), And(GreaterThanOrEqual("foo", 5), LessThan("foo", 10))) == parser.parse( + "foo is null or 5 <= foo and foo < 10" + ) def test_and_or_with_parens() -> None: - assert And(NotNull("x"), Or(LessThan("x", 5), GreaterThan("x", 10))) == parser.parse("x is not null and (x < 5 or 10 < x)") - assert Or(IsNull("x"), And(GreaterThanOrEqual("x", 5), Not(LessThan("x", 10)))) == parser.parse( - "(x is null) or (5 <= x) and not(x < 10)" + assert And(NotNull("foo"), Or(LessThan("foo", 5), GreaterThan("foo", 10))) == parser.parse( + "foo is not null and (foo < 5 or 10 < foo)" + ) + assert Or(IsNull("foo"), And(GreaterThanOrEqual("foo", 5), Not(LessThan("foo", 10)))) == parser.parse( + "(foo is null) or (5 <= foo) and not(foo < 10)" ) def test_starts_with() -> None: - assert StartsWith("x", "data") == parser.parse("x LIKE 'data'") + assert StartsWith("foo", "data") == parser.parse("foo LIKE 'data'") def test_not_starts_with() -> None: - assert NotStartsWith("x", "data") == parser.parse("x NOT LIKE 'data'") + assert NotStartsWith("foo", "data") == parser.parse("foo NOT LIKE 'data'") diff --git a/python/tests/expressions/test_visitors.py b/python/tests/expressions/test_visitors.py index 001d83c71980..50db90ceac03 100644 --- a/python/tests/expressions/test_visitors.py +++ b/python/tests/expressions/test_visitors.py @@ -68,6 +68,7 @@ BooleanExpressionVisitor, BoundBooleanExpressionVisitor, _ManifestEvalVisitor, + expression_evaluator, expression_to_plain_format, rewrite_not, rewrite_to_dnf, @@ -76,6 +77,7 @@ ) from pyiceberg.manifest import ManifestFile, PartitionFieldSummary from pyiceberg.schema import Accessor, Schema +from pyiceberg.typedef import Record from pyiceberg.types import ( DoubleType, FloatType, @@ -1608,3 +1610,22 @@ def test_dnf_to_dask(table_schema_simple: Schema) -> None: ), ) assert expression_to_plain_format(expr) == [[("foo", ">", "hello")], [("bar", "in", {1, 2, 3}), ("baz", "==", True)]] + + +def test_expression_evaluator_null() -> None: + struct = Record(a=None) + schema = Schema(NestedField(1, "a", IntegerType(), required=False), schema_id=1) + assert expression_evaluator(schema, In("a", {1, 2, 3}), case_sensitive=True)(struct) is False + assert expression_evaluator(schema, NotIn("a", {1, 2, 3}), case_sensitive=True)(struct) is True + assert expression_evaluator(schema, IsNaN("a"), case_sensitive=True)(struct) is False + assert expression_evaluator(schema, NotNaN("a"), case_sensitive=True)(struct) is True + assert expression_evaluator(schema, IsNull("a"), case_sensitive=True)(struct) is True + assert expression_evaluator(schema, NotNull("a"), case_sensitive=True)(struct) is False + assert expression_evaluator(schema, EqualTo("a", 1), case_sensitive=True)(struct) is False + assert expression_evaluator(schema, NotEqualTo("a", 1), case_sensitive=True)(struct) is True + assert expression_evaluator(schema, GreaterThanOrEqual("a", 1), case_sensitive=True)(struct) is False + assert expression_evaluator(schema, GreaterThan("a", 1), case_sensitive=True)(struct) is False + assert expression_evaluator(schema, LessThanOrEqual("a", 1), case_sensitive=True)(struct) is False + assert expression_evaluator(schema, LessThan("a", 1), case_sensitive=True)(struct) is False + assert expression_evaluator(schema, StartsWith("a", 1), case_sensitive=True)(struct) is False + assert expression_evaluator(schema, NotStartsWith("a", 1), case_sensitive=True)(struct) is True diff --git a/python/tests/io/test_pyarrow.py b/python/tests/io/test_pyarrow.py index ac09bcfb4b36..01a8f35b1dda 100644 --- a/python/tests/io/test_pyarrow.py +++ b/python/tests/io/test_pyarrow.py @@ -24,9 +24,10 @@ import pyarrow as pa import pyarrow.parquet as pq import pytest -from pyarrow.fs import FileType +from pyarrow.fs import FileType, LocalFileSystem from pyiceberg.avro.resolver import ResolveError +from pyiceberg.catalog.noop import NoopCatalog from pyiceberg.expressions import ( AlwaysFalse, AlwaysTrue, @@ -57,11 +58,12 @@ PyArrowFile, PyArrowFileIO, _ConvertToArrowSchema, + _read_deletes, expression_to_pyarrow, project_table, schema_to_pyarrow, ) -from pyiceberg.manifest import DataFile, FileFormat +from pyiceberg.manifest import DataFile, DataFileContent, FileFormat from pyiceberg.partitioning import PartitionSpec from pyiceberg.schema import Schema, visit from pyiceberg.table import FileScanTask, Table @@ -300,6 +302,21 @@ def test_deleting_s3_file_not_found() -> None: assert "Cannot delete file, does not exist:" in str(exc_info.value) +def test_deleting_hdfs_file_not_found() -> None: + """Test that a PyArrowFile raises a PermissionError when the pyarrow error includes 'No such file or directory'""" + + hdfs_mock = MagicMock() + hdfs_mock.delete_file.side_effect = OSError("Path does not exist") + + with patch.object(PyArrowFileIO, "_get_fs") as submocked: + submocked.return_value = hdfs_mock + + with pytest.raises(FileNotFoundError) as exc_info: + PyArrowFileIO().delete("hdfs://foo/bar.txt") + + assert "Cannot delete file, does not exist:" in str(exc_info.value) + + def test_schema_to_pyarrow_schema(table_schema_nested: Schema) -> None: actual = schema_to_pyarrow(table_schema_nested) expected = """foo: string @@ -798,7 +815,14 @@ def project( return project_table( [ FileScanTask( - DataFile(file_path=file, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3) + DataFile( + content=DataFileContent.DATA, + file_path=file, + file_format=FileFormat.PARQUET, + partition={}, + record_count=3, + file_size_in_bytes=3, + ) ) for file in files ], @@ -813,6 +837,7 @@ def project( ), metadata_location="file://a/b/c.json", io=PyArrowFileIO(), + catalog=NoopCatalog("NoopCatalog"), ), expr or AlwaysTrue(), schema, @@ -1186,6 +1211,106 @@ def test_projection_filter_on_unknown_field(schema_int_str: Schema, file_int_str assert "Could not find field with name unknown_field, case_sensitive=True" in str(exc_info.value) +@pytest.fixture +def deletes_file(tmp_path: str, example_task: FileScanTask) -> str: + path = example_task.file.file_path + table = pa.table({"file_path": [path, path, path], "pos": [1, 3, 5]}) + + deletes_file_path = f"{tmp_path}/deletes.parquet" + pq.write_table(table, deletes_file_path) + + return deletes_file_path + + +def test_read_deletes(deletes_file: str, example_task: FileScanTask) -> None: + deletes = _read_deletes(LocalFileSystem(), DataFile(file_path=deletes_file, file_format=FileFormat.PARQUET)) + assert set(deletes.keys()) == {example_task.file.file_path} + assert list(deletes.values())[0] == pa.chunked_array([[1, 3, 5]]) + + +def test_delete(deletes_file: str, example_task: FileScanTask, table_schema_simple: Schema) -> None: + metadata_location = "file://a/b/c.json" + example_task_with_delete = FileScanTask( + data_file=example_task.file, + delete_files={DataFile(content=DataFileContent.POSITION_DELETES, file_path=deletes_file, file_format=FileFormat.PARQUET)}, + ) + + with_deletes = project_table( + tasks=[example_task_with_delete], + table=Table( + ("namespace", "table"), + metadata=TableMetadataV2( + location=metadata_location, + last_column_id=1, + format_version=2, + current_schema_id=1, + schemas=[table_schema_simple], + partition_specs=[PartitionSpec()], + ), + metadata_location=metadata_location, + io=load_file_io(), + catalog=NoopCatalog("noop"), + ), + row_filter=AlwaysTrue(), + projected_schema=table_schema_simple, + ) + + assert ( + str(with_deletes) + == """pyarrow.Table +foo: string +bar: int64 not null +baz: bool +---- +foo: [["a","c"]] +bar: [[1,3]] +baz: [[true,null]]""" + ) + + +def test_delete_duplicates(deletes_file: str, example_task: FileScanTask, table_schema_simple: Schema) -> None: + metadata_location = "file://a/b/c.json" + example_task_with_delete = FileScanTask( + data_file=example_task.file, + delete_files={ + DataFile(content=DataFileContent.POSITION_DELETES, file_path=deletes_file, file_format=FileFormat.PARQUET), + DataFile(content=DataFileContent.POSITION_DELETES, file_path=deletes_file, file_format=FileFormat.PARQUET), + }, + ) + + with_deletes = project_table( + tasks=[example_task_with_delete], + table=Table( + ("namespace", "table"), + metadata=TableMetadataV2( + location=metadata_location, + last_column_id=1, + format_version=2, + current_schema_id=1, + schemas=[table_schema_simple], + partition_specs=[PartitionSpec()], + ), + metadata_location=metadata_location, + io=load_file_io(), + catalog=NoopCatalog("noop"), + ), + row_filter=AlwaysTrue(), + projected_schema=table_schema_simple, + ) + + assert ( + str(with_deletes) + == """pyarrow.Table +foo: string +bar: int64 not null +baz: bool +---- +foo: [["a","c"]] +bar: [[1,3]] +baz: [[true,null]]""" + ) + + def test_pyarrow_wrap_fsspec(example_task: FileScanTask, table_schema_simple: Schema) -> None: metadata_location = "file://a/b/c.json" projection = project_table( @@ -1202,6 +1327,7 @@ def test_pyarrow_wrap_fsspec(example_task: FileScanTask, table_schema_simple: Sc ), metadata_location=metadata_location, io=load_file_io(properties={"py-io-impl": "pyiceberg.io.fsspec.FsspecFileIO"}, location=metadata_location), + catalog=NoopCatalog("NoopCatalog"), ), case_sensitive=True, projected_schema=table_schema_simple, diff --git a/python/tests/table/test_init.py b/python/tests/table/test_init.py index bf4fafdc3005..87aa0d381563 100644 --- a/python/tests/table/test_init.py +++ b/python/tests/table/test_init.py @@ -18,7 +18,9 @@ from typing import Any, Dict import pytest +from sortedcontainers import SortedList +from pyiceberg.catalog.noop import NoopCatalog from pyiceberg.expressions import ( AlwaysTrue, And, @@ -26,11 +28,17 @@ In, ) from pyiceberg.io import PY_IO_IMPL, load_file_io -from pyiceberg.manifest import DataFile, ManifestContent +from pyiceberg.manifest import ( + DataFile, + DataFileContent, + FileFormat, + ManifestEntry, + ManifestEntryStatus, +) from pyiceberg.partitioning import PartitionField, PartitionSpec from pyiceberg.schema import Schema -from pyiceberg.table import StaticTable, Table, _check_content -from pyiceberg.table.metadata import TableMetadataV2 +from pyiceberg.table import StaticTable, Table, _match_deletes_to_datafile +from pyiceberg.table.metadata import INITIAL_SEQUENCE_NUMBER, TableMetadataV2 from pyiceberg.table.snapshots import ( Operation, Snapshot, @@ -44,7 +52,6 @@ SortOrder, ) from pyiceberg.transforms import BucketTransform, IdentityTransform -from pyiceberg.typedef import Record from pyiceberg.types import LongType, NestedField @@ -56,14 +63,10 @@ def table(example_table_metadata_v2: Dict[str, Any]) -> Table: metadata=table_metadata, metadata_location=f"{table_metadata.location}/uuid.metadata.json", io=load_file_io(), + catalog=NoopCatalog("NoopCatalog"), ) -@pytest.fixture -def static_table(metadata_location: str) -> StaticTable: - return StaticTable.from_metadata(metadata_location) - - def test_schema(table: Table) -> None: assert table.schema() == Schema( NestedField(field_id=1, name="x", field_type=LongType(), required=True), @@ -181,8 +184,8 @@ def test_snapshot_by_name_does_not_exist(table: Table) -> None: def test_history(table: Table) -> None: assert table.history() == [ - SnapshotLogEntry(snapshot_id="3051729675574597004", timestamp_ms=1515100955770), - SnapshotLogEntry(snapshot_id="3055729675574597004", timestamp_ms=1555100955770), + SnapshotLogEntry(snapshot_id=3051729675574597004, timestamp_ms=1515100955770), + SnapshotLogEntry(snapshot_id=3055729675574597004, timestamp_ms=1555100955770), ] @@ -252,27 +255,14 @@ def test_table_scan_projection_unknown_column(table: Table) -> None: assert "Could not find column: 'a'" in str(exc_info.value) -def test_check_content_deletes() -> None: - with pytest.raises(ValueError) as exc_info: - _check_content( - DataFile( - content=ManifestContent.DELETES, - ) - ) - assert "PyIceberg does not support deletes: https://github.com/apache/iceberg/issues/6568" in str(exc_info.value) - - -def test_check_content_data() -> None: - manifest_file = DataFile(content=ManifestContent.DATA) - assert _check_content(manifest_file) == manifest_file - - -def test_check_content_missing_attr() -> None: - r = Record(*([None] * 15)) - assert _check_content(r) == r # type: ignore +def test_static_table_same_as_table(table: Table, metadata_location: str) -> None: + static_table = StaticTable.from_metadata(metadata_location) + assert isinstance(static_table, Table) + assert static_table.metadata == table.metadata -def test_static_table_same_as_table(table: Table, static_table: StaticTable) -> None: +def test_static_table_gz_same_as_table(table: Table, metadata_location_gz: str) -> None: + static_table = StaticTable.from_metadata(metadata_location_gz) assert isinstance(static_table, Table) assert static_table.metadata == table.metadata @@ -280,3 +270,112 @@ def test_static_table_same_as_table(table: Table, static_table: StaticTable) -> def test_static_table_io_does_not_exist(metadata_location: str) -> None: with pytest.raises(ValueError): StaticTable.from_metadata(metadata_location, {PY_IO_IMPL: "pyiceberg.does.not.exist.FileIO"}) + + +def test_match_deletes_to_datafile() -> None: + data_entry = ManifestEntry( + status=ManifestEntryStatus.ADDED, + sequence_number=1, + data_file=DataFile( + content=DataFileContent.DATA, + file_path="s3://bucket/0000.parquet", + file_format=FileFormat.PARQUET, + partition={}, + record_count=3, + file_size_in_bytes=3, + ), + ) + delete_entry_1 = ManifestEntry( + status=ManifestEntryStatus.ADDED, + sequence_number=0, # Older than the data + data_file=DataFile( + content=DataFileContent.POSITION_DELETES, + file_path="s3://bucket/0001-delete.parquet", + file_format=FileFormat.PARQUET, + partition={}, + record_count=3, + file_size_in_bytes=3, + ), + ) + delete_entry_2 = ManifestEntry( + status=ManifestEntryStatus.ADDED, + sequence_number=3, + data_file=DataFile( + content=DataFileContent.POSITION_DELETES, + file_path="s3://bucket/0002-delete.parquet", + file_format=FileFormat.PARQUET, + partition={}, + record_count=3, + file_size_in_bytes=3, + # We don't really care about the tests here + value_counts={}, + null_value_counts={}, + nan_value_counts={}, + lower_bounds={}, + upper_bounds={}, + ), + ) + assert _match_deletes_to_datafile( + data_entry, + SortedList(iterable=[delete_entry_1, delete_entry_2], key=lambda entry: entry.sequence_number or INITIAL_SEQUENCE_NUMBER), + ) == { + delete_entry_2.data_file, + } + + +def test_match_deletes_to_datafile_duplicate_number() -> None: + data_entry = ManifestEntry( + status=ManifestEntryStatus.ADDED, + sequence_number=1, + data_file=DataFile( + content=DataFileContent.DATA, + file_path="s3://bucket/0000.parquet", + file_format=FileFormat.PARQUET, + partition={}, + record_count=3, + file_size_in_bytes=3, + ), + ) + delete_entry_1 = ManifestEntry( + status=ManifestEntryStatus.ADDED, + sequence_number=3, + data_file=DataFile( + content=DataFileContent.POSITION_DELETES, + file_path="s3://bucket/0001-delete.parquet", + file_format=FileFormat.PARQUET, + partition={}, + record_count=3, + file_size_in_bytes=3, + # We don't really care about the tests here + value_counts={}, + null_value_counts={}, + nan_value_counts={}, + lower_bounds={}, + upper_bounds={}, + ), + ) + delete_entry_2 = ManifestEntry( + status=ManifestEntryStatus.ADDED, + sequence_number=3, + data_file=DataFile( + content=DataFileContent.POSITION_DELETES, + file_path="s3://bucket/0002-delete.parquet", + file_format=FileFormat.PARQUET, + partition={}, + record_count=3, + file_size_in_bytes=3, + # We don't really care about the tests here + value_counts={}, + null_value_counts={}, + nan_value_counts={}, + lower_bounds={}, + upper_bounds={}, + ), + ) + assert _match_deletes_to_datafile( + data_entry, + SortedList(iterable=[delete_entry_1, delete_entry_2], key=lambda entry: entry.sequence_number or INITIAL_SEQUENCE_NUMBER), + ) == { + delete_entry_1.data_file, + delete_entry_2.data_file, + } diff --git a/python/tests/table/test_metadata.py b/python/tests/table/test_metadata.py index 8c464c0e5a67..63ef0d07ee8e 100644 --- a/python/tests/table/test_metadata.py +++ b/python/tests/table/test_metadata.py @@ -187,7 +187,7 @@ def test_serialize_v1(example_table_metadata_v1: Dict[str, Any]) -> None: def test_serialize_v2(example_table_metadata_v2: Dict[str, Any]) -> None: table_metadata = TableMetadataV2(**example_table_metadata_v2).json() - expected = """{"location": "s3://bucket/test/location", "table-uuid": "9c12d441-03fe-4693-9a96-a0705ddf69c1", "last-updated-ms": 1602638573590, "last-column-id": 3, "schemas": [{"type": "struct", "fields": [{"id": 1, "name": "x", "type": "long", "required": true}], "schema-id": 0, "identifier-field-ids": []}, {"type": "struct", "fields": [{"id": 1, "name": "x", "type": "long", "required": true}, {"id": 2, "name": "y", "type": "long", "required": true, "doc": "comment"}, {"id": 3, "name": "z", "type": "long", "required": true}], "schema-id": 1, "identifier-field-ids": [1, 2]}], "current-schema-id": 1, "partition-specs": [{"spec-id": 0, "fields": [{"source-id": 1, "field-id": 1000, "transform": "identity", "name": "x"}]}], "default-spec-id": 0, "last-partition-id": 1000, "properties": {"read.split.target.size": "134217728"}, "current-snapshot-id": 3055729675574597004, "snapshots": [{"snapshot-id": 3051729675574597004, "sequence-number": 0, "timestamp-ms": 1515100955770, "manifest-list": "s3://a/b/1.avro", "summary": {"operation": "append"}}, {"snapshot-id": 3055729675574597004, "parent-snapshot-id": 3051729675574597004, "sequence-number": 1, "timestamp-ms": 1555100955770, "manifest-list": "s3://a/b/2.avro", "summary": {"operation": "append"}, "schema-id": 1}], "snapshot-log": [{"snapshot-id": "3051729675574597004", "timestamp-ms": 1515100955770}, {"snapshot-id": "3055729675574597004", "timestamp-ms": 1555100955770}], "metadata-log": [{"metadata-file": "s3://bucket/.../v1.json", "timestamp-ms": 1515100}], "sort-orders": [{"order-id": 3, "fields": [{"source-id": 2, "transform": "identity", "direction": "asc", "null-order": "nulls-first"}, {"source-id": 3, "transform": "bucket[4]", "direction": "desc", "null-order": "nulls-last"}]}], "default-sort-order-id": 3, "refs": {"test": {"snapshot-id": 3051729675574597004, "type": "tag", "max-ref-age-ms": 10000000}, "main": {"snapshot-id": 3055729675574597004, "type": "branch"}}, "format-version": 2, "last-sequence-number": 34}""" + expected = """{"location": "s3://bucket/test/location", "table-uuid": "9c12d441-03fe-4693-9a96-a0705ddf69c1", "last-updated-ms": 1602638573590, "last-column-id": 3, "schemas": [{"type": "struct", "fields": [{"id": 1, "name": "x", "type": "long", "required": true}], "schema-id": 0, "identifier-field-ids": []}, {"type": "struct", "fields": [{"id": 1, "name": "x", "type": "long", "required": true}, {"id": 2, "name": "y", "type": "long", "required": true, "doc": "comment"}, {"id": 3, "name": "z", "type": "long", "required": true}], "schema-id": 1, "identifier-field-ids": [1, 2]}], "current-schema-id": 1, "partition-specs": [{"spec-id": 0, "fields": [{"source-id": 1, "field-id": 1000, "transform": "identity", "name": "x"}]}], "default-spec-id": 0, "last-partition-id": 1000, "properties": {"read.split.target.size": "134217728"}, "current-snapshot-id": 3055729675574597004, "snapshots": [{"snapshot-id": 3051729675574597004, "sequence-number": 0, "timestamp-ms": 1515100955770, "manifest-list": "s3://a/b/1.avro", "summary": {"operation": "append"}}, {"snapshot-id": 3055729675574597004, "parent-snapshot-id": 3051729675574597004, "sequence-number": 1, "timestamp-ms": 1555100955770, "manifest-list": "s3://a/b/2.avro", "summary": {"operation": "append"}, "schema-id": 1}], "snapshot-log": [{"snapshot-id": 3051729675574597004, "timestamp-ms": 1515100955770}, {"snapshot-id": 3055729675574597004, "timestamp-ms": 1555100955770}], "metadata-log": [{"metadata-file": "s3://bucket/.../v1.json", "timestamp-ms": 1515100}], "sort-orders": [{"order-id": 3, "fields": [{"source-id": 2, "transform": "identity", "direction": "asc", "null-order": "nulls-first"}, {"source-id": 3, "transform": "bucket[4]", "direction": "desc", "null-order": "nulls-last"}]}], "default-sort-order-id": 3, "refs": {"test": {"snapshot-id": 3051729675574597004, "type": "tag", "max-ref-age-ms": 10000000}, "main": {"snapshot-id": 3055729675574597004, "type": "branch"}}, "format-version": 2, "last-sequence-number": 34}""" assert table_metadata == expected diff --git a/python/tests/test_integration.py b/python/tests/test_integration.py index 5577f3d84a2b..36099208190b 100644 --- a/python/tests/test_integration.py +++ b/python/tests/test_integration.py @@ -24,10 +24,24 @@ from pyarrow.fs import S3FileSystem from pyiceberg.catalog import Catalog, load_catalog -from pyiceberg.expressions import IsNaN, NotNaN +from pyiceberg.exceptions import NoSuchTableError +from pyiceberg.expressions import ( + And, + GreaterThanOrEqual, + IsNaN, + LessThan, + NotNaN, +) from pyiceberg.io.pyarrow import pyarrow_to_schema from pyiceberg.schema import Schema from pyiceberg.table import Table +from pyiceberg.types import ( + BooleanType, + IntegerType, + NestedField, + StringType, + TimestampType, +) @pytest.fixture() @@ -64,6 +78,62 @@ def table_test_all_types(catalog: Catalog) -> Table: return catalog.load_table("default.test_all_types") +TABLE_NAME = ("default", "t1") + + +@pytest.fixture() +def table(catalog: Catalog) -> Table: + try: + catalog.drop_table(TABLE_NAME) + except NoSuchTableError: + pass # Just to make sure that the table doesn't exist + + schema = Schema( + NestedField(field_id=1, name="str", field_type=StringType(), required=False), + NestedField(field_id=2, name="int", field_type=IntegerType(), required=True), + NestedField(field_id=3, name="bool", field_type=BooleanType(), required=False), + NestedField(field_id=4, name="datetime", field_type=TimestampType(), required=False), + schema_id=1, + ) + + return catalog.create_table(identifier=TABLE_NAME, schema=schema) + + +@pytest.mark.integration +def test_table_properties(table: Table) -> None: + assert table.properties == {} + + with table.transaction() as transaction: + transaction.set_properties(abc="🤪") + + assert table.properties == {"abc": "🤪"} + + with table.transaction() as transaction: + transaction.remove_properties("abc") + + assert table.properties == {} + + table = table.transaction().set_properties(abc="def").commit_transaction() + + assert table.properties == {"abc": "def"} + + table = table.transaction().remove_properties("abc").commit_transaction() + + assert table.properties == {} + + +@pytest.fixture() +def test_positional_mor_deletes(catalog: Catalog) -> Table: + """Table that has positional deletes""" + return catalog.load_table("default.test_positional_mor_deletes") + + +@pytest.fixture() +def test_positional_mor_double_deletes(catalog: Catalog) -> Table: + """Table that has multiple positional deletes""" + return catalog.load_table("default.test_positional_mor_double_deletes") + + @pytest.mark.integration def test_pyarrow_nan(table_test_null_nan: Table) -> None: arrow_table = table_test_null_nan.scan(row_filter=IsNaN("col_numeric"), selected_fields=("idx", "col_numeric")).to_arrow() @@ -159,3 +229,73 @@ def test_pyarrow_to_iceberg_all_types(table_test_all_types: Table) -> None: stored_iceberg_schema = Schema.parse_raw(parquet_schema.metadata.get(b"iceberg.schema")) converted_iceberg_schema = pyarrow_to_schema(parquet_schema) assert converted_iceberg_schema == stored_iceberg_schema + + +@pytest.mark.integration +def test_pyarrow_deletes(test_positional_mor_deletes: Table) -> None: + # number, letter + # (1, 'a'), + # (2, 'b'), + # (3, 'c'), + # (4, 'd'), + # (5, 'e'), + # (6, 'f'), + # (7, 'g'), + # (8, 'h'), + # (9, 'i'), <- deleted + # (10, 'j'), + # (11, 'k'), + # (12, 'l') + arrow_table = test_positional_mor_deletes.scan().to_arrow() + assert arrow_table["number"].to_pylist() == [1, 2, 3, 4, 5, 6, 7, 8, 10, 11, 12] + + # Checking the filter + arrow_table = test_positional_mor_deletes.scan( + row_filter=And(GreaterThanOrEqual("letter", "e"), LessThan("letter", "k")) + ).to_arrow() + assert arrow_table["number"].to_pylist() == [5, 6, 7, 8, 10] + + # Testing the combination of a filter and a limit + arrow_table = test_positional_mor_deletes.scan( + row_filter=And(GreaterThanOrEqual("letter", "e"), LessThan("letter", "k")), limit=1 + ).to_arrow() + assert arrow_table["number"].to_pylist() == [5] + + # Testing the slicing of indices + arrow_table = test_positional_mor_deletes.scan(limit=3).to_arrow() + assert arrow_table["number"].to_pylist() == [1, 2, 3] + + +@pytest.mark.integration +def test_pyarrow_deletes_double(test_positional_mor_double_deletes: Table) -> None: + # number, letter + # (1, 'a'), + # (2, 'b'), + # (3, 'c'), + # (4, 'd'), + # (5, 'e'), + # (6, 'f'), <- second delete + # (7, 'g'), + # (8, 'h'), + # (9, 'i'), <- first delete + # (10, 'j'), + # (11, 'k'), + # (12, 'l') + arrow_table = test_positional_mor_double_deletes.scan().to_arrow() + assert arrow_table["number"].to_pylist() == [1, 2, 3, 4, 5, 7, 8, 10, 11, 12] + + # Checking the filter + arrow_table = test_positional_mor_double_deletes.scan( + row_filter=And(GreaterThanOrEqual("letter", "e"), LessThan("letter", "k")) + ).to_arrow() + assert arrow_table["number"].to_pylist() == [5, 7, 8, 10] + + # Testing the combination of a filter and a limit + arrow_table = test_positional_mor_double_deletes.scan( + row_filter=And(GreaterThanOrEqual("letter", "e"), LessThan("letter", "k")), limit=1 + ).to_arrow() + assert arrow_table["number"].to_pylist() == [5] + + # Testing the slicing of indices + arrow_table = test_positional_mor_double_deletes.scan(limit=8).to_arrow() + assert arrow_table["number"].to_pylist() == [1, 2, 3, 4, 5, 7, 8, 10] diff --git a/python/tests/test_types.py b/python/tests/test_types.py index e2c0272b454e..dbe54688c73c 100644 --- a/python/tests/test_types.py +++ b/python/tests/test_types.py @@ -15,6 +15,7 @@ # specific language governing permissions and limitations # under the License. # pylint: disable=W0123,W0613 +import pickle from typing import Type import pytest @@ -63,6 +64,7 @@ @pytest.mark.parametrize("input_index, input_type", non_parameterized_types) def test_repr_primitive_types(input_index: int, input_type: Type[PrimitiveType]) -> None: assert isinstance(eval(repr(input_type())), input_type) + assert input_type == pickle.loads(pickle.dumps(input_type)) @pytest.mark.parametrize( @@ -109,6 +111,7 @@ def test_fixed_type() -> None: assert str(type_var) == str(eval(repr(type_var))) assert type_var == FixedType(5) assert type_var != FixedType(6) + assert type_var == pickle.loads(pickle.dumps(type_var)) def test_decimal_type() -> None: @@ -120,6 +123,7 @@ def test_decimal_type() -> None: assert str(type_var) == str(eval(repr(type_var))) assert type_var == DecimalType(9, 2) assert type_var != DecimalType(9, 3) + assert type_var == pickle.loads(pickle.dumps(type_var)) def test_struct_type() -> None: @@ -140,6 +144,7 @@ def test_struct_type() -> None: assert str(type_var) == str(eval(repr(type_var))) assert type_var == eval(repr(type_var)) assert type_var != StructType(NestedField(1, "optional_field", IntegerType(), required=True)) + assert type_var == pickle.loads(pickle.dumps(type_var)) def test_list_type() -> None: @@ -163,6 +168,7 @@ def test_list_type() -> None: ), True, ) + assert type_var == pickle.loads(pickle.dumps(type_var)) def test_map_type() -> None: @@ -175,6 +181,7 @@ def test_map_type() -> None: assert type_var == eval(repr(type_var)) assert type_var != MapType(1, LongType(), 2, UUIDType(), False) assert type_var != MapType(1, DoubleType(), 2, StringType(), True) + assert type_var == pickle.loads(pickle.dumps(type_var)) def test_nested_field() -> None: @@ -200,6 +207,7 @@ def test_nested_field() -> None: assert field_var.field_id == 1 assert isinstance(field_var.field_type, StructType) assert str(field_var) == str(eval(repr(field_var))) + assert field_var == pickle.loads(pickle.dumps(field_var)) @pytest.mark.parametrize("input_index,input_type", non_parameterized_types) diff --git a/python/tests/utils/test_datetime.py b/python/tests/utils/test_datetime.py new file mode 100644 index 000000000000..46743399d1e7 --- /dev/null +++ b/python/tests/utils/test_datetime.py @@ -0,0 +1,69 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +from datetime import datetime, timezone, tzinfo + +import pytest +import pytz + +from pyiceberg.utils.datetime import datetime_to_millis + +timezones = [ + pytz.timezone("Etc/GMT"), + pytz.timezone("Etc/GMT+0"), + pytz.timezone("Etc/GMT+1"), + pytz.timezone("Etc/GMT+10"), + pytz.timezone("Etc/GMT+11"), + pytz.timezone("Etc/GMT+12"), + pytz.timezone("Etc/GMT+2"), + pytz.timezone("Etc/GMT+3"), + pytz.timezone("Etc/GMT+4"), + pytz.timezone("Etc/GMT+5"), + pytz.timezone("Etc/GMT+6"), + pytz.timezone("Etc/GMT+7"), + pytz.timezone("Etc/GMT+8"), + pytz.timezone("Etc/GMT+9"), + pytz.timezone("Etc/GMT-0"), + pytz.timezone("Etc/GMT-1"), + pytz.timezone("Etc/GMT-10"), + pytz.timezone("Etc/GMT-11"), + pytz.timezone("Etc/GMT-12"), + pytz.timezone("Etc/GMT-13"), + pytz.timezone("Etc/GMT-14"), + pytz.timezone("Etc/GMT-2"), + pytz.timezone("Etc/GMT-3"), + pytz.timezone("Etc/GMT-4"), + pytz.timezone("Etc/GMT-5"), + pytz.timezone("Etc/GMT-6"), + pytz.timezone("Etc/GMT-7"), + pytz.timezone("Etc/GMT-8"), + pytz.timezone("Etc/GMT-9"), +] + + +def test_datetime_to_millis() -> None: + dt = datetime(2023, 7, 10, 10, 10, 10, 123456) + expected = int(dt.replace(tzinfo=timezone.utc).timestamp() * 1_000) + datetime_millis = datetime_to_millis(dt) + assert datetime_millis == expected + + +@pytest.mark.parametrize("tz", timezones) +def test_datetime_tz_to_millis(tz: tzinfo) -> None: + dt = datetime(2023, 7, 10, 10, 10, 10, 123456, tzinfo=tz) + expected = int(dt.timestamp() * 1_000) + datetime_millis = datetime_to_millis(dt) + assert datetime_millis == expected diff --git a/python/tests/utils/test_manifest.py b/python/tests/utils/test_manifest.py index 29905ce55dcf..76a4a8a2b4d9 100644 --- a/python/tests/utils/test_manifest.py +++ b/python/tests/utils/test_manifest.py @@ -19,11 +19,12 @@ from pyiceberg.io.pyarrow import PyArrowFileIO from pyiceberg.manifest import ( DataFile, + DataFileContent, FileFormat, ManifestContent, ManifestEntryStatus, + ManifestFile, PartitionFieldSummary, - read_manifest_entry, read_manifest_list, ) from pyiceberg.table import Snapshot @@ -31,18 +32,20 @@ def test_read_manifest_entry(generated_manifest_entry_file: str) -> None: - input_file = PyArrowFileIO().new_input(location=generated_manifest_entry_file) - manifest_entries = list(read_manifest_entry(input_file)) + manifest = ManifestFile( + manifest_path=generated_manifest_entry_file, manifest_length=0, partition_spec_id=0, sequence_number=None, partitions=[] + ) + manifest_entries = manifest.fetch_manifest_entry(PyArrowFileIO()) manifest_entry = manifest_entries[0] assert manifest_entry.status == ManifestEntryStatus.ADDED assert manifest_entry.snapshot_id == 8744736658442914487 - assert manifest_entry.sequence_number is None + assert manifest_entry.data_sequence_number is None assert isinstance(manifest_entry.data_file, DataFile) data_file = manifest_entry.data_file - assert data_file.content is None + assert data_file.content is DataFileContent.DATA assert ( data_file.file_path == "/home/iceberg/warehouse/nyc/taxis_partitioned/data/VendorID=null/00000-633-d8a4223e-dc97-45a1-86e1-adaba6e8abd7-00001.parquet" @@ -215,6 +218,17 @@ def test_read_manifest_v1(generated_manifest_file_file_v1: str) -> None: assert partition.lower_bound == b"\x01\x00\x00\x00" assert partition.upper_bound == b"\x02\x00\x00\x00" + entries = manifest_list.fetch_manifest_entry(io) + + assert isinstance(entries, list) + + entry = entries[0] + + assert entry.data_sequence_number == 0 + assert entry.file_sequence_number == 0 + assert entry.snapshot_id == 8744736658442914487 + assert entry.status == ManifestEntryStatus.ADDED + def test_read_manifest_v2(generated_manifest_file_file_v2: str) -> None: io = load_file_io() @@ -232,8 +246,8 @@ def test_read_manifest_v2(generated_manifest_file_file_v2: str) -> None: assert manifest_list.manifest_length == 7989 assert manifest_list.partition_spec_id == 0 assert manifest_list.content == ManifestContent.DELETES - assert manifest_list.sequence_number is None # inheritance - assert manifest_list.min_sequence_number is None # inheritance + assert manifest_list.sequence_number == 3 + assert manifest_list.min_sequence_number == 3 assert manifest_list.added_snapshot_id == 9182715666859759686 assert manifest_list.added_files_count == 3 assert manifest_list.existing_files_count == 0 @@ -253,3 +267,14 @@ def test_read_manifest_v2(generated_manifest_file_file_v2: str) -> None: assert partition.contains_nan is False assert partition.lower_bound == b"\x01\x00\x00\x00" assert partition.upper_bound == b"\x02\x00\x00\x00" + + entries = manifest_list.fetch_manifest_entry(io) + + assert isinstance(entries, list) + + entry = entries[0] + + assert entry.data_sequence_number == 3 + assert entry.file_sequence_number == 3 + assert entry.snapshot_id == 8744736658442914487 + assert entry.status == ManifestEntryStatus.ADDED diff --git a/python/tests/utils/test_schema_conversion.py b/python/tests/utils/test_schema_conversion.py index 6a8c5a28c75a..2c42c445e432 100644 --- a/python/tests/utils/test_schema_conversion.py +++ b/python/tests/utils/test_schema_conversion.py @@ -37,7 +37,7 @@ from pyiceberg.utils.schema_conversion import AvroSchemaConversion -def test_iceberg_to_avro(avro_schema_manifest_file_v1: Dict[str, Any]) -> None: +def test_avro_to_iceberg(avro_schema_manifest_file_v1: Dict[str, Any]) -> None: iceberg_schema = AvroSchemaConversion().avro_to_iceberg(avro_schema_manifest_file_v1) expected_iceberg_schema = Schema( NestedField( @@ -354,3 +354,17 @@ def test_logical_map_with_invalid_fields() -> None: AvroSchemaConversion()._convert_logical_map_type(avro_type) assert "Invalid key-value pair schema:" in str(exc_info.value) + + +def test_iceberg_to_avro_manifest_list(avro_schema_manifest_file_v1: Dict[str, Any]) -> None: + """Round trip the manifest list""" + iceberg_schema = AvroSchemaConversion().avro_to_iceberg(avro_schema_manifest_file_v1) + avro_result = AvroSchemaConversion().iceberg_to_avro(iceberg_schema, schema_name="manifest_file") + assert avro_schema_manifest_file_v1 == avro_result + + +def test_iceberg_to_avro_manifest(avro_schema_manifest_entry: Dict[str, Any]) -> None: + """Round trip the manifest itself""" + iceberg_schema = AvroSchemaConversion().avro_to_iceberg(avro_schema_manifest_entry) + avro_result = AvroSchemaConversion().iceberg_to_avro(iceberg_schema, schema_name="manifest_entry") + assert avro_schema_manifest_entry == avro_result diff --git a/snowflake/src/test/java/org/apache/iceberg/snowflake/JdbcSnowflakeClientTest.java b/snowflake/src/test/java/org/apache/iceberg/snowflake/JdbcSnowflakeClientTest.java index 4fd0ae0e21ed..6da321facc13 100644 --- a/snowflake/src/test/java/org/apache/iceberg/snowflake/JdbcSnowflakeClientTest.java +++ b/snowflake/src/test/java/org/apache/iceberg/snowflake/JdbcSnowflakeClientTest.java @@ -38,14 +38,17 @@ import org.apache.iceberg.jdbc.UncheckedInterruptedException; import org.apache.iceberg.jdbc.UncheckedSQLException; import org.assertj.core.api.Assertions; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.ArgumentMatchers; import org.mockito.Mock; -import org.mockito.junit.MockitoJUnitRunner; +import org.mockito.junit.jupiter.MockitoExtension; +import org.mockito.junit.jupiter.MockitoSettings; +import org.mockito.quality.Strictness; -@RunWith(MockitoJUnitRunner.class) +@ExtendWith(MockitoExtension.class) +@MockitoSettings(strictness = Strictness.LENIENT) public class JdbcSnowflakeClientTest { @Mock private Connection mockConnection; @Mock private JdbcClientPool mockClientPool; @@ -54,11 +57,10 @@ public class JdbcSnowflakeClientTest { private JdbcSnowflakeClient snowflakeClient; - @Before + @BeforeEach public void before() throws SQLException, InterruptedException { snowflakeClient = new JdbcSnowflakeClient(mockClientPool); snowflakeClient.setQueryHarness(mockQueryHarness); - doAnswer(invocation -> ((ClientPool.Action) invocation.getArguments()[0]).run(mockConnection)) .when(mockClientPool) .run(any(ClientPool.Action.class)); diff --git a/snowflake/src/test/java/org/apache/iceberg/snowflake/NamespaceHelpersTest.java b/snowflake/src/test/java/org/apache/iceberg/snowflake/NamespaceHelpersTest.java index 2dd7fb6ec9af..ef47f5b784fa 100644 --- a/snowflake/src/test/java/org/apache/iceberg/snowflake/NamespaceHelpersTest.java +++ b/snowflake/src/test/java/org/apache/iceberg/snowflake/NamespaceHelpersTest.java @@ -21,7 +21,7 @@ import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class NamespaceHelpersTest { @Test diff --git a/snowflake/src/test/java/org/apache/iceberg/snowflake/SnowflakeCatalogTest.java b/snowflake/src/test/java/org/apache/iceberg/snowflake/SnowflakeCatalogTest.java index 416f64a91e45..adda8bc65f20 100644 --- a/snowflake/src/test/java/org/apache/iceberg/snowflake/SnowflakeCatalogTest.java +++ b/snowflake/src/test/java/org/apache/iceberg/snowflake/SnowflakeCatalogTest.java @@ -33,8 +33,8 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; import org.assertj.core.api.Assertions; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; public class SnowflakeCatalogTest { @@ -45,7 +45,7 @@ public class SnowflakeCatalogTest { private SnowflakeCatalog.FileIOFactory fakeFileIOFactory; private Map properties; - @Before + @BeforeEach public void before() { catalog = new SnowflakeCatalog(); diff --git a/spark/v3.1/build.gradle b/spark/v3.1/build.gradle index bfb73ab9f230..f60ca6b8e249 100644 --- a/spark/v3.1/build.gradle +++ b/spark/v3.1/build.gradle @@ -67,6 +67,9 @@ project(':iceberg-spark:iceberg-spark-3.1_2.12') { exclude group: 'org.apache.avro', module: 'avro' exclude group: 'org.apache.arrow' exclude group: 'org.apache.parquet' + // to make sure netty libs only come from project(':iceberg-arrow') + exclude group: 'io.netty', module: 'netty-buffer' + exclude group: 'io.netty', module: 'netty-common' exclude group: 'org.roaringbitmap' } @@ -89,6 +92,9 @@ project(':iceberg-spark:iceberg-spark-3.1_2.12') { testImplementation("org.apache.hadoop:hadoop-minicluster") { exclude group: 'org.apache.avro', module: 'avro' + // to make sure netty libs only come from project(':iceberg-arrow') + exclude group: 'io.netty', module: 'netty-buffer' + exclude group: 'io.netty', module: 'netty-common' } testImplementation project(path: ':iceberg-hive-metastore') testImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') @@ -143,6 +149,9 @@ project(":iceberg-spark:iceberg-spark-extensions-3.1_2.12") { exclude group: 'org.apache.avro', module: 'avro' exclude group: 'org.apache.arrow' exclude group: 'org.apache.parquet' + // to make sure netty libs only come from project(':iceberg-arrow') + exclude group: 'io.netty', module: 'netty-buffer' + exclude group: 'io.netty', module: 'netty-common' exclude group: 'org.roaringbitmap' } @@ -220,6 +229,7 @@ project(':iceberg-spark:iceberg-spark-runtime-3.1_2.12') { integrationImplementation "org.apache.spark:spark-hive_2.12:${sparkVersion}" integrationImplementation 'org.junit.vintage:junit-vintage-engine' integrationImplementation 'org.slf4j:slf4j-simple' + integrationImplementation 'org.assertj:assertj-core' integrationImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') integrationImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') integrationImplementation project(path: ':iceberg-spark:iceberg-spark-3.1_2.12', configuration: 'testArtifacts') @@ -255,7 +265,7 @@ project(':iceberg-spark:iceberg-spark-runtime-3.1_2.12') { relocate 'io.airlift', 'org.apache.iceberg.shaded.io.airlift' relocate 'org.apache.httpcomponents.client5', 'org.apache.iceberg.shaded.org.apache.httpcomponents.client5' // relocate Arrow and related deps to shade Iceberg specific version - relocate 'io.netty.buffer', 'org.apache.iceberg.shaded.io.netty.buffer' + relocate 'io.netty', 'org.apache.iceberg.shaded.io.netty' relocate 'org.apache.arrow', 'org.apache.iceberg.shaded.org.apache.arrow' relocate 'com.carrotsearch', 'org.apache.iceberg.shaded.com.carrotsearch' relocate 'org.threeten.extra', 'org.apache.iceberg.shaded.org.threeten.extra' @@ -267,6 +277,7 @@ project(':iceberg-spark:iceberg-spark-runtime-3.1_2.12') { task integrationTest(type: Test) { description = "Test Spark3 Runtime Jar against Spark 3.1" group = "verification" + jvmArgs += project.property('extraJvmArgs') testClassesDirs = sourceSets.integration.output.classesDirs classpath = sourceSets.integration.runtimeClasspath + files(shadowJar.archiveFile.get().asFile.path) inputs.file(shadowJar.archiveFile.get().asFile.path) diff --git a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala index 8507fb1c9f80..e8547f11d345 100644 --- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala +++ b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala @@ -110,6 +110,7 @@ class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergS val branchRetention = branchOptionsContext.flatMap(branchOptions => Option(branchOptions.refRetain())) val branchRefAgeMs = branchRetention.map(retain => TimeUnit.valueOf(retain.timeUnit().getText.toUpperCase(Locale.ENGLISH)).toMillis(retain.number().getText.toLong)) + val create = createOrReplaceBranchClause.CREATE() != null val replace = ctx.createReplaceBranchClause().REPLACE() != null val ifNotExists = createOrReplaceBranchClause.EXISTS() != null @@ -124,6 +125,7 @@ class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergS typedVisit[Seq[String]](ctx.multipartIdentifier), branchName.getText, branchOptions, + create, replace, ifNotExists) } diff --git a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala index f555a9a98a06..06ee4cf1625c 100644 --- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala +++ b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala @@ -25,6 +25,7 @@ case class CreateOrReplaceBranch( table: Seq[String], branch: String, branchOptions: BranchOptions, + create: Boolean, replace: Boolean, ifNotExists: Boolean) extends Command { diff --git a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala index 9b378cf84e25..651b5c62e159 100644 --- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala +++ b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala @@ -32,6 +32,7 @@ case class CreateOrReplaceBranchExec( ident: Identifier, branch: String, branchOptions: BranchOptions, + create: Boolean, replace: Boolean, ifNotExists: Boolean) extends V2CommandExec { @@ -51,15 +52,18 @@ case class CreateOrReplaceBranchExec( "Cannot complete create or replace branch operation on %s, main has no snapshot", ident) val manageSnapshots = iceberg.table().manageSnapshots() - if (!replace) { - val ref = iceberg.table().refs().get(branch) - if (ref != null && ifNotExists) { + val refExists = null != iceberg.table().refs().get(branch) + + if (create && replace && !refExists) { + manageSnapshots.createBranch(branch, snapshotId) + } else if (replace) { + manageSnapshots.replaceBranch(branch, snapshotId) + } else { + if (refExists && ifNotExists) { return Nil } manageSnapshots.createBranch(branch, snapshotId) - } else { - manageSnapshots.replaceBranch(branch, snapshotId) } if (branchOptions.numSnapshots.nonEmpty) { diff --git a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala index fdf8cb56d869..a86c84c6416c 100644 --- a/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala +++ b/spark/v3.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala @@ -71,8 +71,8 @@ case class ExtendedDataSourceV2Strategy(spark: SparkSession) extends Strategy { ReplacePartitionFieldExec(catalog, ident, transformFrom, transformTo, name) :: Nil case CreateOrReplaceBranch( - IcebergCatalogAndIdentifier(catalog, ident), branch, branchOptions, replace, ifNotExists) => - CreateOrReplaceBranchExec(catalog, ident, branch, branchOptions, replace, ifNotExists) :: Nil + IcebergCatalogAndIdentifier(catalog, ident), branch, branchOptions, create, replace, ifNotExists) => + CreateOrReplaceBranchExec(catalog, ident, branch, branchOptions, create, replace, ifNotExists) :: Nil case DropBranch(IcebergCatalogAndIdentifier(catalog, ident), branch, ifExists) => DropBranchExec(catalog, ident, branch, ifExists) :: Nil diff --git a/spark/v3.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java b/spark/v3.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java index 76e92f624f1a..f23dc58af60b 100644 --- a/spark/v3.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java +++ b/spark/v3.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; @@ -511,6 +513,71 @@ public void testCreateOrReplace() throws NoSuchTableException { Assert.assertEquals(first, ref.snapshotId()); } + @Test + public void createOrReplace() throws NoSuchTableException { + Table table = insertRows(); + long first = table.currentSnapshot().snapshotId(); + String branchName = "b1"; + insertRows(); + long second = table.currentSnapshot().snapshotId(); + table.manageSnapshots().createBranch(branchName, second).commit(); + + sql( + "ALTER TABLE %s CREATE OR REPLACE BRANCH %s AS OF VERSION %d", + tableName, branchName, first); + table.refresh(); + assertThat(table.refs().get(branchName).snapshotId()).isEqualTo(second); + } + + @Test + public void createOrReplaceWithNonExistingBranch() throws NoSuchTableException { + Table table = insertRows(); + String branchName = "b1"; + insertRows(); + long snapshotId = table.currentSnapshot().snapshotId(); + + sql( + "ALTER TABLE %s CREATE OR REPLACE BRANCH %s AS OF VERSION %d", + tableName, branchName, snapshotId); + table.refresh(); + assertThat(table.refs().get(branchName).snapshotId()).isEqualTo(snapshotId); + } + + @Test + public void replaceBranch() throws NoSuchTableException { + Table table = insertRows(); + long first = table.currentSnapshot().snapshotId(); + String branchName = "b1"; + long expectedMaxRefAgeMs = 1000; + table + .manageSnapshots() + .createBranch(branchName, first) + .setMaxRefAgeMs(branchName, expectedMaxRefAgeMs) + .commit(); + + insertRows(); + long second = table.currentSnapshot().snapshotId(); + + sql("ALTER TABLE %s REPLACE BRANCH %s AS OF VERSION %d", tableName, branchName, second); + table.refresh(); + SnapshotRef ref = table.refs().get(branchName); + assertThat(ref.snapshotId()).isEqualTo(second); + assertThat(ref.maxRefAgeMs()).isEqualTo(expectedMaxRefAgeMs); + } + + @Test + public void replaceBranchDoesNotExist() throws NoSuchTableException { + Table table = insertRows(); + + Assertions.assertThatThrownBy( + () -> + sql( + "ALTER TABLE %s REPLACE BRANCH %s AS OF VERSION %d", + tableName, "someBranch", table.currentSnapshot().snapshotId())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Branch does not exist: someBranch"); + } + private Table insertRows() throws NoSuchTableException { List records = ImmutableList.of(new SimpleRecord(1, "a"), new SimpleRecord(2, "b")); diff --git a/spark/v3.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestIcebergExpressions.java b/spark/v3.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestIcebergExpressions.java index 8d2e10ea17eb..b1ba53455123 100644 --- a/spark/v3.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestIcebergExpressions.java +++ b/spark/v3.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestIcebergExpressions.java @@ -24,6 +24,7 @@ import org.apache.spark.sql.Column; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.expressions.IcebergBucketTransform; import org.apache.spark.sql.catalyst.expressions.IcebergTruncateTransform; import org.junit.After; import org.junit.Test; @@ -71,4 +72,34 @@ public void testTruncateExpressions() { ImmutableList.of(row(100, 10000L, new BigDecimal("10.50"), "10", "12")), sql("SELECT int_c, long_c, dec_c, str_c, CAST(binary_c AS STRING) FROM v")); } + + @Test + public void testBucketExpressions() { + sql( + "CREATE TABLE %s ( " + + " int_c INT, long_c LONG, dec_c DECIMAL(4, 2), str_c STRING, binary_c BINARY " + + ") USING iceberg", + tableName); + + sql( + "CREATE TEMPORARY VIEW emp " + + "AS SELECT * FROM VALUES (101, 10001, 10.65, '101-Employee', CAST('1234' AS BINARY)) " + + "AS EMP(int_c, long_c, dec_c, str_c, binary_c)"); + + sql("INSERT INTO %s SELECT * FROM emp", tableName); + + Dataset df = spark.sql("SELECT * FROM " + tableName); + df.select( + new Column(new IcebergBucketTransform(2, df.col("int_c").expr())).as("int_c"), + new Column(new IcebergBucketTransform(3, df.col("long_c").expr())).as("long_c"), + new Column(new IcebergBucketTransform(4, df.col("dec_c").expr())).as("dec_c"), + new Column(new IcebergBucketTransform(5, df.col("str_c").expr())).as("str_c"), + new Column(new IcebergBucketTransform(6, df.col("binary_c").expr())).as("binary_c")) + .createOrReplaceTempView("v"); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(0, 2, 0, 4, 1)), + sql("SELECT int_c, long_c, dec_c, str_c, binary_c FROM v")); + } } diff --git a/spark/v3.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java b/spark/v3.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java index 83126b700eaf..d1adee74f28d 100644 --- a/spark/v3.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java +++ b/spark/v3.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java @@ -184,4 +184,43 @@ public void testInvalidMigrateCases() { "Cannot handle an empty identifier", () -> sql("CALL %s.system.migrate('')", catalogName)); } + + @Test + public void testMigratePartitionWithSpecialCharacter() throws IOException { + Assume.assumeTrue(catalogName.equals("spark_catalog")); + String location = temp.newFolder().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string, dt date) USING parquet " + + "PARTITIONED BY (data, dt) LOCATION '%s'", + tableName, location); + sql("INSERT INTO TABLE %s VALUES (1, '2023/05/30', date '2023-05-30')", tableName); + Object result = scalarSql("CALL %s.system.migrate('%s')", catalogName, tableName); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1L, "2023/05/30", java.sql.Date.valueOf("2023-05-30"))), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @Test + public void testMigrateEmptyPartitionedTable() throws Exception { + Assume.assumeTrue(catalogName.equals("spark_catalog")); + String location = temp.newFolder().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet PARTITIONED BY (id) LOCATION '%s'", + tableName, location); + Object result = scalarSql("CALL %s.system.migrate('%s')", catalogName, tableName); + Assert.assertEquals(0L, result); + } + + @Test + public void testMigrateEmptyTable() throws Exception { + Assume.assumeTrue(catalogName.equals("spark_catalog")); + String location = temp.newFolder().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + tableName, location); + Object result = scalarSql("CALL %s.system.migrate('%s')", catalogName, tableName); + Assert.assertEquals(0L, result); + } } diff --git a/spark/v3.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java b/spark/v3.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java index 7d5b5f88a18c..197e80fbd1f2 100644 --- a/spark/v3.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java +++ b/spark/v3.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java @@ -32,10 +32,16 @@ import java.util.List; import java.util.Map; import java.util.UUID; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.iceberg.AssertHelpers; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; import org.apache.iceberg.Files; import org.apache.iceberg.GenericBlobMetadata; import org.apache.iceberg.GenericStatisticsFile; +import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.StatisticsFile; import org.apache.iceberg.Table; import org.apache.iceberg.Transaction; @@ -49,6 +55,8 @@ import org.apache.iceberg.spark.Spark3Util; import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.parser.ParseException; import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; @@ -462,4 +470,102 @@ private static File tableLocation(Table table) { file.isDirectory(), "Table location '%s' does not point to a directory", location); return file; } + + @Test + public void testRemoveOrphanFilesProcedureWithPrefixMode() + throws NoSuchTableException, ParseException, IOException { + if (catalogName.equals("testhadoop")) { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + } else { + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg LOCATION '%s'", + tableName, temp.newFolder().toURI().toString()); + } + Table table = Spark3Util.loadIcebergTable(spark, tableName); + String location = table.location(); + Path originalPath = new Path(location); + FileSystem localFs = FileSystem.getLocal(new Configuration()); + Path originalDataPath = new Path(originalPath, "data"); + localFs.mkdirs(originalDataPath); + localFs.create(new Path(originalDataPath, "data-a.parquet")).close(); + localFs.create(new Path(originalDataPath, "data-b.parquet")).close(); + + URI uri = originalPath.toUri(); + Path newParentPath = new Path("file1", uri.getAuthority(), uri.getPath()); + + DataFile dataFile1 = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath(new Path(newParentPath, "data/data-a.parquet").toString()) + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + DataFile dataFile2 = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath(new Path(newParentPath, "data/data-b.parquet").toString()) + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + + table.newFastAppend().appendFile(dataFile1).appendFile(dataFile2).commit(); + + Timestamp currentTimestamp = Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis())); + + Assert.assertEquals( + 0, + sql( + "CALL %s.system.remove_orphan_files(" + + "table => '%s'," + + "older_than => TIMESTAMP '%s'," + + "equal_schemes => map('file1', 'file'))", + catalogName, tableIdent, currentTimestamp) + .size()); + + // Test with no equal schemes + AssertHelpers.assertThrows( + "Should complain about removing orphan files", + ValidationException.class, + "Conflicting authorities/schemes: [(file1, file)]", + () -> + sql( + "CALL %s.system.remove_orphan_files(" + + "table => '%s'," + + "older_than => TIMESTAMP '%s')", + catalogName, tableIdent, currentTimestamp)); + + AssertHelpers.assertThrows( + "Should complain about removing orphan files", + ValidationException.class, + "Conflicting authorities/schemes: [(file1, file)]", + () -> + sql( + "CALL %s.system.remove_orphan_files(" + + "table => '%s'," + + "older_than => TIMESTAMP '%s'," + + "prefix_mismatch_mode => 'error')", + catalogName, tableIdent, currentTimestamp)); + + Assert.assertEquals( + 2, + sql( + "CALL %s.system.remove_orphan_files(" + + "table => '%s'," + + "older_than => TIMESTAMP '%s'," + + "prefix_mismatch_mode => 'delete')", + catalogName, tableIdent, currentTimestamp) + .size()); + + Assert.assertEquals( + 0, + sql( + "CALL %s.system.remove_orphan_files(" + + "table => '%s'," + + "older_than => TIMESTAMP '%s'," + + "prefix_mismatch_mode => 'ignore')", + catalogName, tableIdent, currentTimestamp) + .size()); + + // Drop table in afterEach has purge and fails due to invalid scheme "file1" used in this test + // Dropping the table here + sql("DROP TABLE %s", tableName); + } } diff --git a/spark/v3.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetWriteDistributionAndOrdering.java b/spark/v3.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetWriteDistributionAndOrdering.java index e7e52806792d..b994050d4070 100644 --- a/spark/v3.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetWriteDistributionAndOrdering.java +++ b/spark/v3.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetWriteDistributionAndOrdering.java @@ -20,11 +20,13 @@ import static org.apache.iceberg.expressions.Expressions.bucket; +import java.util.List; import java.util.Map; import org.apache.iceberg.NullOrder; import org.apache.iceberg.SortOrder; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -299,4 +301,21 @@ public void testSetWriteDistributedAndLocallyOrderedInverted() { SortOrder expected = SortOrder.builderFor(table.schema()).withOrderId(1).asc("id").build(); Assert.assertEquals("Sort order must match", expected, table.sortOrder()); } + + @Test + public void testDefaultSortOnBinaryBucketedColumn() { + sql( + "CREATE TABLE %s (c1 INT, c2 Binary) " + + "USING iceberg " + + "PARTITIONED BY (bucket(2, c2))", + tableName); + + sql("INSERT INTO %s VALUES (1, X'A1B1'), (2, X'A2B2')", tableName); + + byte[] bytes1 = new byte[] {-95, -79}; + byte[] bytes2 = new byte[] {-94, -78}; + List expected = ImmutableList.of(row(1, bytes1), row(2, bytes2)); + + assertEquals("Rows must match", expected, sql("SELECT * FROM %s ORDER BY c1", tableName)); + } } diff --git a/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java index 641b957d1176..ea400a779235 100644 --- a/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java +++ b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java @@ -20,7 +20,9 @@ import java.util.Map; import java.util.concurrent.Callable; +import org.apache.iceberg.SnapshotSummary; 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.ExceptionUtil; /** utility class to accept thread local commit properties */ @@ -35,13 +37,19 @@ private CommitMetadata() {} * running the code wrapped as a caller, and any snapshot committed within the callable object * will be attached with the metadata defined in properties * - * @param properties extra commit metadata to attach to the snapshot committed within callable + * @param properties extra commit metadata to attach to the snapshot committed within callable. + * The prefix will be removed for properties starting with {@link + * SnapshotSummary#EXTRA_METADATA_PREFIX} * @param callable the code to be executed * @param exClass the expected type of exception which would be thrown from callable */ public static R withCommitProperties( Map properties, Callable callable, Class exClass) throws E { - COMMIT_PROPERTIES.set(properties); + Map props = Maps.newHashMap(); + properties.forEach( + (k, v) -> props.put(k.replace(SnapshotSummary.EXTRA_METADATA_PREFIX, ""), v)); + + COMMIT_PROPERTIES.set(props); try { return callable.call(); } catch (Throwable e) { diff --git a/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java index c1216f47ba82..ca19a982c9a6 100644 --- a/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java +++ b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java @@ -446,12 +446,12 @@ public static void importSparkTable( } else { List sourceTablePartitions = getPartitions(spark, sourceTableIdent, partitionFilter); - Preconditions.checkArgument( - !sourceTablePartitions.isEmpty(), - "Cannot find any partitions in table %s", - sourceTableIdent); - importSparkPartitions( - spark, sourceTablePartitions, targetTable, spec, stagingDir, checkDuplicateFiles); + if (sourceTablePartitions.isEmpty()) { + targetTable.newAppend().commit(); + } else { + importSparkPartitions( + spark, sourceTablePartitions, targetTable, spec, stagingDir, checkDuplicateFiles); + } } } catch (AnalysisException e) { throw SparkExceptionUtil.toUncheckedException( diff --git a/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/actions/BaseDeleteOrphanFilesSparkAction.java b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/actions/BaseDeleteOrphanFilesSparkAction.java index 72b6268026ab..b23de35afb54 100644 --- a/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/actions/BaseDeleteOrphanFilesSparkAction.java +++ b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/actions/BaseDeleteOrphanFilesSparkAction.java @@ -23,8 +23,13 @@ import java.io.File; import java.io.IOException; +import java.io.Serializable; +import java.net.URI; +import java.util.Collections; import java.util.Iterator; import java.util.List; +import java.util.Map; +import java.util.Objects; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; import java.util.function.Consumer; @@ -39,13 +44,21 @@ import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.hadoop.HiddenPathFilter; +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.Splitter; +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.Iterators; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.spark.JobGroupInfo; +import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.Tasks; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.api.java.function.MapPartitionsFunction; import org.apache.spark.broadcast.Broadcast; import org.apache.spark.sql.Column; import org.apache.spark.sql.Dataset; @@ -58,6 +71,7 @@ import org.apache.spark.util.SerializableConfiguration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import scala.Tuple2; /** * An action that removes orphan metadata, data and delete files by listing a given location and @@ -79,6 +93,8 @@ public class BaseDeleteOrphanFilesSparkAction implements DeleteOrphanFiles { private static final Logger LOG = LoggerFactory.getLogger(BaseDeleteOrphanFilesSparkAction.class); + private static final Splitter COMMA = Splitter.on(","); + private static final Map EQUAL_SCHEMES_DEFAULT = ImmutableMap.of("s3n,s3a", "s3"); private static final UserDefinedFunction filenameUDF = functions.udf( (String path) -> { @@ -97,6 +113,9 @@ public class BaseDeleteOrphanFilesSparkAction private final int partitionDiscoveryParallelism; private final Table table; + private Map equalSchemes = flattenMap(EQUAL_SCHEMES_DEFAULT); + private Map equalAuthorities = Collections.emptyMap(); + private PrefixMismatchMode prefixMismatchMode = PrefixMismatchMode.ERROR; private String location = null; private long olderThanTimestamp = System.currentTimeMillis() - TimeUnit.DAYS.toMillis(3); private Consumer deleteFunc = @@ -134,6 +153,29 @@ public BaseDeleteOrphanFilesSparkAction executeDeleteWith(ExecutorService execut return this; } + @Override + public BaseDeleteOrphanFilesSparkAction prefixMismatchMode( + PrefixMismatchMode newPrefixMismatchMode) { + this.prefixMismatchMode = newPrefixMismatchMode; + return this; + } + + @Override + public BaseDeleteOrphanFilesSparkAction equalSchemes(Map newEqualSchemes) { + this.equalSchemes = Maps.newHashMap(); + equalSchemes.putAll(flattenMap(EQUAL_SCHEMES_DEFAULT)); + equalSchemes.putAll(flattenMap(newEqualSchemes)); + return this; + } + + @Override + public BaseDeleteOrphanFilesSparkAction equalAuthorities( + Map newEqualAuthorities) { + this.equalAuthorities = Maps.newHashMap(); + equalAuthorities.putAll(flattenMap(newEqualAuthorities)); + return this; + } + @Override public BaseDeleteOrphanFilesSparkAction location(String newLocation) { this.location = newLocation; @@ -174,13 +216,9 @@ private DeleteOrphanFiles.Result doExecute() { Dataset validFileDF = validDataFileDF.union(validMetadataFileDF); Dataset actualFileDF = buildActualFileDF(); - Column actualFileName = filenameUDF.apply(actualFileDF.col("file_path")); - Column validFileName = filenameUDF.apply(validFileDF.col("file_path")); - Column nameEqual = actualFileName.equalTo(validFileName); - Column actualContains = actualFileDF.col("file_path").contains(validFileDF.col("file_path")); - Column joinCond = nameEqual.and(actualContains); List orphanFiles = - actualFileDF.join(validFileDF, joinCond, "leftanti").as(Encoders.STRING()).collectAsList(); + findOrphanFiles( + spark(), actualFileDF, validFileDF, equalSchemes, equalAuthorities, prefixMismatchMode); Tasks.foreach(orphanFiles) .noRetry() @@ -296,4 +334,160 @@ private static FlatMapFunction, String> listDirsRecursively( return files.iterator(); }; } + + @VisibleForTesting + static List findOrphanFiles( + SparkSession spark, + Dataset actualFileDF, + Dataset validFileDF, + Map equalSchemes, + Map equalAuthorities, + PrefixMismatchMode prefixMismatchMode) { + Dataset actualFileMetadataDS = + actualFileDF.mapPartitions( + toFileMetadata(equalSchemes, equalAuthorities), Encoders.bean(FileMetadata.class)); + Dataset validFileMetadataDS = + validFileDF.mapPartitions( + toFileMetadata(equalSchemes, equalAuthorities), Encoders.bean(FileMetadata.class)); + + SetAccumulator> conflicts = new SetAccumulator<>(); + spark.sparkContext().register(conflicts); + + Column joinCond = actualFileMetadataDS.col("path").equalTo(validFileMetadataDS.col("path")); + + List orphanFiles = + actualFileMetadataDS + .joinWith(validFileMetadataDS, joinCond, "leftouter") + .mapPartitions(findOrphanFiles(prefixMismatchMode, conflicts), Encoders.STRING()) + .collectAsList(); + + if (prefixMismatchMode == PrefixMismatchMode.ERROR && !conflicts.value().isEmpty()) { + throw new ValidationException( + "Unable to determine whether certain files are orphan. " + + "Metadata references files that match listed/provided files except for authority/scheme. " + + "Please, inspect the conflicting authorities/schemes and provide which of them are equal " + + "by further configuring the action via equalSchemes() and equalAuthorities() methods. " + + "Set the prefix mismatch mode to 'NONE' to ignore remaining locations with conflicting " + + "authorities/schemes or to 'DELETE' iff you are ABSOLUTELY confident that remaining conflicting " + + "authorities/schemes are different. It will be impossible to recover deleted files. " + + "Conflicting authorities/schemes: %s.", + conflicts.value()); + } + + return orphanFiles; + } + + private static Map flattenMap(Map map) { + Map flattenedMap = Maps.newHashMap(); + if (map != null) { + for (String key : map.keySet()) { + String value = map.get(key); + for (String splitKey : COMMA.split(key)) { + flattenedMap.put(splitKey.trim(), value.trim()); + } + } + } + return flattenedMap; + } + + private static MapPartitionsFunction, String> findOrphanFiles( + PrefixMismatchMode mode, SetAccumulator> conflicts) { + return rows -> { + Iterator transformed = + Iterators.transform( + rows, + row -> { + FileMetadata actual = row._1; + FileMetadata valid = row._2; + + if (valid == null) { + return actual.location; + } + + boolean schemeMatch = + Strings.isNullOrEmpty(valid.scheme) + || valid.scheme.equalsIgnoreCase(actual.scheme); + boolean authorityMatch = + Strings.isNullOrEmpty(valid.authority) + || valid.authority.equalsIgnoreCase(actual.authority); + + if ((!schemeMatch || !authorityMatch) && mode == PrefixMismatchMode.DELETE) { + return actual.location; + } else { + if (!schemeMatch) { + conflicts.add(Pair.of(valid.scheme, actual.scheme)); + } + if (!authorityMatch) { + conflicts.add(Pair.of(valid.authority, actual.authority)); + } + } + + return null; + }); + return Iterators.filter(transformed, Objects::nonNull); + }; + } + + private static MapPartitionsFunction toFileMetadata( + Map equalSchemesMap, Map equalAuthoritiesMap) { + return rows -> + Iterators.transform( + rows, + row -> { + String location = row.getString(0); + URI uri = new Path(location).toUri(); + String scheme = equalSchemesMap.getOrDefault(uri.getScheme(), uri.getScheme()); + String authority = + equalAuthoritiesMap.getOrDefault(uri.getAuthority(), uri.getAuthority()); + return new FileMetadata(scheme, authority, uri.getPath(), location); + }); + } + + public static class FileMetadata implements Serializable { + private String scheme; + private String authority; + private String path; + private String location; + + public FileMetadata(String scheme, String authority, String path, String location) { + this.scheme = scheme; + this.authority = authority; + this.path = path; + this.location = location; + } + + public FileMetadata() {} + + public void setScheme(String scheme) { + this.scheme = scheme; + } + + public void setAuthority(String authority) { + this.authority = authority; + } + + public void setPath(String path) { + this.path = path; + } + + public void setLocation(String location) { + this.location = location; + } + + public String getScheme() { + return scheme; + } + + public String getAuthority() { + return authority; + } + + public String getPath() { + return path; + } + + public String getLocation() { + return location; + } + } } diff --git a/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/actions/SetAccumulator.java b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/actions/SetAccumulator.java new file mode 100644 index 000000000000..745169fc1efd --- /dev/null +++ b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/actions/SetAccumulator.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.actions; + +import java.util.Collections; +import java.util.Set; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.spark.util.AccumulatorV2; + +public class SetAccumulator extends AccumulatorV2> { + + private final Set set = Collections.synchronizedSet(Sets.newHashSet()); + + @Override + public boolean isZero() { + return set.isEmpty(); + } + + @Override + public AccumulatorV2> copy() { + SetAccumulator newAccumulator = new SetAccumulator<>(); + newAccumulator.set.addAll(set); + return newAccumulator; + } + + @Override + public void reset() { + set.clear(); + } + + @Override + public void add(T v) { + set.add(v); + } + + @Override + public void merge(AccumulatorV2> other) { + set.addAll(other.value()); + } + + @Override + public Set value() { + return set; + } +} diff --git a/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/procedures/RemoveOrphanFilesProcedure.java b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/procedures/RemoveOrphanFilesProcedure.java index b41ada175498..f91d59307163 100644 --- a/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/procedures/RemoveOrphanFilesProcedure.java +++ b/spark/v3.1/spark/src/main/java/org/apache/iceberg/spark/procedures/RemoveOrphanFilesProcedure.java @@ -18,14 +18,17 @@ */ package org.apache.iceberg.spark.procedures; +import java.util.Map; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import org.apache.iceberg.Table; import org.apache.iceberg.actions.DeleteOrphanFiles; +import org.apache.iceberg.actions.DeleteOrphanFiles.PrefixMismatchMode; 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.Maps; import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors; import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.iceberg.spark.actions.SparkActions; @@ -40,6 +43,7 @@ import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; import org.apache.spark.unsafe.types.UTF8String; +import scala.runtime.BoxedUnit; /** * A procedure that removes orphan files in a table. @@ -54,7 +58,10 @@ public class RemoveOrphanFilesProcedure extends BaseProcedure { ProcedureParameter.optional("older_than", DataTypes.TimestampType), ProcedureParameter.optional("location", DataTypes.StringType), ProcedureParameter.optional("dry_run", DataTypes.BooleanType), - ProcedureParameter.optional("max_concurrent_deletes", DataTypes.IntegerType) + ProcedureParameter.optional("max_concurrent_deletes", DataTypes.IntegerType), + ProcedureParameter.optional("equal_schemes", STRING_MAP), + ProcedureParameter.optional("equal_authorities", STRING_MAP), + ProcedureParameter.optional("prefix_mismatch_mode", DataTypes.StringType), }; private static final StructType OUTPUT_TYPE = @@ -87,6 +94,7 @@ public StructType outputType() { } @Override + @SuppressWarnings("checkstyle:CyclomaticComplexity") public InternalRow[] call(InternalRow args) { Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name()); Long olderThanMillis = args.isNullAt(1) ? null : DateTimeUtil.microsToMillis(args.getLong(1)); @@ -99,6 +107,33 @@ public InternalRow[] call(InternalRow args) { "max_concurrent_deletes should have value > 0, value: %s", maxConcurrentDeletes); + Map equalSchemes = Maps.newHashMap(); + if (!args.isNullAt(5)) { + args.getMap(5) + .foreach( + DataTypes.StringType, + DataTypes.StringType, + (k, v) -> { + equalSchemes.put(k.toString(), v.toString()); + return BoxedUnit.UNIT; + }); + } + + Map equalAuthorities = Maps.newHashMap(); + if (!args.isNullAt(6)) { + args.getMap(6) + .foreach( + DataTypes.StringType, + DataTypes.StringType, + (k, v) -> { + equalSchemes.put(k.toString(), v.toString()); + return BoxedUnit.UNIT; + }); + } + + PrefixMismatchMode prefixMismatchMode = + args.isNullAt(7) ? null : PrefixMismatchMode.fromString(args.getString(7)); + return withIcebergTable( tableIdent, table -> { @@ -124,6 +159,13 @@ public InternalRow[] call(InternalRow args) { action.executeDeleteWith(removeService(maxConcurrentDeletes)); } + action.equalSchemes(equalSchemes); + action.equalAuthorities(equalAuthorities); + + if (prefixMismatchMode != null) { + action.prefixMismatchMode(prefixMismatchMode); + } + DeleteOrphanFiles.Result result = action.execute(); return toOutputRows(result); diff --git a/spark/v3.1/spark/src/main/scala/org/apache/spark/sql/catalyst/expressions/TransformExpressions.scala b/spark/v3.1/spark/src/main/scala/org/apache/spark/sql/catalyst/expressions/TransformExpressions.scala index 45e429bdc99f..29a92315941d 100644 --- a/spark/v3.1/spark/src/main/scala/org/apache/spark/sql/catalyst/expressions/TransformExpressions.scala +++ b/spark/v3.1/spark/src/main/scala/org/apache/spark/sql/catalyst/expressions/TransformExpressions.scala @@ -96,6 +96,9 @@ case class IcebergBucketTransform(numBuckets: Int, child: Expression) extends Ic // TODO: pass bytes without the copy out of the InternalRow val t = Transforms.bucket[ByteBuffer](Types.BinaryType.get(), numBuckets) s: Any => t(ByteBuffer.wrap(s.asInstanceOf[UTF8String].getBytes)).toInt + case _: BinaryType => + val t = Transforms.bucket[Any](numBuckets).bind(icebergInputType) + b: Any => t(ByteBuffer.wrap(b.asInstanceOf[Array[Byte]])).toInt case _ => val t = Transforms.bucket[Any](icebergInputType, numBuckets) a: Any => t(a).toInt diff --git a/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java b/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java index 352ac403bb0b..cb6302d6d7ab 100644 --- a/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java +++ b/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java @@ -42,6 +42,7 @@ import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.execution.ui.SQLExecutionUIData; import org.apache.spark.sql.internal.SQLConf; +import org.assertj.core.api.Assertions; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -172,7 +173,11 @@ protected void assertEquals(String context, Object[] expectedRow, Object[] actua Object actualValue = actualRow[col]; if (expectedValue != null && expectedValue.getClass().isArray()) { String newContext = String.format("%s (nested col %d)", context, col + 1); - assertEquals(newContext, (Object[]) expectedValue, (Object[]) actualValue); + if (expectedValue instanceof byte[]) { + Assertions.assertThat(actualValue).as(newContext).isEqualTo(expectedValue); + } else { + assertEquals(newContext, (Object[]) expectedValue, (Object[]) actualValue); + } } else if (expectedValue != ANY) { Assert.assertEquals(context + " contents should match", expectedValue, actualValue); } diff --git a/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java b/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java index 8877fea51c3e..19456643b837 100644 --- a/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java +++ b/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java @@ -734,4 +734,124 @@ public void testRemoveOrphanFilesWithStatisticFiles() throws Exception { .isEqualTo(statsLocation.toURI().toString()); Assertions.assertThat(statsLocation.exists()).as("stats file should be deleted").isFalse(); } + + @Test + public void testPathsWithExtraSlashes() { + List validFiles = Lists.newArrayList("file:///dir1/dir2/file1"); + List actualFiles = Lists.newArrayList("file:///dir1/////dir2///file1"); + executeTest(validFiles, actualFiles, Lists.newArrayList()); + } + + @Test + public void testPathsWithValidFileHavingNoAuthority() { + List validFiles = Lists.newArrayList("hdfs:///dir1/dir2/file1"); + List actualFiles = Lists.newArrayList("hdfs://servicename/dir1/dir2/file1"); + executeTest(validFiles, actualFiles, Lists.newArrayList()); + } + + @Test + public void testPathsWithActualFileHavingNoAuthority() { + List validFiles = Lists.newArrayList("hdfs://servicename/dir1/dir2/file1"); + List actualFiles = Lists.newArrayList("hdfs:///dir1/dir2/file1"); + executeTest(validFiles, actualFiles, Lists.newArrayList()); + } + + @Test + public void testPathsWithEqualSchemes() { + List validFiles = Lists.newArrayList("scheme1://bucket1/dir1/dir2/file1"); + List actualFiles = Lists.newArrayList("scheme2://bucket1/dir1/dir2/file1"); + AssertHelpers.assertThrows( + "Test remove orphan files with equal schemes", + ValidationException.class, + "Conflicting authorities/schemes: [(scheme1, scheme2)]", + () -> + executeTest( + validFiles, + actualFiles, + Lists.newArrayList(), + ImmutableMap.of(), + ImmutableMap.of(), + DeleteOrphanFiles.PrefixMismatchMode.ERROR)); + + Map equalSchemes = Maps.newHashMap(); + equalSchemes.put("scheme1", "scheme"); + equalSchemes.put("scheme2", "scheme"); + executeTest( + validFiles, + actualFiles, + Lists.newArrayList(), + equalSchemes, + ImmutableMap.of(), + DeleteOrphanFiles.PrefixMismatchMode.ERROR); + } + + @Test + public void testPathsWithEqualAuthorities() { + List validFiles = Lists.newArrayList("hdfs://servicename1/dir1/dir2/file1"); + List actualFiles = Lists.newArrayList("hdfs://servicename2/dir1/dir2/file1"); + AssertHelpers.assertThrows( + "Test remove orphan files with equal authorities", + ValidationException.class, + "Conflicting authorities/schemes: [(servicename1, servicename2)]", + () -> + executeTest( + validFiles, + actualFiles, + Lists.newArrayList(), + ImmutableMap.of(), + ImmutableMap.of(), + DeleteOrphanFiles.PrefixMismatchMode.ERROR)); + + Map equalAuthorities = Maps.newHashMap(); + equalAuthorities.put("servicename1", "servicename"); + equalAuthorities.put("servicename2", "servicename"); + executeTest( + validFiles, + actualFiles, + Lists.newArrayList(), + ImmutableMap.of(), + equalAuthorities, + DeleteOrphanFiles.PrefixMismatchMode.ERROR); + } + + @Test + public void testRemoveOrphanFileActionWithDeleteMode() { + List validFiles = Lists.newArrayList("hdfs://servicename1/dir1/dir2/file1"); + List actualFiles = Lists.newArrayList("hdfs://servicename2/dir1/dir2/file1"); + + executeTest( + validFiles, + actualFiles, + Lists.newArrayList("hdfs://servicename2/dir1/dir2/file1"), + ImmutableMap.of(), + ImmutableMap.of(), + DeleteOrphanFiles.PrefixMismatchMode.DELETE); + } + + private void executeTest( + List validFiles, List actualFiles, List expectedOrphanFiles) { + executeTest( + validFiles, + actualFiles, + expectedOrphanFiles, + ImmutableMap.of(), + ImmutableMap.of(), + DeleteOrphanFiles.PrefixMismatchMode.IGNORE); + } + + private void executeTest( + List validFiles, + List actualFiles, + List expectedOrphanFiles, + Map equalSchemes, + Map equalAuthorities, + DeleteOrphanFiles.PrefixMismatchMode mode) { + Dataset validFilesDF = spark.createDataset(validFiles, Encoders.STRING()).toDF(); + Dataset actualFilesDF = spark.createDataset(actualFiles, Encoders.STRING()).toDF(); + + List orphanFiles = + BaseDeleteOrphanFilesSparkAction.findOrphanFiles( + spark, actualFilesDF, validFilesDF, equalSchemes, equalAuthorities, mode); + Assert.assertEquals(expectedOrphanFiles, orphanFiles); + } } diff --git a/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java b/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java index 65539edbe6b1..faab1914b5da 100644 --- a/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java +++ b/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java @@ -40,10 +40,13 @@ import org.apache.arrow.vector.ValueVector; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericData.Record; +import org.apache.iceberg.DataFile; import org.apache.iceberg.FileContent; +import org.apache.iceberg.FileScanTask; import org.apache.iceberg.ManifestFile; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; +import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.spark.data.vectorized.IcebergArrowColumnVector; @@ -796,4 +799,9 @@ public static Dataset selectNonDerived(Dataset metadataTable) { public static Types.StructType nonDerivedSchema(Dataset metadataTable) { return SparkSchemaUtil.convert(TestHelpers.selectNonDerived(metadataTable).schema()).asStruct(); } + + public static List dataFiles(Table table) { + CloseableIterable tasks = table.newScan().includeColumnStats().planFiles(); + return Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); + } } diff --git a/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java b/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java index 750474564078..5eeb87d747e4 100644 --- a/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java +++ b/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java @@ -24,7 +24,6 @@ import java.math.RoundingMode; import java.util.List; import java.util.Map; -import java.util.Set; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.DataFile; @@ -34,13 +33,14 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.hadoop.HadoopTables; import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.relocated.com.google.common.math.LongMath; import org.apache.iceberg.spark.CommitMetadata; import org.apache.iceberg.spark.SparkReadOptions; @@ -52,6 +52,7 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; +import org.assertj.core.api.Assertions; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -427,8 +428,14 @@ public void testExtraSnapshotMetadataWithSQL() throws InterruptedException, IOEx Thread writerThread = new Thread( () -> { - Map properties = Maps.newHashMap(); - properties.put("writer-thread", String.valueOf(Thread.currentThread().getName())); + Map properties = + ImmutableMap.of( + "writer-thread", + String.valueOf(Thread.currentThread().getName()), + SnapshotSummary.EXTRA_METADATA_PREFIX + "extra-key", + "someValue", + SnapshotSummary.EXTRA_METADATA_PREFIX + "another-key", + "anotherValue"); CommitMetadata.withCommitProperties( properties, () -> { @@ -440,12 +447,13 @@ public void testExtraSnapshotMetadataWithSQL() throws InterruptedException, IOEx writerThread.setName("test-extra-commit-message-writer-thread"); writerThread.start(); writerThread.join(); - Set threadNames = Sets.newHashSet(); - for (Snapshot snapshot : table.snapshots()) { - threadNames.add(snapshot.summary().get("writer-thread")); - } - Assert.assertEquals(2, threadNames.size()); - Assert.assertTrue(threadNames.contains(null)); - Assert.assertTrue(threadNames.contains("test-extra-commit-message-writer-thread")); + + List snapshots = Lists.newArrayList(table.snapshots()); + Assert.assertEquals(2, snapshots.size()); + Assert.assertNull(snapshots.get(0).summary().get("writer-thread")); + Assertions.assertThat(snapshots.get(1).summary()) + .containsEntry("writer-thread", "test-extra-commit-message-writer-thread") + .containsEntry("extra-key", "someValue") + .containsEntry("another-key", "anotherValue"); } } diff --git a/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index 2dda34689062..f84a39b735c6 100644 --- a/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/v3.1/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -25,6 +25,7 @@ import java.io.IOException; import java.io.UncheckedIOException; +import java.util.Arrays; import java.util.Comparator; import java.util.List; import java.util.StringJoiner; @@ -45,6 +46,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.actions.DeleteOrphanFiles; +import org.apache.iceberg.actions.RewriteManifests; import org.apache.iceberg.avro.Avro; import org.apache.iceberg.avro.AvroSchemaUtil; import org.apache.iceberg.catalog.TableIdentifier; @@ -1217,7 +1219,7 @@ public void testAllManifestsTable() { @Test public void testUnpartitionedPartitionsTable() { TableIdentifier tableIdentifier = TableIdentifier.of("db", "unpartitioned_partitions_test"); - createTable(tableIdentifier, SCHEMA, PartitionSpec.unpartitioned()); + Table table = createTable(tableIdentifier, SCHEMA, PartitionSpec.unpartitioned()); Dataset df = spark.createDataFrame(Lists.newArrayList(new SimpleRecord(1, "a")), SimpleRecord.class); @@ -1232,6 +1234,11 @@ public void testUnpartitionedPartitionsTable() { Types.StructType.of( required(2, "record_count", Types.LongType.get(), "Count of records in data files"), required(3, "file_count", Types.IntegerType.get(), "Count of data files"), + required( + 11, + "total_data_file_size_in_bytes", + Types.LongType.get(), + "Total size in bytes of data files"), required( 5, "position_delete_record_count", @@ -1251,7 +1258,17 @@ public void testUnpartitionedPartitionsTable() { 8, "equality_delete_file_count", Types.IntegerType.get(), - "Count of equality delete files")); + "Count of equality delete files"), + optional( + 9, + "last_updated_at", + Types.TimestampType.withZone(), + "Commit time of snapshot that last updated this partition"), + optional( + 10, + "last_updated_snapshot_id", + Types.LongType.get(), + "Id of snapshot that last updated this partition")); Table partitionsTable = loadTable(tableIdentifier, "partitions"); @@ -1264,8 +1281,13 @@ public void testUnpartitionedPartitionsTable() { new GenericRecordBuilder(AvroSchemaUtil.convert(partitionsTable.schema(), "partitions")); GenericData.Record expectedRow = builder + .set("last_updated_at", table.currentSnapshot().timestampMillis() * 1000) + .set("last_updated_snapshot_id", table.currentSnapshot().snapshotId()) .set("record_count", 1L) .set("file_count", 1) + .set( + "total_data_file_size_in_bytes", + totalSizeInBytes(table.currentSnapshot().addedDataFiles(table.io()))) .set("position_delete_record_count", 0L) .set("position_delete_file_count", 0) .set("equality_delete_record_count", 0L) @@ -1309,6 +1331,9 @@ public void testPartitionsTable() { .mode("append") .save(loadLocation(tableIdentifier)); + table.refresh(); + long secondCommitId = table.currentSnapshot().snapshotId(); + List actual = spark .read() @@ -1329,22 +1354,32 @@ public void testPartitionsTable() { .set("partition", partitionBuilder.set("id", 1).build()) .set("record_count", 1L) .set("file_count", 1) + .set( + "total_data_file_size_in_bytes", + totalSizeInBytes(table.snapshot(firstCommitId).addedDataFiles(table.io()))) .set("position_delete_record_count", 0L) .set("position_delete_file_count", 0) .set("equality_delete_record_count", 0L) .set("equality_delete_file_count", 0) .set("spec_id", 0) + .set("last_updated_at", table.snapshot(firstCommitId).timestampMillis() * 1000) + .set("last_updated_snapshot_id", firstCommitId) .build()); expected.add( builder .set("partition", partitionBuilder.set("id", 2).build()) .set("record_count", 1L) .set("file_count", 1) + .set( + "total_data_file_size_in_bytes", + totalSizeInBytes(table.snapshot(secondCommitId).addedDataFiles(table.io()))) .set("position_delete_record_count", 0L) .set("position_delete_file_count", 0) .set("equality_delete_record_count", 0L) .set("equality_delete_file_count", 0) .set("spec_id", 0) + .set("last_updated_at", table.snapshot(secondCommitId).timestampMillis() * 1000) + .set("last_updated_snapshot_id", secondCommitId) .build()); Assert.assertEquals("Partitions table should have two rows", 2, expected.size()); @@ -1387,11 +1422,158 @@ public void testPartitionsTable() { .load(loadLocation(tableIdentifier, "partitions")) .filter("partition.id < 2 or record_count=1") .collectAsList(); - Assert.assertEquals("Actual results should have one row", 2, nonFiltered.size()); + Assert.assertEquals("Actual results should have two row", 2, nonFiltered.size()); + for (int i = 0; i < 2; i += 1) { + TestHelpers.assertEqualsSafe( + partitionsTable.schema().asStruct(), expected.get(i), actual.get(i)); + } + } + + @Test + public void testPartitionsTableLastUpdatedSnapshot() { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "partitions_test"); + Table table = createTable(tableIdentifier, SCHEMA, SPEC); + Table partitionsTable = loadTable(tableIdentifier, "partitions"); + Dataset df1 = + spark.createDataFrame( + Lists.newArrayList(new SimpleRecord(1, "1"), new SimpleRecord(2, "2")), + SimpleRecord.class); + Dataset df2 = + spark.createDataFrame(Lists.newArrayList(new SimpleRecord(2, "20")), SimpleRecord.class); + + df1.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + table.refresh(); + long firstCommitId = table.currentSnapshot().snapshotId(); + + // add a second file + df2.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + table.refresh(); + long secondCommitId = table.currentSnapshot().snapshotId(); + + // check if rewrite manifest does not override metadata about data file's creating snapshot + RewriteManifests.Result rewriteManifestResult = + SparkActions.get().rewriteManifests(table).execute(); + Assert.assertEquals( + "rewrite replaced 2 manifests", + 2, + Iterables.size(rewriteManifestResult.rewrittenManifests())); + Assert.assertEquals( + "rewrite added 1 manifests", 1, Iterables.size(rewriteManifestResult.addedManifests())); + + List actual = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "partitions")) + .orderBy("partition.id") + .collectAsList(); + + GenericRecordBuilder builder = + new GenericRecordBuilder(AvroSchemaUtil.convert(partitionsTable.schema(), "partitions")); + GenericRecordBuilder partitionBuilder = + new GenericRecordBuilder( + AvroSchemaUtil.convert( + partitionsTable.schema().findType("partition").asStructType(), "partition")); + + List dataFiles = TestHelpers.dataFiles(table); + assertDataFilePartitions(dataFiles, Arrays.asList(1, 2, 2)); + + List expected = Lists.newArrayList(); + expected.add( + builder + .set("partition", partitionBuilder.set("id", 1).build()) + .set("record_count", 1L) + .set("file_count", 1) + .set("total_data_file_size_in_bytes", dataFiles.get(0).fileSizeInBytes()) + .set("position_delete_record_count", 0L) + .set("position_delete_file_count", 0) + .set("equality_delete_record_count", 0L) + .set("equality_delete_file_count", 0) + .set("spec_id", 0) + .set("last_updated_at", table.snapshot(firstCommitId).timestampMillis() * 1000) + .set("last_updated_snapshot_id", firstCommitId) + .build()); + expected.add( + builder + .set("partition", partitionBuilder.set("id", 2).build()) + .set("record_count", 2L) + .set("file_count", 2) + .set( + "total_data_file_size_in_bytes", + dataFiles.get(1).fileSizeInBytes() + dataFiles.get(2).fileSizeInBytes()) + .set("position_delete_record_count", 0L) + .set("position_delete_file_count", 0) + .set("equality_delete_record_count", 0L) + .set("equality_delete_file_count", 0) + .set("spec_id", 0) + .set("last_updated_at", table.snapshot(secondCommitId).timestampMillis() * 1000) + .set("last_updated_snapshot_id", secondCommitId) + .build()); + + Assert.assertEquals("Partitions table should have two rows", 2, expected.size()); + Assert.assertEquals("Actual results should have two rows", 2, actual.size()); for (int i = 0; i < 2; i += 1) { TestHelpers.assertEqualsSafe( partitionsTable.schema().asStruct(), expected.get(i), actual.get(i)); } + + // check predicate push down + List filtered = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "partitions")) + .filter("partition.id < 2") + .collectAsList(); + Assert.assertEquals("Actual results should have one row", 1, filtered.size()); + TestHelpers.assertEqualsSafe( + partitionsTable.schema().asStruct(), expected.get(0), filtered.get(0)); + + // check for snapshot expiration + // if snapshot with firstCommitId is expired, + // we expect the partition of id=1 will no longer have last updated timestamp and snapshotId + SparkActions.get().expireSnapshots(table).expireSnapshotId(firstCommitId).execute(); + GenericData.Record newPartitionRecord = + builder + .set("partition", partitionBuilder.set("id", 1).build()) + .set("record_count", 1L) + .set("file_count", 1) + .set("total_data_file_size_in_bytes", dataFiles.get(0).fileSizeInBytes()) + .set("position_delete_record_count", 0L) + .set("position_delete_file_count", 0) + .set("equality_delete_record_count", 0L) + .set("equality_delete_file_count", 0) + .set("spec_id", 0) + .set("last_updated_at", null) + .set("last_updated_snapshot_id", null) + .build(); + expected.remove(0); + expected.add(0, newPartitionRecord); + + List actualAfterSnapshotExpiration = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "partitions")) + .collectAsList(); + Assert.assertEquals( + "Actual results should have two row", 2, actualAfterSnapshotExpiration.size()); + for (int i = 0; i < 2; i += 1) { + TestHelpers.assertEqualsSafe( + partitionsTable.schema().asStruct(), + expected.get(i), + actualAfterSnapshotExpiration.get(i)); + } } @Test @@ -1870,4 +2052,23 @@ public static Dataset selectNonDerived(Dataset metadataTable) { public static Types.StructType nonDerivedSchema(Dataset metadataTable) { return SparkSchemaUtil.convert(selectNonDerived(metadataTable).schema()).asStruct(); } + + private long totalSizeInBytes(Iterable dataFiles) { + return Lists.newArrayList(dataFiles).stream().mapToLong(DataFile::fileSizeInBytes).sum(); + } + + private void assertDataFilePartitions( + List dataFiles, List expectedPartitionIds) { + Assert.assertEquals( + "Table should have " + expectedPartitionIds.size() + " data files", + expectedPartitionIds.size(), + dataFiles.size()); + + for (int i = 0; i < dataFiles.size(); ++i) { + Assert.assertEquals( + "Data file should have partition of id " + expectedPartitionIds.get(i), + expectedPartitionIds.get(i).intValue(), + dataFiles.get(i).partition().get(0, Integer.class).intValue()); + } + } } diff --git a/spark/v3.2/build.gradle b/spark/v3.2/build.gradle index 2b57f49075c0..49ef3e3c319e 100644 --- a/spark/v3.2/build.gradle +++ b/spark/v3.2/build.gradle @@ -225,6 +225,7 @@ project(":iceberg-spark:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersio integrationImplementation "org.apache.spark:spark-hive_${scalaVersion}:${sparkVersion}" integrationImplementation 'org.junit.vintage:junit-vintage-engine' integrationImplementation 'org.slf4j:slf4j-simple' + integrationImplementation 'org.assertj:assertj-core' integrationImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') integrationImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') integrationImplementation project(path: ":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}", configuration: 'testArtifacts') @@ -272,6 +273,7 @@ project(":iceberg-spark:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersio task integrationTest(type: Test) { description = "Test Spark3 Runtime Jar against Spark ${sparkMajorVersion}" group = "verification" + jvmArgs += project.property('extraJvmArgs') testClassesDirs = sourceSets.integration.output.classesDirs classpath = sourceSets.integration.runtimeClasspath + files(shadowJar.archiveFile.get().asFile.path) inputs.file(shadowJar.archiveFile.get().asFile.path) diff --git a/spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala b/spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala index f758cb08fd3d..2e438de2b8cd 100644 --- a/spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala +++ b/spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala @@ -116,6 +116,7 @@ class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergS val branchRetention = branchOptionsContext.flatMap(branchOptions => Option(branchOptions.refRetain())) val branchRefAgeMs = branchRetention.map(retain => TimeUnit.valueOf(retain.timeUnit().getText.toUpperCase(Locale.ENGLISH)).toMillis(retain.number().getText.toLong)) + val create = createOrReplaceBranchClause.CREATE() != null val replace = ctx.createReplaceBranchClause().REPLACE() != null val ifNotExists = createOrReplaceBranchClause.EXISTS() != null @@ -130,6 +131,7 @@ class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergS typedVisit[Seq[String]](ctx.multipartIdentifier), branchName.getText, branchOptions, + create, replace, ifNotExists) } @@ -153,12 +155,14 @@ class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergS tagRefAgeMs ) + val create = createTagClause.CREATE() != null val replace = createTagClause.REPLACE() != null val ifNotExists = createTagClause.EXISTS() != null CreateOrReplaceTag(typedVisit[Seq[String]](ctx.multipartIdentifier), tagName, tagOptions, + create, replace, ifNotExists) } diff --git a/spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala b/spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala index 2a22484499cf..b7981a3c7a0d 100644 --- a/spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala +++ b/spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala @@ -25,6 +25,7 @@ case class CreateOrReplaceBranch( table: Seq[String], branch: String, branchOptions: BranchOptions, + create: Boolean, replace: Boolean, ifNotExists: Boolean) extends LeafCommand { diff --git a/spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala b/spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala index e48f7d8ed04c..6e7db84a90fb 100644 --- a/spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala +++ b/spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala @@ -25,6 +25,7 @@ case class CreateOrReplaceTag( table: Seq[String], tag: String, tagOptions: TagOptions, + create: Boolean, replace: Boolean, ifNotExists: Boolean) extends LeafCommand { diff --git a/spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala b/spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala index 8552ab132f45..142ed1357135 100644 --- a/spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala +++ b/spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala @@ -32,6 +32,7 @@ case class CreateOrReplaceBranchExec( ident: Identifier, branch: String, branchOptions: BranchOptions, + create: Boolean, replace: Boolean, ifNotExists: Boolean) extends LeafV2CommandExec { @@ -51,15 +52,18 @@ case class CreateOrReplaceBranchExec( "Cannot complete create or replace branch operation on %s, main has no snapshot", ident) val manageSnapshots = iceberg.table().manageSnapshots() - if (!replace) { - val ref = iceberg.table().refs().get(branch) - if (ref != null && ifNotExists) { + val refExists = null != iceberg.table().refs().get(branch) + + if (create && replace && !refExists) { + manageSnapshots.createBranch(branch, snapshotId) + } else if (replace) { + manageSnapshots.replaceBranch(branch, snapshotId) + } else { + if (refExists && ifNotExists) { return Nil } manageSnapshots.createBranch(branch, snapshotId) - } else { - manageSnapshots.replaceBranch(branch, snapshotId) } if (branchOptions.numSnapshots.nonEmpty) { diff --git a/spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala b/spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala index 7ca193d1b156..372cd7548632 100644 --- a/spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala +++ b/spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala @@ -31,6 +31,7 @@ case class CreateOrReplaceTagExec( ident: Identifier, tag: String, tagOptions: TagOptions, + create: Boolean, replace: Boolean, ifNotExists: Boolean) extends LeafV2CommandExec { @@ -50,15 +51,18 @@ case class CreateOrReplaceTagExec( "Cannot complete create or replace tag operation on %s, main has no snapshot", ident) val manageSnapshot = iceberg.table.manageSnapshots() - if (!replace) { - val ref = iceberg.table().refs().get(tag) - if (ref != null && ifNotExists) { + val refExists = null != iceberg.table().refs().get(tag) + + if (create && replace && !refExists) { + manageSnapshot.createTag(tag, snapshotId) + } else if (replace) { + manageSnapshot.replaceTag(tag, snapshotId) + } else { + if (refExists && ifNotExists) { return Nil } manageSnapshot.createTag(tag, snapshotId) - } else { - manageSnapshot.replaceTag(tag, snapshotId) } if (tagOptions.snapshotRefRetain.nonEmpty) { diff --git a/spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala b/spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala index 326574bf25e4..0a27d49287f2 100644 --- a/spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala +++ b/spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala @@ -72,11 +72,12 @@ case class ExtendedDataSourceV2Strategy(spark: SparkSession) extends Strategy wi ReplacePartitionFieldExec(catalog, ident, transformFrom, transformTo, name) :: Nil case CreateOrReplaceBranch( - IcebergCatalogAndIdentifier(catalog, ident), branch, branchOptions, replace, ifNotExists) => - CreateOrReplaceBranchExec(catalog, ident, branch, branchOptions, replace, ifNotExists) :: Nil + IcebergCatalogAndIdentifier(catalog, ident), branch, branchOptions, create, replace, ifNotExists) => + CreateOrReplaceBranchExec(catalog, ident, branch, branchOptions, create, replace, ifNotExists) :: Nil - case CreateOrReplaceTag(IcebergCatalogAndIdentifier(catalog, ident), tag, tagOptions, replace, ifNotExists) => - CreateOrReplaceTagExec(catalog, ident, tag, tagOptions, replace, ifNotExists) :: Nil + case CreateOrReplaceTag( + IcebergCatalogAndIdentifier(catalog, ident), tag, tagOptions, create, replace, ifNotExists) => + CreateOrReplaceTagExec(catalog, ident, tag, tagOptions, create, replace, ifNotExists) :: Nil case DropBranch(IcebergCatalogAndIdentifier(catalog, ident), branch, ifExists) => DropBranchExec(catalog, ident, branch, ifExists) :: Nil diff --git a/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java b/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java index 9af201b09898..10426ebb5594 100644 --- a/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java +++ b/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java @@ -103,6 +103,34 @@ public void addDataUnpartitioned() { sql("SELECT * FROM %s ORDER BY id", tableName)); } + @Test + public void deleteAndAddBackUnpartitioned() { + createUnpartitionedFileTable("parquet"); + + String createIceberg = + "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg"; + + sql(createIceberg, tableName); + + sql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`')", + catalogName, tableName, fileTableDir.getAbsolutePath()); + + String deleteData = "DELETE FROM %s"; + sql(deleteData, tableName); + + Object result = + scalarSql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`')", + catalogName, tableName, fileTableDir.getAbsolutePath()); + Assert.assertEquals(2L, result); + + assertEquals( + "Iceberg table contains correct data", + sql("SELECT * FROM %s ORDER BY id", sourceTableName), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + @Ignore // TODO Classpath issues prevent us from actually writing to a Spark ORC table public void addDataUnpartitionedOrc() { createUnpartitionedFileTable("orc"); @@ -397,6 +425,34 @@ public void addPartitionToPartitioned() { sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName)); } + @Test + public void deleteAndAddBackPartitioned() { + createPartitionedFileTable("parquet"); + + String createIceberg = + "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg PARTITIONED BY (id)"; + + sql(createIceberg, tableName); + + sql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('id', 1))", + catalogName, tableName, fileTableDir.getAbsolutePath()); + + String deleteData = "DELETE FROM %s where id = 1"; + sql(deleteData, tableName); + + Object result = + scalarSql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('id', 1))", + catalogName, tableName, fileTableDir.getAbsolutePath()); + Assert.assertEquals(2L, result); + + assertEquals( + "Iceberg table contains correct data", + sql("SELECT id, name, dept, subdept FROM %s WHERE id = 1 ORDER BY id", sourceTableName), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + @Test public void addPartitionToPartitionedSnapshotIdInheritanceEnabledInTwoRuns() { createPartitionedFileTable("parquet"); diff --git a/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java b/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java index 55a2a1c142b6..2c3cbac02820 100644 --- a/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java +++ b/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; @@ -512,6 +514,71 @@ public void testCreateOrReplace() throws NoSuchTableException { Assert.assertEquals(first, ref.snapshotId()); } + @Test + public void createOrReplace() throws NoSuchTableException { + Table table = insertRows(); + long first = table.currentSnapshot().snapshotId(); + String branchName = "b1"; + insertRows(); + long second = table.currentSnapshot().snapshotId(); + table.manageSnapshots().createBranch(branchName, second).commit(); + + sql( + "ALTER TABLE %s CREATE OR REPLACE BRANCH %s AS OF VERSION %d", + tableName, branchName, first); + table.refresh(); + assertThat(table.refs().get(branchName).snapshotId()).isEqualTo(second); + } + + @Test + public void createOrReplaceWithNonExistingBranch() throws NoSuchTableException { + Table table = insertRows(); + String branchName = "b1"; + insertRows(); + long snapshotId = table.currentSnapshot().snapshotId(); + + sql( + "ALTER TABLE %s CREATE OR REPLACE BRANCH %s AS OF VERSION %d", + tableName, branchName, snapshotId); + table.refresh(); + assertThat(table.refs().get(branchName).snapshotId()).isEqualTo(snapshotId); + } + + @Test + public void replaceBranch() throws NoSuchTableException { + Table table = insertRows(); + long first = table.currentSnapshot().snapshotId(); + String branchName = "b1"; + long expectedMaxRefAgeMs = 1000; + table + .manageSnapshots() + .createBranch(branchName, first) + .setMaxRefAgeMs(branchName, expectedMaxRefAgeMs) + .commit(); + + insertRows(); + long second = table.currentSnapshot().snapshotId(); + + sql("ALTER TABLE %s REPLACE BRANCH %s AS OF VERSION %d", tableName, branchName, second); + table.refresh(); + SnapshotRef ref = table.refs().get(branchName); + assertThat(ref.snapshotId()).isEqualTo(second); + assertThat(ref.maxRefAgeMs()).isEqualTo(expectedMaxRefAgeMs); + } + + @Test + public void replaceBranchDoesNotExist() throws NoSuchTableException { + Table table = insertRows(); + + Assertions.assertThatThrownBy( + () -> + sql( + "ALTER TABLE %s REPLACE BRANCH %s AS OF VERSION %d", + tableName, "someBranch", table.currentSnapshot().snapshotId())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Branch does not exist: someBranch"); + } + private Table insertRows() throws NoSuchTableException { List records = ImmutableList.of(new SimpleRecord(1, "a"), new SimpleRecord(2, "b")); diff --git a/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java b/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java index c46445b31db1..7352ab0809c6 100644 --- a/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java +++ b/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java @@ -30,7 +30,6 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -145,11 +144,11 @@ public void testDeleteFileThenMetadataDelete() throws Exception { // Metadata Delete Table table = Spark3Util.loadIcebergTable(spark, tableName); - Set dataFilesBefore = TestHelpers.dataFiles(table); + List dataFilesBefore = TestHelpers.dataFiles(table); sql("DELETE FROM %s AS t WHERE t.id = 1", tableName); - Set dataFilesAfter = TestHelpers.dataFiles(table); + List dataFilesAfter = TestHelpers.dataFiles(table); Assert.assertTrue( "Data file should have been removed", dataFilesBefore.size() > dataFilesAfter.size()); diff --git a/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestIcebergExpressions.java b/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestIcebergExpressions.java index 8d2e10ea17eb..b1ba53455123 100644 --- a/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestIcebergExpressions.java +++ b/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestIcebergExpressions.java @@ -24,6 +24,7 @@ import org.apache.spark.sql.Column; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.expressions.IcebergBucketTransform; import org.apache.spark.sql.catalyst.expressions.IcebergTruncateTransform; import org.junit.After; import org.junit.Test; @@ -71,4 +72,34 @@ public void testTruncateExpressions() { ImmutableList.of(row(100, 10000L, new BigDecimal("10.50"), "10", "12")), sql("SELECT int_c, long_c, dec_c, str_c, CAST(binary_c AS STRING) FROM v")); } + + @Test + public void testBucketExpressions() { + sql( + "CREATE TABLE %s ( " + + " int_c INT, long_c LONG, dec_c DECIMAL(4, 2), str_c STRING, binary_c BINARY " + + ") USING iceberg", + tableName); + + sql( + "CREATE TEMPORARY VIEW emp " + + "AS SELECT * FROM VALUES (101, 10001, 10.65, '101-Employee', CAST('1234' AS BINARY)) " + + "AS EMP(int_c, long_c, dec_c, str_c, binary_c)"); + + sql("INSERT INTO %s SELECT * FROM emp", tableName); + + Dataset df = spark.sql("SELECT * FROM " + tableName); + df.select( + new Column(new IcebergBucketTransform(2, df.col("int_c").expr())).as("int_c"), + new Column(new IcebergBucketTransform(3, df.col("long_c").expr())).as("long_c"), + new Column(new IcebergBucketTransform(4, df.col("dec_c").expr())).as("dec_c"), + new Column(new IcebergBucketTransform(5, df.col("str_c").expr())).as("str_c"), + new Column(new IcebergBucketTransform(6, df.col("binary_c").expr())).as("binary_c")) + .createOrReplaceTempView("v"); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(0, 2, 0, 4, 1)), + sql("SELECT int_c, long_c, dec_c, str_c, binary_c FROM v")); + } } diff --git a/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java b/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java index 83126b700eaf..d1adee74f28d 100644 --- a/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java +++ b/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java @@ -184,4 +184,43 @@ public void testInvalidMigrateCases() { "Cannot handle an empty identifier", () -> sql("CALL %s.system.migrate('')", catalogName)); } + + @Test + public void testMigratePartitionWithSpecialCharacter() throws IOException { + Assume.assumeTrue(catalogName.equals("spark_catalog")); + String location = temp.newFolder().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string, dt date) USING parquet " + + "PARTITIONED BY (data, dt) LOCATION '%s'", + tableName, location); + sql("INSERT INTO TABLE %s VALUES (1, '2023/05/30', date '2023-05-30')", tableName); + Object result = scalarSql("CALL %s.system.migrate('%s')", catalogName, tableName); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1L, "2023/05/30", java.sql.Date.valueOf("2023-05-30"))), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @Test + public void testMigrateEmptyPartitionedTable() throws Exception { + Assume.assumeTrue(catalogName.equals("spark_catalog")); + String location = temp.newFolder().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet PARTITIONED BY (id) LOCATION '%s'", + tableName, location); + Object result = scalarSql("CALL %s.system.migrate('%s')", catalogName, tableName); + Assert.assertEquals(0L, result); + } + + @Test + public void testMigrateEmptyTable() throws Exception { + Assume.assumeTrue(catalogName.equals("spark_catalog")); + String location = temp.newFolder().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + tableName, location); + Object result = scalarSql("CALL %s.system.migrate('%s')", catalogName, tableName); + Assert.assertEquals(0L, result); + } } diff --git a/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRequiredDistributionAndOrdering.java b/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRequiredDistributionAndOrdering.java index cfa7f6622aed..02ae0e6ac664 100644 --- a/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRequiredDistributionAndOrdering.java +++ b/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRequiredDistributionAndOrdering.java @@ -237,6 +237,23 @@ public void testDefaultSortOnStringBucketedColumn() { assertEquals("Rows must match", expected, sql("SELECT * FROM %s ORDER BY c1", tableName)); } + @Test + public void testDefaultSortOnBinaryBucketedColumn() { + sql( + "CREATE TABLE %s (c1 INT, c2 Binary) " + + "USING iceberg " + + "PARTITIONED BY (bucket(2, c2))", + tableName); + + sql("INSERT INTO %s VALUES (1, X'A1B1'), (2, X'A2B2')", tableName); + + byte[] bytes1 = new byte[] {-95, -79}; + byte[] bytes2 = new byte[] {-94, -78}; + List expected = ImmutableList.of(row(1, bytes1), row(2, bytes2)); + + assertEquals("Rows must match", expected, sql("SELECT * FROM %s ORDER BY c1", tableName)); + } + @Test public void testDefaultSortOnDecimalTruncatedColumn() { sql( diff --git a/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestTagDDL.java b/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestTagDDL.java index ec3148de6cb5..866a965e33e6 100644 --- a/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestTagDDL.java +++ b/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestTagDDL.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.List; import java.util.Locale; import java.util.Map; @@ -364,6 +366,18 @@ public void testDropTagIfExists() throws NoSuchTableException { Assert.assertNull("The tag needs to be dropped.", table.refs().get(tagName)); } + @Test + public void createOrReplaceWithNonExistingTag() throws NoSuchTableException { + Table table = insertRows(); + String tagName = "t1"; + insertRows(); + long snapshotId = table.currentSnapshot().snapshotId(); + + sql("ALTER TABLE %s CREATE OR REPLACE TAG %s AS OF VERSION %d", tableName, tagName, snapshotId); + table.refresh(); + assertThat(table.refs().get(tagName).snapshotId()).isEqualTo(snapshotId); + } + private Table insertRows() throws NoSuchTableException { List records = ImmutableList.of(new SimpleRecord(1, "a"), new SimpleRecord(2, "b")); diff --git a/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java b/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java index 641b957d1176..ea400a779235 100644 --- a/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java +++ b/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java @@ -20,7 +20,9 @@ import java.util.Map; import java.util.concurrent.Callable; +import org.apache.iceberg.SnapshotSummary; 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.ExceptionUtil; /** utility class to accept thread local commit properties */ @@ -35,13 +37,19 @@ private CommitMetadata() {} * running the code wrapped as a caller, and any snapshot committed within the callable object * will be attached with the metadata defined in properties * - * @param properties extra commit metadata to attach to the snapshot committed within callable + * @param properties extra commit metadata to attach to the snapshot committed within callable. + * The prefix will be removed for properties starting with {@link + * SnapshotSummary#EXTRA_METADATA_PREFIX} * @param callable the code to be executed * @param exClass the expected type of exception which would be thrown from callable */ public static R withCommitProperties( Map properties, Callable callable, Class exClass) throws E { - COMMIT_PROPERTIES.set(properties); + Map props = Maps.newHashMap(); + properties.forEach( + (k, v) -> props.put(k.replace(SnapshotSummary.EXTRA_METADATA_PREFIX, ""), v)); + + COMMIT_PROPERTIES.set(props); try { return callable.call(); } catch (Throwable e) { diff --git a/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/SparkSessionCatalog.java b/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/SparkSessionCatalog.java index c041df2ae3c5..c5f2b168e4cc 100644 --- a/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/SparkSessionCatalog.java +++ b/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/SparkSessionCatalog.java @@ -22,6 +22,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.spark.source.HasIcebergCatalog; @@ -277,7 +278,10 @@ public void renameTable(Identifier from, Identifier to) @Override public final void initialize(String name, CaseInsensitiveStringMap options) { - if (options.containsKey("type") && options.get("type").equalsIgnoreCase("hive")) { + if (options.containsKey(CatalogUtil.ICEBERG_CATALOG_TYPE) + && options + .get(CatalogUtil.ICEBERG_CATALOG_TYPE) + .equalsIgnoreCase(CatalogUtil.ICEBERG_CATALOG_TYPE_HIVE)) { validateHmsUri(options.get(CatalogProperties.URI)); } diff --git a/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java b/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java index 4bfff5b2c44a..d1956322907b 100644 --- a/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java +++ b/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java @@ -440,12 +440,12 @@ public static void importSparkTable( } else { List sourceTablePartitions = getPartitions(spark, sourceTableIdent, partitionFilter); - Preconditions.checkArgument( - !sourceTablePartitions.isEmpty(), - "Cannot find any partitions in table %s", - sourceTableIdent); - importSparkPartitions( - spark, sourceTablePartitions, targetTable, spec, stagingDir, checkDuplicateFiles); + if (sourceTablePartitions.isEmpty()) { + targetTable.newAppend().commit(); + } else { + importSparkPartitions( + spark, sourceTablePartitions, targetTable, spec, stagingDir, checkDuplicateFiles); + } } } catch (AnalysisException e) { throw SparkExceptionUtil.toUncheckedException( @@ -534,7 +534,7 @@ private static void importUnpartitionedSparkTable( .createDataset(Lists.transform(files, f -> f.path().toString()), Encoders.STRING()) .toDF("file_path"); Dataset existingFiles = - loadMetadataTable(spark, targetTable, MetadataTableType.ENTRIES); + loadMetadataTable(spark, targetTable, MetadataTableType.ENTRIES).filter("status != 2"); Column joinCond = existingFiles.col("data_file.file_path").equalTo(importedFiles.col("file_path")); Dataset duplicates = @@ -605,7 +605,8 @@ public static void importSparkPartitions( filesToImport .map((MapFunction) f -> f.path().toString(), Encoders.STRING()) .toDF("file_path"); - Dataset existingFiles = loadMetadataTable(spark, targetTable, MetadataTableType.ENTRIES); + Dataset existingFiles = + loadMetadataTable(spark, targetTable, MetadataTableType.ENTRIES).filter("status != 2"); Column joinCond = existingFiles.col("data_file.file_path").equalTo(importedFiles.col("file_path")); Dataset duplicates = diff --git a/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java b/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java index 5cd5b24547f9..c1144d944a66 100644 --- a/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java +++ b/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java @@ -420,7 +420,7 @@ public Scan buildCopyOnWriteScan() { @Override public Statistics estimateStatistics() { - return ((SparkScan) build()).estimateStatistics(); + return ((SupportsReportStatistics) build()).estimateStatistics(); } @Override diff --git a/spark/v3.2/spark/src/main/scala/org/apache/spark/sql/catalyst/expressions/TransformExpressions.scala b/spark/v3.2/spark/src/main/scala/org/apache/spark/sql/catalyst/expressions/TransformExpressions.scala index 2d2c2fb9aae5..b15cc275b6cb 100644 --- a/spark/v3.2/spark/src/main/scala/org/apache/spark/sql/catalyst/expressions/TransformExpressions.scala +++ b/spark/v3.2/spark/src/main/scala/org/apache/spark/sql/catalyst/expressions/TransformExpressions.scala @@ -110,6 +110,9 @@ case class IcebergBucketTransform(numBuckets: Int, child: Expression) extends Ic // TODO: pass bytes without the copy out of the InternalRow val t = Transforms.bucket[ByteBuffer](Types.BinaryType.get(), numBuckets) s: Any => t(ByteBuffer.wrap(s.asInstanceOf[UTF8String].getBytes)).toInt + case _: BinaryType => + val t = Transforms.bucket[Any](numBuckets).bind(icebergInputType) + b: Any => t(ByteBuffer.wrap(b.asInstanceOf[Array[Byte]])).toInt case _ => val t = Transforms.bucket[Any](icebergInputType, numBuckets) a: Any => t(a).toInt diff --git a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java index a698b9cc6944..6fbc3a31aa19 100644 --- a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java +++ b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java @@ -94,6 +94,7 @@ import org.apache.iceberg.spark.SparkTableUtil; import org.apache.iceberg.spark.SparkTestBase; import org.apache.iceberg.spark.actions.RewriteDataFilesSparkAction.RewriteExecutionContext; +import org.apache.iceberg.spark.data.TestHelpers; import org.apache.iceberg.spark.source.ThreeColumnRecord; import org.apache.iceberg.types.Comparators; import org.apache.iceberg.types.Conversions; @@ -253,9 +254,7 @@ public void testBinPackWithDeletes() throws Exception { shouldHaveFiles(table, 8); table.refresh(); - CloseableIterable tasks = table.newScan().planFiles(); - List dataFiles = - Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); + List dataFiles = TestHelpers.dataFiles(table); int total = (int) dataFiles.stream().mapToLong(ContentFile::recordCount).sum(); RowDelta rowDelta = table.newRowDelta(); @@ -299,9 +298,7 @@ public void testBinPackWithDeleteAllData() { shouldHaveFiles(table, 1); table.refresh(); - CloseableIterable tasks = table.newScan().planFiles(); - List dataFiles = - Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); + List dataFiles = TestHelpers.dataFiles(table); int total = (int) dataFiles.stream().mapToLong(ContentFile::recordCount).sum(); RowDelta rowDelta = table.newRowDelta(); diff --git a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java index 78b3df230738..82d0f0daf595 100644 --- a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java +++ b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java @@ -49,6 +49,8 @@ import org.apache.iceberg.ManifestFile; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.relocated.com.google.common.collect.Streams; @@ -781,14 +783,18 @@ public static List deleteManifests(Table table) { return table.currentSnapshot().deleteManifests(table.io()); } - public static Set dataFiles(Table table) { - Set dataFiles = Sets.newHashSet(); + public static List dataFiles(Table table) { + return dataFiles(table, null); + } - for (FileScanTask task : table.newScan().planFiles()) { - dataFiles.add(task.file()); + public static List dataFiles(Table table, String branch) { + TableScan scan = table.newScan(); + if (branch != null) { + scan.useRef(branch); } - return dataFiles; + CloseableIterable tasks = scan.includeColumnStats().planFiles(); + return Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); } public static Set deleteFiles(Table table) { diff --git a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java index 06e4965a068a..dac560eb1a86 100644 --- a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java +++ b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java @@ -33,10 +33,12 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.hadoop.HadoopTables; import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.math.LongMath; @@ -53,6 +55,7 @@ import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.assertj.core.api.Assertions; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -428,8 +431,14 @@ public void testExtraSnapshotMetadataWithSQL() throws InterruptedException, IOEx Thread writerThread = new Thread( () -> { - Map properties = Maps.newHashMap(); - properties.put("writer-thread", String.valueOf(Thread.currentThread().getName())); + Map properties = + ImmutableMap.of( + "writer-thread", + String.valueOf(Thread.currentThread().getName()), + SnapshotSummary.EXTRA_METADATA_PREFIX + "extra-key", + "someValue", + SnapshotSummary.EXTRA_METADATA_PREFIX + "another-key", + "anotherValue"); CommitMetadata.withCommitProperties( properties, () -> { @@ -445,8 +454,10 @@ public void testExtraSnapshotMetadataWithSQL() throws InterruptedException, IOEx List snapshots = Lists.newArrayList(table.snapshots()); Assert.assertEquals(2, snapshots.size()); Assert.assertNull(snapshots.get(0).summary().get("writer-thread")); - Assert.assertEquals( - "test-extra-commit-message-writer-thread", snapshots.get(1).summary().get("writer-thread")); + Assertions.assertThat(snapshots.get(1).summary()) + .containsEntry("writer-thread", "test-extra-commit-message-writer-thread") + .containsEntry("extra-key", "someValue") + .containsEntry("another-key", "anotherValue"); } @Test @@ -462,8 +473,14 @@ public void testExtraSnapshotMetadataWithDelete() Thread writerThread = new Thread( () -> { - Map properties = Maps.newHashMap(); - properties.put("writer-thread", String.valueOf(Thread.currentThread().getName())); + Map properties = + ImmutableMap.of( + "writer-thread", + String.valueOf(Thread.currentThread().getName()), + SnapshotSummary.EXTRA_METADATA_PREFIX + "extra-key", + "someValue", + SnapshotSummary.EXTRA_METADATA_PREFIX + "another-key", + "anotherValue"); CommitMetadata.withCommitProperties( properties, () -> { @@ -480,7 +497,9 @@ public void testExtraSnapshotMetadataWithDelete() List snapshots = Lists.newArrayList(table.snapshots()); Assert.assertEquals(2, snapshots.size()); Assert.assertNull(snapshots.get(0).summary().get("writer-thread")); - Assert.assertEquals( - "test-extra-commit-message-delete-thread", snapshots.get(1).summary().get("writer-thread")); + Assertions.assertThat(snapshots.get(1).summary()) + .containsEntry("writer-thread", "test-extra-commit-message-delete-thread") + .containsEntry("extra-key", "someValue") + .containsEntry("another-key", "anotherValue"); } } diff --git a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index b89b78b23f02..5250b217d44c 100644 --- a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -25,6 +25,7 @@ import java.io.IOException; import java.io.UncheckedIOException; +import java.util.Arrays; import java.util.Comparator; import java.util.List; import java.util.StringJoiner; @@ -45,6 +46,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.actions.DeleteOrphanFiles; +import org.apache.iceberg.actions.RewriteManifests; import org.apache.iceberg.avro.Avro; import org.apache.iceberg.avro.AvroSchemaUtil; import org.apache.iceberg.catalog.TableIdentifier; @@ -1222,7 +1224,7 @@ public void testAllManifestsTable() { @Test public void testUnpartitionedPartitionsTable() { TableIdentifier tableIdentifier = TableIdentifier.of("db", "unpartitioned_partitions_test"); - createTable(tableIdentifier, SCHEMA, PartitionSpec.unpartitioned()); + Table table = createTable(tableIdentifier, SCHEMA, PartitionSpec.unpartitioned()); Dataset df = spark.createDataFrame(Lists.newArrayList(new SimpleRecord(1, "a")), SimpleRecord.class); @@ -1237,6 +1239,11 @@ public void testUnpartitionedPartitionsTable() { Types.StructType.of( required(2, "record_count", Types.LongType.get(), "Count of records in data files"), required(3, "file_count", Types.IntegerType.get(), "Count of data files"), + required( + 11, + "total_data_file_size_in_bytes", + Types.LongType.get(), + "Total size in bytes of data files"), required( 5, "position_delete_record_count", @@ -1256,7 +1263,17 @@ public void testUnpartitionedPartitionsTable() { 8, "equality_delete_file_count", Types.IntegerType.get(), - "Count of equality delete files")); + "Count of equality delete files"), + optional( + 9, + "last_updated_at", + Types.TimestampType.withZone(), + "Commit time of snapshot that last updated this partition"), + optional( + 10, + "last_updated_snapshot_id", + Types.LongType.get(), + "Id of snapshot that last updated this partition")); Table partitionsTable = loadTable(tableIdentifier, "partitions"); @@ -1269,8 +1286,13 @@ public void testUnpartitionedPartitionsTable() { new GenericRecordBuilder(AvroSchemaUtil.convert(partitionsTable.schema(), "partitions")); GenericData.Record expectedRow = builder + .set("last_updated_at", table.currentSnapshot().timestampMillis() * 1000) + .set("last_updated_snapshot_id", table.currentSnapshot().snapshotId()) .set("record_count", 1L) .set("file_count", 1) + .set( + "total_data_file_size_in_bytes", + totalSizeInBytes(table.currentSnapshot().addedDataFiles(table.io()))) .set("position_delete_record_count", 0L) .set("position_delete_file_count", 0) .set("equality_delete_record_count", 0L) @@ -1314,6 +1336,9 @@ public void testPartitionsTable() { .mode("append") .save(loadLocation(tableIdentifier)); + table.refresh(); + long secondCommitId = table.currentSnapshot().snapshotId(); + List actual = spark .read() @@ -1334,22 +1359,32 @@ public void testPartitionsTable() { .set("partition", partitionBuilder.set("id", 1).build()) .set("record_count", 1L) .set("file_count", 1) + .set( + "total_data_file_size_in_bytes", + totalSizeInBytes(table.snapshot(firstCommitId).addedDataFiles(table.io()))) .set("position_delete_record_count", 0L) .set("position_delete_file_count", 0) .set("equality_delete_record_count", 0L) .set("equality_delete_file_count", 0) .set("spec_id", 0) + .set("last_updated_at", table.snapshot(firstCommitId).timestampMillis() * 1000) + .set("last_updated_snapshot_id", firstCommitId) .build()); expected.add( builder .set("partition", partitionBuilder.set("id", 2).build()) .set("record_count", 1L) .set("file_count", 1) + .set( + "total_data_file_size_in_bytes", + totalSizeInBytes(table.snapshot(secondCommitId).addedDataFiles(table.io()))) .set("position_delete_record_count", 0L) .set("position_delete_file_count", 0) .set("equality_delete_record_count", 0L) .set("equality_delete_file_count", 0) .set("spec_id", 0) + .set("last_updated_at", table.snapshot(secondCommitId).timestampMillis() * 1000) + .set("last_updated_snapshot_id", secondCommitId) .build()); Assert.assertEquals("Partitions table should have two rows", 2, expected.size()); @@ -1392,13 +1427,160 @@ public void testPartitionsTable() { .load(loadLocation(tableIdentifier, "partitions")) .filter("partition.id < 2 or record_count=1") .collectAsList(); - Assert.assertEquals("Actual results should have one row", 2, nonFiltered.size()); + Assert.assertEquals("Actual results should have two row", 2, nonFiltered.size()); for (int i = 0; i < 2; i += 1) { TestHelpers.assertEqualsSafe( partitionsTable.schema().asStruct(), expected.get(i), actual.get(i)); } } + @Test + public void testPartitionsTableLastUpdatedSnapshot() { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "partitions_test"); + Table table = createTable(tableIdentifier, SCHEMA, SPEC); + Table partitionsTable = loadTable(tableIdentifier, "partitions"); + Dataset df1 = + spark.createDataFrame( + Lists.newArrayList(new SimpleRecord(1, "1"), new SimpleRecord(2, "2")), + SimpleRecord.class); + Dataset df2 = + spark.createDataFrame(Lists.newArrayList(new SimpleRecord(2, "20")), SimpleRecord.class); + + df1.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + table.refresh(); + long firstCommitId = table.currentSnapshot().snapshotId(); + + // add a second file + df2.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + table.refresh(); + long secondCommitId = table.currentSnapshot().snapshotId(); + + // check if rewrite manifest does not override metadata about data file's creating snapshot + RewriteManifests.Result rewriteManifestResult = + SparkActions.get().rewriteManifests(table).execute(); + Assert.assertEquals( + "rewrite replaced 2 manifests", + 2, + Iterables.size(rewriteManifestResult.rewrittenManifests())); + Assert.assertEquals( + "rewrite added 1 manifests", 1, Iterables.size(rewriteManifestResult.addedManifests())); + + List actual = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "partitions")) + .orderBy("partition.id") + .collectAsList(); + + GenericRecordBuilder builder = + new GenericRecordBuilder(AvroSchemaUtil.convert(partitionsTable.schema(), "partitions")); + GenericRecordBuilder partitionBuilder = + new GenericRecordBuilder( + AvroSchemaUtil.convert( + partitionsTable.schema().findType("partition").asStructType(), "partition")); + + List dataFiles = TestHelpers.dataFiles(table); + assertDataFilePartitions(dataFiles, Arrays.asList(1, 2, 2)); + + List expected = Lists.newArrayList(); + expected.add( + builder + .set("partition", partitionBuilder.set("id", 1).build()) + .set("record_count", 1L) + .set("file_count", 1) + .set("total_data_file_size_in_bytes", dataFiles.get(0).fileSizeInBytes()) + .set("position_delete_record_count", 0L) + .set("position_delete_file_count", 0) + .set("equality_delete_record_count", 0L) + .set("equality_delete_file_count", 0) + .set("spec_id", 0) + .set("last_updated_at", table.snapshot(firstCommitId).timestampMillis() * 1000) + .set("last_updated_snapshot_id", firstCommitId) + .build()); + expected.add( + builder + .set("partition", partitionBuilder.set("id", 2).build()) + .set("record_count", 2L) + .set("file_count", 2) + .set( + "total_data_file_size_in_bytes", + dataFiles.get(1).fileSizeInBytes() + dataFiles.get(2).fileSizeInBytes()) + .set("position_delete_record_count", 0L) + .set("position_delete_file_count", 0) + .set("equality_delete_record_count", 0L) + .set("equality_delete_file_count", 0) + .set("spec_id", 0) + .set("last_updated_at", table.snapshot(secondCommitId).timestampMillis() * 1000) + .set("last_updated_snapshot_id", secondCommitId) + .build()); + + Assert.assertEquals("Partitions table should have two rows", 2, expected.size()); + Assert.assertEquals("Actual results should have two rows", 2, actual.size()); + for (int i = 0; i < 2; i += 1) { + TestHelpers.assertEqualsSafe( + partitionsTable.schema().asStruct(), expected.get(i), actual.get(i)); + } + + // check predicate push down + List filtered = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "partitions")) + .filter("partition.id < 2") + .collectAsList(); + Assert.assertEquals("Actual results should have one row", 1, filtered.size()); + TestHelpers.assertEqualsSafe( + partitionsTable.schema().asStruct(), expected.get(0), filtered.get(0)); + + // check for snapshot expiration + // if snapshot with firstCommitId is expired, + // we expect the partition of id=1 will no longer have last updated timestamp and snapshotId + SparkActions.get().expireSnapshots(table).expireSnapshotId(firstCommitId).execute(); + GenericData.Record newPartitionRecord = + builder + .set("partition", partitionBuilder.set("id", 1).build()) + .set("record_count", 1L) + .set("file_count", 1) + .set("total_data_file_size_in_bytes", dataFiles.get(0).fileSizeInBytes()) + .set("position_delete_record_count", 0L) + .set("position_delete_file_count", 0) + .set("equality_delete_record_count", 0L) + .set("equality_delete_file_count", 0) + .set("spec_id", 0) + .set("last_updated_at", null) + .set("last_updated_snapshot_id", null) + .build(); + expected.remove(0); + expected.add(0, newPartitionRecord); + + List actualAfterSnapshotExpiration = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "partitions")) + .collectAsList(); + Assert.assertEquals( + "Actual results should have two row", 2, actualAfterSnapshotExpiration.size()); + for (int i = 0; i < 2; i += 1) { + TestHelpers.assertEqualsSafe( + partitionsTable.schema().asStruct(), + expected.get(i), + actualAfterSnapshotExpiration.get(i)); + } + } + @Test public void testPartitionsTableDeleteStats() { TableIdentifier tableIdentifier = TableIdentifier.of("db", "partitions_test"); @@ -1416,6 +1598,7 @@ public void testPartitionsTableDeleteStats() { .save(loadLocation(tableIdentifier)); table.refresh(); + long firstCommitId = table.currentSnapshot().snapshotId(); // add a second file df2.select("id", "data") @@ -1428,6 +1611,8 @@ public void testPartitionsTableDeleteStats() { table.updateProperties().set(TableProperties.FORMAT_VERSION, "2").commit(); DeleteFile deleteFile = writePosDeleteFile(table); table.newRowDelta().addDeletes(deleteFile).commit(); + table.refresh(); + long posDeleteCommitId = table.currentSnapshot().snapshotId(); List actual = spark @@ -1450,23 +1635,34 @@ public void testPartitionsTableDeleteStats() { .set("partition", partitionBuilder.set("id", 1).build()) .set("record_count", 1L) .set("file_count", 1) + .set( + "total_data_file_size_in_bytes", + totalSizeInBytes(table.snapshot(firstCommitId).addedDataFiles(table.io()))) .set("position_delete_record_count", 0L) .set("position_delete_file_count", 0) .set("equality_delete_record_count", 0L) .set("equality_delete_file_count", 0) .set("spec_id", 0) + .set("last_updated_at", table.snapshot(firstCommitId).timestampMillis() * 1000) + .set("last_updated_snapshot_id", firstCommitId) .build()); expected.add( builder .set("partition", partitionBuilder.set("id", 2).build()) .set("record_count", 1L) .set("file_count", 1) + .set( + "total_data_file_size_in_bytes", + totalSizeInBytes(table.snapshot(firstCommitId).addedDataFiles(table.io()))) .set("position_delete_record_count", 1L) // should be incremented now .set("position_delete_file_count", 1) // should be incremented now .set("equality_delete_record_count", 0L) .set("equality_delete_file_count", 0) .set("spec_id", 0) + .set("last_updated_at", table.snapshot(posDeleteCommitId).timestampMillis() * 1000) + .set("last_updated_snapshot_id", posDeleteCommitId) .build()); + for (int i = 0; i < 2; i += 1) { TestHelpers.assertEqualsSafe( partitionsTable.schema().asStruct(), expected.get(i), actual.get(i)); @@ -1475,6 +1671,8 @@ public void testPartitionsTableDeleteStats() { // test equality delete DeleteFile eqDeleteFile = writeEqDeleteFile(table); table.newRowDelta().addDeletes(eqDeleteFile).commit(); + table.refresh(); + long eqDeleteCommitId = table.currentSnapshot().snapshotId(); actual = spark .read() @@ -1495,6 +1693,8 @@ public void testPartitionsTableDeleteStats() { .set("equality_delete_record_count", 1L) // should be incremented now .set("equality_delete_file_count", 1) // should be incremented now .set("spec_id", 0) + .set("last_updated_at", table.snapshot(eqDeleteCommitId).timestampMillis() * 1000) + .set("last_updated_snapshot_id", eqDeleteCommitId) .build()); for (int i = 0; i < 2; i += 1) { TestHelpers.assertEqualsSafe( @@ -1989,4 +2189,23 @@ private DeleteFile writeEqDeleteFile(Table table) { throw new RuntimeException(e); } } + + private long totalSizeInBytes(Iterable dataFiles) { + return Lists.newArrayList(dataFiles).stream().mapToLong(DataFile::fileSizeInBytes).sum(); + } + + private void assertDataFilePartitions( + List dataFiles, List expectedPartitionIds) { + Assert.assertEquals( + "Table should have " + expectedPartitionIds.size() + " data files", + expectedPartitionIds.size(), + dataFiles.size()); + + for (int i = 0; i < dataFiles.size(); ++i) { + Assert.assertEquals( + "Data file should have partition of id " + expectedPartitionIds.get(i), + expectedPartitionIds.get(i).intValue(), + dataFiles.get(i).partition().get(0, Integer.class).intValue()); + } + } } diff --git a/spark/v3.3/build.gradle b/spark/v3.3/build.gradle index f3cb291033d8..efa4942fa281 100644 --- a/spark/v3.3/build.gradle +++ b/spark/v3.3/build.gradle @@ -230,6 +230,7 @@ project(":iceberg-spark:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersio integrationImplementation "org.apache.spark:spark-hive_${scalaVersion}:${sparkVersion}" integrationImplementation 'org.junit.vintage:junit-vintage-engine' integrationImplementation 'org.slf4j:slf4j-simple' + integrationImplementation 'org.assertj:assertj-core' integrationImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') integrationImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') integrationImplementation project(path: ":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}", configuration: 'testArtifacts') diff --git a/spark/v3.3/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala b/spark/v3.3/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala index f758cb08fd3d..2e438de2b8cd 100644 --- a/spark/v3.3/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala +++ b/spark/v3.3/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala @@ -116,6 +116,7 @@ class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergS val branchRetention = branchOptionsContext.flatMap(branchOptions => Option(branchOptions.refRetain())) val branchRefAgeMs = branchRetention.map(retain => TimeUnit.valueOf(retain.timeUnit().getText.toUpperCase(Locale.ENGLISH)).toMillis(retain.number().getText.toLong)) + val create = createOrReplaceBranchClause.CREATE() != null val replace = ctx.createReplaceBranchClause().REPLACE() != null val ifNotExists = createOrReplaceBranchClause.EXISTS() != null @@ -130,6 +131,7 @@ class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergS typedVisit[Seq[String]](ctx.multipartIdentifier), branchName.getText, branchOptions, + create, replace, ifNotExists) } @@ -153,12 +155,14 @@ class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergS tagRefAgeMs ) + val create = createTagClause.CREATE() != null val replace = createTagClause.REPLACE() != null val ifNotExists = createTagClause.EXISTS() != null CreateOrReplaceTag(typedVisit[Seq[String]](ctx.multipartIdentifier), tagName, tagOptions, + create, replace, ifNotExists) } diff --git a/spark/v3.3/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala b/spark/v3.3/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala index 2a22484499cf..b7981a3c7a0d 100644 --- a/spark/v3.3/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala +++ b/spark/v3.3/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala @@ -25,6 +25,7 @@ case class CreateOrReplaceBranch( table: Seq[String], branch: String, branchOptions: BranchOptions, + create: Boolean, replace: Boolean, ifNotExists: Boolean) extends LeafCommand { diff --git a/spark/v3.3/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala b/spark/v3.3/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala index e48f7d8ed04c..6e7db84a90fb 100644 --- a/spark/v3.3/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala +++ b/spark/v3.3/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala @@ -25,6 +25,7 @@ case class CreateOrReplaceTag( table: Seq[String], tag: String, tagOptions: TagOptions, + create: Boolean, replace: Boolean, ifNotExists: Boolean) extends LeafCommand { diff --git a/spark/v3.3/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala b/spark/v3.3/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala index 6457875b15a4..d4328d4b9227 100644 --- a/spark/v3.3/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala +++ b/spark/v3.3/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala @@ -32,6 +32,7 @@ case class CreateOrReplaceBranchExec( ident: Identifier, branch: String, branchOptions: BranchOptions, + create: Boolean, replace: Boolean, ifNotExists: Boolean) extends LeafV2CommandExec { @@ -51,15 +52,18 @@ case class CreateOrReplaceBranchExec( "Cannot complete create or replace branch operation on %s, main has no snapshot", ident) val manageSnapshots = iceberg.table().manageSnapshots() - if (!replace) { - val ref = iceberg.table().refs().get(branch) - if (ref != null && ifNotExists) { + val refExists = null != iceberg.table().refs().get(branch) + + if (create && replace && !refExists) { + manageSnapshots.createBranch(branch, snapshotId) + } else if (replace) { + manageSnapshots.replaceBranch(branch, snapshotId) + } else { + if (refExists && ifNotExists) { return Nil } manageSnapshots.createBranch(branch, snapshotId) - } else { - manageSnapshots.replaceBranch(branch, snapshotId) } if (branchOptions.numSnapshots.nonEmpty) { diff --git a/spark/v3.3/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala b/spark/v3.3/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala index 7ca193d1b156..372cd7548632 100644 --- a/spark/v3.3/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala +++ b/spark/v3.3/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala @@ -31,6 +31,7 @@ case class CreateOrReplaceTagExec( ident: Identifier, tag: String, tagOptions: TagOptions, + create: Boolean, replace: Boolean, ifNotExists: Boolean) extends LeafV2CommandExec { @@ -50,15 +51,18 @@ case class CreateOrReplaceTagExec( "Cannot complete create or replace tag operation on %s, main has no snapshot", ident) val manageSnapshot = iceberg.table.manageSnapshots() - if (!replace) { - val ref = iceberg.table().refs().get(tag) - if (ref != null && ifNotExists) { + val refExists = null != iceberg.table().refs().get(tag) + + if (create && replace && !refExists) { + manageSnapshot.createTag(tag, snapshotId) + } else if (replace) { + manageSnapshot.replaceTag(tag, snapshotId) + } else { + if (refExists && ifNotExists) { return Nil } manageSnapshot.createTag(tag, snapshotId) - } else { - manageSnapshot.replaceTag(tag, snapshotId) } if (tagOptions.snapshotRefRetain.nonEmpty) { diff --git a/spark/v3.3/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala b/spark/v3.3/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala index b35e3a8d99ba..14e59b7f988e 100644 --- a/spark/v3.3/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala +++ b/spark/v3.3/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala @@ -66,11 +66,12 @@ case class ExtendedDataSourceV2Strategy(spark: SparkSession) extends Strategy wi AddPartitionFieldExec(catalog, ident, transform, name) :: Nil case CreateOrReplaceBranch( - IcebergCatalogAndIdentifier(catalog, ident), branch, branchOptions, replace, ifNotExists) => - CreateOrReplaceBranchExec(catalog, ident, branch, branchOptions, replace, ifNotExists) :: Nil + IcebergCatalogAndIdentifier(catalog, ident), branch, branchOptions, create, replace, ifNotExists) => + CreateOrReplaceBranchExec(catalog, ident, branch, branchOptions, create, replace, ifNotExists) :: Nil - case CreateOrReplaceTag(IcebergCatalogAndIdentifier(catalog, ident), tag, tagOptions, replace, ifNotExists) => - CreateOrReplaceTagExec(catalog, ident, tag, tagOptions, replace, ifNotExists) :: Nil + case CreateOrReplaceTag( + IcebergCatalogAndIdentifier(catalog, ident), tag, tagOptions, create, replace, ifNotExists) => + CreateOrReplaceTagExec(catalog, ident, tag, tagOptions, create, replace, ifNotExists) :: Nil case DropBranch(IcebergCatalogAndIdentifier(catalog, ident), branch, ifExists) => DropBranchExec(catalog, ident, branch, ifExists) :: Nil diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java index 5dd8c751863a..fcc124dee5b9 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; @@ -292,6 +294,71 @@ public void testDropBranchIfExists() { Assert.assertNull(ref); } + @Test + public void createOrReplace() throws NoSuchTableException { + Table table = insertRows(); + long first = table.currentSnapshot().snapshotId(); + String branchName = "b1"; + insertRows(); + long second = table.currentSnapshot().snapshotId(); + table.manageSnapshots().createBranch(branchName, second).commit(); + + sql( + "ALTER TABLE %s CREATE OR REPLACE BRANCH %s AS OF VERSION %d", + tableName, branchName, first); + table.refresh(); + assertThat(table.refs().get(branchName).snapshotId()).isEqualTo(second); + } + + @Test + public void createOrReplaceWithNonExistingBranch() throws NoSuchTableException { + Table table = insertRows(); + String branchName = "b1"; + insertRows(); + long snapshotId = table.currentSnapshot().snapshotId(); + + sql( + "ALTER TABLE %s CREATE OR REPLACE BRANCH %s AS OF VERSION %d", + tableName, branchName, snapshotId); + table.refresh(); + assertThat(table.refs().get(branchName).snapshotId()).isEqualTo(snapshotId); + } + + @Test + public void replaceBranch() throws NoSuchTableException { + Table table = insertRows(); + long first = table.currentSnapshot().snapshotId(); + String branchName = "b1"; + long expectedMaxRefAgeMs = 1000; + table + .manageSnapshots() + .createBranch(branchName, first) + .setMaxRefAgeMs(branchName, expectedMaxRefAgeMs) + .commit(); + + insertRows(); + long second = table.currentSnapshot().snapshotId(); + + sql("ALTER TABLE %s REPLACE BRANCH %s AS OF VERSION %d", tableName, branchName, second); + table.refresh(); + SnapshotRef ref = table.refs().get(branchName); + assertThat(ref.snapshotId()).isEqualTo(second); + assertThat(ref.maxRefAgeMs()).isEqualTo(expectedMaxRefAgeMs); + } + + @Test + public void replaceBranchDoesNotExist() throws NoSuchTableException { + Table table = insertRows(); + + Assertions.assertThatThrownBy( + () -> + sql( + "ALTER TABLE %s REPLACE BRANCH %s AS OF VERSION %d", + tableName, "someBranch", table.currentSnapshot().snapshotId())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Branch does not exist: someBranch"); + } + private Table insertRows() throws NoSuchTableException { List records = ImmutableList.of(new SimpleRecord(1, "a"), new SimpleRecord(2, "b")); diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java index dc12b0145d50..0a4a1073c3f9 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.extensions; +import static org.junit.Assert.assertThrows; + import java.util.List; import java.util.Map; import org.apache.iceberg.ChangelogOperation; @@ -45,13 +47,13 @@ public void removeTable() { sql("DROP TABLE IF EXISTS %s", tableName); } - public void createTableWith2Columns() { + public void createTableWithTwoColumns() { sql("CREATE TABLE %s (id INT, data STRING) USING iceberg", tableName); sql("ALTER TABLE %s SET TBLPROPERTIES ('format-version'='%d')", tableName, 1); sql("ALTER TABLE %s ADD PARTITION FIELD data", tableName); } - private void createTableWith3Columns() { + private void createTableWithThreeColumns() { sql("CREATE TABLE %s (id INT, data STRING, age INT) USING iceberg", tableName); sql("ALTER TABLE %s SET TBLPROPERTIES ('format-version'='%d')", tableName, 1); sql("ALTER TABLE %s ADD PARTITION FIELD id", tableName); @@ -65,7 +67,7 @@ private void createTableWithIdentifierField() { @Test public void testCustomizedViewName() { - createTableWith2Columns(); + createTableWithTwoColumns(); sql("INSERT INTO %s VALUES (1, 'a')", tableName); sql("INSERT INTO %s VALUES (2, 'b')", tableName); @@ -98,7 +100,7 @@ public void testCustomizedViewName() { @Test public void testNoSnapshotIdInput() { - createTableWith2Columns(); + createTableWithTwoColumns(); sql("INSERT INTO %s VALUES (1, 'a')", tableName); Table table = validationCatalog.loadTable(tableIdent); Snapshot snap0 = table.currentSnapshot(); @@ -129,7 +131,7 @@ public void testNoSnapshotIdInput() { @Test public void testTimestampsBasedQuery() { - createTableWith2Columns(); + createTableWithTwoColumns(); long beginning = System.currentTimeMillis(); sql("INSERT INTO %s VALUES (1, 'a')", tableName); @@ -189,7 +191,7 @@ public void testTimestampsBasedQuery() { @Test public void testWithCarryovers() { - createTableWith2Columns(); + createTableWithTwoColumns(); sql("INSERT INTO %s VALUES (1, 'a')", tableName); Table table = validationCatalog.loadTable(tableIdent); Snapshot snap0 = table.currentSnapshot(); @@ -224,7 +226,7 @@ public void testWithCarryovers() { @Test public void testUpdate() { - createTableWith2Columns(); + createTableWithTwoColumns(); sql("ALTER TABLE %s DROP PARTITION FIELD data", tableName); sql("ALTER TABLE %s ADD PARTITION FIELD id", tableName); @@ -283,7 +285,7 @@ public void testUpdateWithIdentifierField() { @Test public void testUpdateWithFilter() { - createTableWith2Columns(); + createTableWithTwoColumns(); sql("ALTER TABLE %s DROP PARTITION FIELD data", tableName); sql("ALTER TABLE %s ADD PARTITION FIELD id", tableName); @@ -315,7 +317,7 @@ public void testUpdateWithFilter() { @Test public void testUpdateWithMultipleIdentifierColumns() { - createTableWith3Columns(); + createTableWithThreeColumns(); sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11)", tableName); Table table = validationCatalog.loadTable(tableIdent); @@ -347,7 +349,7 @@ public void testUpdateWithMultipleIdentifierColumns() { @Test public void testRemoveCarryOvers() { - createTableWith3Columns(); + createTableWithThreeColumns(); sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11), (2, 'e', 12)", tableName); Table table = validationCatalog.loadTable(tableIdent); @@ -381,7 +383,7 @@ public void testRemoveCarryOvers() { @Test public void testRemoveCarryOversWithoutUpdatedRows() { - createTableWith3Columns(); + createTableWithThreeColumns(); sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11), (2, 'e', 12)", tableName); Table table = validationCatalog.loadTable(tableIdent); @@ -411,9 +413,74 @@ public void testRemoveCarryOversWithoutUpdatedRows() { sql("select * from %s order by _change_ordinal, id, data", viewName)); } + @Test + public void testNetChangesWithRemoveCarryOvers() { + // partitioned by id + createTableWithThreeColumns(); + + // insert rows: (1, 'a', 12) (2, 'b', 11) (2, 'e', 12) + sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11), (2, 'e', 12)", tableName); + Table table = validationCatalog.loadTable(tableIdent); + Snapshot snap1 = table.currentSnapshot(); + + // delete rows: (2, 'b', 11) (2, 'e', 12) + // insert rows: (3, 'c', 13) (2, 'd', 11) (2, 'e', 12) + sql("INSERT OVERWRITE %s VALUES (3, 'c', 13), (2, 'd', 11), (2, 'e', 12)", tableName); + table.refresh(); + Snapshot snap2 = table.currentSnapshot(); + + // delete rows: (2, 'd', 11) (2, 'e', 12) (3, 'c', 13) + // insert rows: (3, 'c', 15) (2, 'e', 12) + sql("INSERT OVERWRITE %s VALUES (3, 'c', 15), (2, 'e', 12)", tableName); + table.refresh(); + Snapshot snap3 = table.currentSnapshot(); + + // test with all snapshots + List returns = + sql( + "CALL %s.system.create_changelog_view(table => '%s', net_changes => true)", + catalogName, tableName); + + String viewName = (String) returns.get(0)[0]; + + assertEquals( + "Rows should match", + ImmutableList.of( + row(1, "a", 12, INSERT, 0, snap1.snapshotId()), + row(3, "c", 15, INSERT, 2, snap3.snapshotId()), + row(2, "e", 12, INSERT, 2, snap3.snapshotId())), + sql("select * from %s order by _change_ordinal, data", viewName)); + + // test with snap2 and snap3 + sql( + "CALL %s.system.create_changelog_view(table => '%s', " + + "options => map('start-snapshot-id','%s'), " + + "net_changes => true)", + catalogName, tableName, snap1.snapshotId()); + + assertEquals( + "Rows should match", + ImmutableList.of( + row(2, "b", 11, DELETE, 0, snap2.snapshotId()), + row(3, "c", 15, INSERT, 1, snap3.snapshotId())), + sql("select * from %s order by _change_ordinal, data", viewName)); + } + + @Test + public void testNetChangesWithComputeUpdates() { + createTableWithTwoColumns(); + assertThrows( + "Should fail because net_changes is not supported with computing updates", + IllegalArgumentException.class, + () -> + sql( + "CALL %s.system.create_changelog_view(table => '%s', identifier_columns => array('id'), net_changes => true)", + catalogName, tableName)); + } + @Test public void testNotRemoveCarryOvers() { - createTableWith3Columns(); + createTableWithThreeColumns(); sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11), (2, 'e', 12)", tableName); Table table = validationCatalog.loadTable(tableIdent); diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java index 463bf2a27b0e..934f1b91f726 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java @@ -25,12 +25,12 @@ import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES; import static org.apache.iceberg.TableProperties.SPLIT_SIZE; import static org.apache.spark.sql.functions.lit; +import static org.assertj.core.api.Assumptions.assumeThat; import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -46,6 +46,7 @@ import org.apache.iceberg.ManifestFile; import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.GenericRecord; @@ -152,11 +153,11 @@ public void testDeleteFileThenMetadataDelete() throws Exception { // Metadata Delete Table table = Spark3Util.loadIcebergTable(spark, tableName); - Set dataFilesBefore = TestHelpers.dataFiles(table, branch); + List dataFilesBefore = TestHelpers.dataFiles(table, branch); sql("DELETE FROM %s AS t WHERE t.id = 1", commitTarget()); - Set dataFilesAfter = TestHelpers.dataFiles(table, branch); + List dataFilesAfter = TestHelpers.dataFiles(table, branch); Assert.assertTrue( "Data file should have been removed", dataFilesBefore.size() > dataFilesAfter.size()); @@ -1135,6 +1136,42 @@ public void testDeleteToWapBranchWithTableBranchIdentifier() throws NoSuchTableE branch))); } + @Test + public void testDeleteToCustomWapBranchWithoutWhereClause() throws NoSuchTableException { + assumeThat(branch) + .as("Run only if custom WAP branch is not main") + .isNotNull() + .isNotEqualTo(SnapshotRef.MAIN_BRANCH); + + createAndInitPartitionedTable(); + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s' = 'true')", + tableName, TableProperties.WRITE_AUDIT_PUBLISH_ENABLED); + append(tableName, new Employee(0, "hr"), new Employee(1, "hr"), new Employee(2, "hr")); + createBranchIfNeeded(); + + withSQLConf( + ImmutableMap.of(SparkSQLProperties.WAP_BRANCH, branch), + () -> { + sql("DELETE FROM %s t WHERE id=1", tableName); + Assertions.assertThat(spark.table(tableName).count()).isEqualTo(2L); + Assertions.assertThat(spark.table(tableName + ".branch_" + branch).count()).isEqualTo(2L); + Assertions.assertThat(spark.table(tableName + ".branch_main").count()) + .as("Should not modify main branch") + .isEqualTo(3L); + }); + withSQLConf( + ImmutableMap.of(SparkSQLProperties.WAP_BRANCH, branch), + () -> { + sql("DELETE FROM %s t", tableName); + Assertions.assertThat(spark.table(tableName).count()).isEqualTo(0L); + Assertions.assertThat(spark.table(tableName + ".branch_" + branch).count()).isEqualTo(0L); + Assertions.assertThat(spark.table(tableName + ".branch_main").count()) + .as("Should not modify main branch") + .isEqualTo(3L); + }); + } + // TODO: multiple stripes for ORC protected void createAndInitPartitionedTable() { diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java index 8b2950b74f8d..b8b5df099089 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java @@ -103,7 +103,7 @@ public void testMigrateWithOptions() throws IOException { ImmutableList.of(row(1L, "a"), row(1L, "a")), sql("SELECT * FROM %s ORDER BY id", tableName)); - sql("DROP TABLE IF EXISTS %s", tableName + "_BACKUP_"); + sql("DROP TABLE IF EXISTS %s", tableName + "_BACKUP_"); } @Test @@ -184,4 +184,43 @@ public void testInvalidMigrateCases() { "Cannot handle an empty identifier", () -> sql("CALL %s.system.migrate('')", catalogName)); } + + @Test + public void testMigratePartitionWithSpecialCharacter() throws IOException { + Assume.assumeTrue(catalogName.equals("spark_catalog")); + String location = temp.newFolder().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string, dt date) USING parquet " + + "PARTITIONED BY (data, dt) LOCATION '%s'", + tableName, location); + sql("INSERT INTO TABLE %s VALUES (1, '2023/05/30', date '2023-05-30')", tableName); + Object result = scalarSql("CALL %s.system.migrate('%s')", catalogName, tableName); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1L, "2023/05/30", java.sql.Date.valueOf("2023-05-30"))), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @Test + public void testMigrateEmptyPartitionedTable() throws Exception { + Assume.assumeTrue(catalogName.equals("spark_catalog")); + String location = temp.newFolder().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet PARTITIONED BY (id) LOCATION '%s'", + tableName, location); + Object result = scalarSql("CALL %s.system.migrate('%s')", catalogName, tableName); + Assert.assertEquals(0L, result); + } + + @Test + public void testMigrateEmptyTable() throws Exception { + Assume.assumeTrue(catalogName.equals("spark_catalog")); + String location = temp.newFolder().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + tableName, location); + Object result = scalarSql("CALL %s.system.migrate('%s')", catalogName, tableName); + Assert.assertEquals(0L, result); + } } diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java new file mode 100644 index 000000000000..da9b61f4d74d --- /dev/null +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java @@ -0,0 +1,417 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.extensions; + +import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; +import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT; +import static org.apache.spark.sql.functions.expr; +import static org.apache.spark.sql.functions.lit; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.Closeable; +import java.io.IOException; +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Timestamp; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Files; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.MetadataTableUtils; +import org.apache.iceberg.PositionDeletesScanTask; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.ScanTask; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.RewritePositionDeleteFiles.FileGroupRewriteResult; +import org.apache.iceberg.actions.RewritePositionDeleteFiles.Result; +import org.apache.iceberg.actions.SizeBasedFileRewriter; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.deletes.PositionDeleteWriter; +import org.apache.iceberg.encryption.EncryptedFiles; +import org.apache.iceberg.encryption.EncryptedOutputFile; +import org.apache.iceberg.encryption.EncryptionKeyMetadata; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.actions.SparkActions; +import org.apache.iceberg.util.Pair; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.types.StructType; +import org.junit.After; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runners.Parameterized; + +public class TestRewritePositionDeleteFiles extends SparkExtensionsTestBase { + + private static final Map CATALOG_PROPS = + ImmutableMap.of( + "type", "hive", + "default-namespace", "default", + "cache-enabled", "false"); + + private static final String PARTITION_COL = "partition_col"; + private static final int NUM_DATA_FILES = 5; + private static final int ROWS_PER_DATA_FILE = 100; + private static final int DELETE_FILES_PER_PARTITION = 2; + private static final int DELETE_FILE_SIZE = 10; + + @Parameterized.Parameters( + name = "formatVersion = {0}, catalogName = {1}, implementation = {2}, config = {3}") + public static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HIVE.catalogName(), + SparkCatalogConfig.HIVE.implementation(), + CATALOG_PROPS + } + }; + } + + @Rule public TemporaryFolder temp = new TemporaryFolder(); + + public TestRewritePositionDeleteFiles( + String catalogName, String implementation, Map config) { + super(catalogName, implementation, config); + } + + @After + public void cleanup() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @Test + public void testDatePartition() throws Exception { + createTable("date"); + Date baseDate = Date.valueOf("2023-01-01"); + insertData(i -> Date.valueOf(baseDate.toLocalDate().plusDays(i))); + testDanglingDelete(); + } + + @Test + public void testBooleanPartition() throws Exception { + createTable("boolean"); + insertData(i -> i % 2 == 0, 2); + testDanglingDelete(2); + } + + @Test + public void testTimestampPartition() throws Exception { + createTable("timestamp"); + Timestamp baseTimestamp = Timestamp.valueOf("2023-01-01 15:30:00"); + insertData(i -> Timestamp.valueOf(baseTimestamp.toLocalDateTime().plusDays(i))); + testDanglingDelete(); + } + + @Test + public void testBytePartition() throws Exception { + createTable("byte"); + insertData(i -> i); + testDanglingDelete(); + } + + @Test + public void testDecimalPartition() throws Exception { + createTable("decimal(18, 10)"); + BigDecimal baseDecimal = new BigDecimal("1.0"); + insertData(i -> baseDecimal.add(new BigDecimal(i))); + testDanglingDelete(); + } + + @Test + public void testBinaryPartition() throws Exception { + createTable("binary"); + insertData(i -> java.nio.ByteBuffer.allocate(4).putInt(i).array()); + testDanglingDelete(); + } + + @Test + public void testCharPartition() throws Exception { + createTable("char(10)"); + insertData(Object::toString); + testDanglingDelete(); + } + + @Test + public void testVarcharPartition() throws Exception { + createTable("varchar(10)"); + insertData(Object::toString); + testDanglingDelete(); + } + + @Test + public void testIntPartition() throws Exception { + createTable("int"); + insertData(i -> i); + testDanglingDelete(); + } + + @Test + public void testDaysPartitionTransform() throws Exception { + createTable("timestamp", PARTITION_COL, String.format("days(%s)", PARTITION_COL)); + Timestamp baseTimestamp = Timestamp.valueOf("2023-01-01 15:30:00"); + insertData(i -> Timestamp.valueOf(baseTimestamp.toLocalDateTime().plusDays(i))); + testDanglingDelete(); + } + + @Test + public void testNullTransform() throws Exception { + createTable("int"); + insertData(i -> i == 0 ? null : 1, 2); + testDanglingDelete(2); + } + + @Test + public void testPartitionColWithDot() throws Exception { + String partitionColWithDot = "`partition.col`"; + createTable("int", partitionColWithDot, partitionColWithDot); + insertData(partitionColWithDot, i -> i, NUM_DATA_FILES); + testDanglingDelete(partitionColWithDot, NUM_DATA_FILES); + } + + private void testDanglingDelete() throws Exception { + testDanglingDelete(NUM_DATA_FILES); + } + + private void testDanglingDelete(int numDataFiles) throws Exception { + testDanglingDelete(PARTITION_COL, numDataFiles); + } + + private void testDanglingDelete(String partitionCol, int numDataFiles) throws Exception { + Table table = Spark3Util.loadIcebergTable(spark, tableName); + + List dataFiles = dataFiles(table); + assertThat(dataFiles).hasSize(numDataFiles); + + SparkActions.get(spark) + .rewriteDataFiles(table) + .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .execute(); + + // write dangling delete files for 'old data files' + writePosDeletesForFiles(table, dataFiles); + List deleteFiles = deleteFiles(table); + assertThat(deleteFiles).hasSize(numDataFiles * DELETE_FILES_PER_PARTITION); + + List expectedRecords = records(tableName, partitionCol); + + Result result = + SparkActions.get(spark) + .rewritePositionDeletes(table) + .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .execute(); + + List newDeleteFiles = deleteFiles(table); + assertThat(newDeleteFiles).as("Remaining dangling deletes").isEmpty(); + checkResult(result, deleteFiles, Lists.newArrayList(), numDataFiles); + + List actualRecords = records(tableName, partitionCol); + assertEquals("Rows must match", expectedRecords, actualRecords); + } + + private void createTable(String partitionType) { + createTable(partitionType, PARTITION_COL, PARTITION_COL); + } + + private void createTable(String partitionType, String partitionCol, String partitionTransform) { + sql( + "CREATE TABLE %s (id long, %s %s, c1 string, c2 string) " + + "USING iceberg " + + "PARTITIONED BY (%s) " + + "TBLPROPERTIES('format-version'='2')", + tableName, partitionCol, partitionType, partitionTransform); + } + + private void insertData(Function partitionValueFunction) throws Exception { + insertData(partitionValueFunction, NUM_DATA_FILES); + } + + private void insertData(Function partitionValueFunction, int numDataFiles) + throws Exception { + insertData(PARTITION_COL, partitionValueFunction, numDataFiles); + } + + private void insertData( + String partitionCol, Function partitionValue, int numDataFiles) throws Exception { + for (int i = 0; i < numDataFiles; i++) { + Dataset df = + spark + .range(0, ROWS_PER_DATA_FILE) + .withColumn(partitionCol, lit(partitionValue.apply(i))) + .withColumn("c1", expr("CAST(id AS STRING)")) + .withColumn("c2", expr("CAST(id AS STRING)")); + appendAsFile(df); + } + } + + private void appendAsFile(Dataset df) throws Exception { + // ensure the schema is precise + StructType sparkSchema = spark.table(tableName).schema(); + spark.createDataFrame(df.rdd(), sparkSchema).coalesce(1).writeTo(tableName).append(); + } + + private void writePosDeletesForFiles(Table table, List files) throws IOException { + + Map> filesByPartition = + files.stream().collect(Collectors.groupingBy(ContentFile::partition)); + List deleteFiles = + Lists.newArrayListWithCapacity(DELETE_FILES_PER_PARTITION * filesByPartition.size()); + + for (Map.Entry> filesByPartitionEntry : + filesByPartition.entrySet()) { + + StructLike partition = filesByPartitionEntry.getKey(); + List partitionFiles = filesByPartitionEntry.getValue(); + + int deletesForPartition = partitionFiles.size() * DELETE_FILE_SIZE; + assertThat(deletesForPartition % DELETE_FILE_SIZE) + .as("Number of delete files per partition modulo number of data files in this partition") + .isEqualTo(0); + int deleteFileSize = deletesForPartition / DELETE_FILES_PER_PARTITION; + + int counter = 0; + List> deletes = Lists.newArrayList(); + for (DataFile partitionFile : partitionFiles) { + for (int deletePos = 0; deletePos < DELETE_FILE_SIZE; deletePos++) { + deletes.add(Pair.of(partitionFile.path(), (long) deletePos)); + counter++; + if (counter == deleteFileSize) { + // Dump to file and reset variables + OutputFile output = Files.localOutput(temp.newFile()); + deleteFiles.add(writeDeleteFile(table, output, partition, deletes)); + counter = 0; + deletes.clear(); + } + } + } + } + + RowDelta rowDelta = table.newRowDelta(); + deleteFiles.forEach(rowDelta::addDeletes); + rowDelta.commit(); + } + + private DeleteFile writeDeleteFile( + Table table, OutputFile out, StructLike partition, List> deletes) + throws IOException { + FileFormat format = defaultFormat(table.properties()); + FileAppenderFactory factory = new GenericAppenderFactory(table.schema(), table.spec()); + + PositionDeleteWriter writer = + factory.newPosDeleteWriter(encrypt(out), format, partition); + PositionDelete posDelete = PositionDelete.create(); + try (Closeable toClose = writer) { + for (Pair delete : deletes) { + writer.write(posDelete.set(delete.first(), delete.second(), null)); + } + } + + return writer.toDeleteFile(); + } + + private static EncryptedOutputFile encrypt(OutputFile out) { + return EncryptedFiles.encryptedOutput(out, EncryptionKeyMetadata.EMPTY); + } + + private static FileFormat defaultFormat(Map properties) { + String formatString = properties.getOrDefault(DEFAULT_FILE_FORMAT, DEFAULT_FILE_FORMAT_DEFAULT); + return FileFormat.fromString(formatString); + } + + private List records(String table, String partitionCol) { + return rowsToJava( + spark.read().format("iceberg").load(table).sort(partitionCol, "id").collectAsList()); + } + + private long size(List deleteFiles) { + return deleteFiles.stream().mapToLong(DeleteFile::fileSizeInBytes).sum(); + } + + private List dataFiles(Table table) { + CloseableIterable tasks = table.newScan().includeColumnStats().planFiles(); + return Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); + } + + private List deleteFiles(Table table) { + Table deletesTable = + MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES); + CloseableIterable tasks = deletesTable.newBatchScan().planFiles(); + return Lists.newArrayList( + CloseableIterable.transform(tasks, t -> ((PositionDeletesScanTask) t).file())); + } + + private void checkResult( + Result result, + List rewrittenDeletes, + List newDeletes, + int expectedGroups) { + assertThat(result.rewrittenDeleteFilesCount()) + .as("Rewritten delete files") + .isEqualTo(rewrittenDeletes.size()); + assertThat(result.addedDeleteFilesCount()) + .as("Added delete files") + .isEqualTo(newDeletes.size()); + assertThat(result.rewrittenBytesCount()) + .as("Rewritten delete bytes") + .isEqualTo(size(rewrittenDeletes)); + assertThat(result.addedBytesCount()).as("New Delete byte count").isEqualTo(size(newDeletes)); + + assertThat(result.rewriteResults()).as("Rewritten group count").hasSize(expectedGroups); + assertThat( + result.rewriteResults().stream() + .mapToInt(FileGroupRewriteResult::rewrittenDeleteFilesCount) + .sum()) + .as("Rewritten delete file count in all groups") + .isEqualTo(rewrittenDeletes.size()); + assertThat( + result.rewriteResults().stream() + .mapToInt(FileGroupRewriteResult::addedDeleteFilesCount) + .sum()) + .as("Added delete file count in all groups") + .isEqualTo(newDeletes.size()); + assertThat( + result.rewriteResults().stream() + .mapToLong(FileGroupRewriteResult::rewrittenBytesCount) + .sum()) + .as("Rewritten delete bytes in all groups") + .isEqualTo(size(rewrittenDeletes)); + assertThat( + result.rewriteResults().stream() + .mapToLong(FileGroupRewriteResult::addedBytesCount) + .sum()) + .as("Added delete bytes in all groups") + .isEqualTo(size(newDeletes)); + } +} diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestTagDDL.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestTagDDL.java index ec3148de6cb5..866a965e33e6 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestTagDDL.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestTagDDL.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.List; import java.util.Locale; import java.util.Map; @@ -364,6 +366,18 @@ public void testDropTagIfExists() throws NoSuchTableException { Assert.assertNull("The tag needs to be dropped.", table.refs().get(tagName)); } + @Test + public void createOrReplaceWithNonExistingTag() throws NoSuchTableException { + Table table = insertRows(); + String tagName = "t1"; + insertRows(); + long snapshotId = table.currentSnapshot().snapshotId(); + + sql("ALTER TABLE %s CREATE OR REPLACE TAG %s AS OF VERSION %d", tableName, tagName, snapshotId); + table.refresh(); + assertThat(table.refs().get(tagName).snapshotId()).isEqualTo(snapshotId); + } + private Table insertRows() throws NoSuchTableException { List records = ImmutableList.of(new SimpleRecord(1, "a"), new SimpleRecord(2, "b")); diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/ChangelogIterator.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/ChangelogIterator.java index 5f30c5fd4e56..cc44b1f3992c 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/ChangelogIterator.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/ChangelogIterator.java @@ -20,8 +20,10 @@ import java.util.Iterator; import java.util.Objects; +import java.util.Set; import org.apache.iceberg.ChangelogOperation; import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Iterators; import org.apache.spark.sql.Row; import org.apache.spark.sql.types.StructType; @@ -35,9 +37,11 @@ public abstract class ChangelogIterator implements Iterator { private final Iterator rowIterator; private final int changeTypeIndex; + private final StructType rowType; protected ChangelogIterator(Iterator rowIterator, StructType rowType) { this.rowIterator = rowIterator; + this.rowType = rowType; this.changeTypeIndex = rowType.fieldIndex(MetadataColumns.CHANGE_TYPE.name()); } @@ -45,6 +49,16 @@ protected int changeTypeIndex() { return changeTypeIndex; } + protected StructType rowType() { + return rowType; + } + + protected String changeType(Row row) { + String changeType = row.getString(changeTypeIndex()); + Preconditions.checkNotNull(changeType, "Change type should not be null"); + return changeType; + } + protected Iterator rowIterator() { return rowIterator; } @@ -79,7 +93,35 @@ public static Iterator removeCarryovers(Iterator rowIterator, StructTy return Iterators.filter(changelogIterator, Objects::nonNull); } + public static Iterator removeNetCarryovers(Iterator rowIterator, StructType rowType) { + ChangelogIterator changelogIterator = new RemoveNetCarryoverIterator(rowIterator, rowType); + return Iterators.filter(changelogIterator, Objects::nonNull); + } + + protected boolean isSameRecord(Row currentRow, Row nextRow, int[] indicesToIdentifySameRow) { + for (int idx : indicesToIdentifySameRow) { + if (isDifferentValue(currentRow, nextRow, idx)) { + return false; + } + } + + return true; + } + protected boolean isDifferentValue(Row currentRow, Row nextRow, int idx) { return !Objects.equals(nextRow.get(idx), currentRow.get(idx)); } + + protected static int[] generateIndicesToIdentifySameRow( + int totalColumnCount, Set metadataColumnIndices) { + int[] indices = new int[totalColumnCount - metadataColumnIndices.size()]; + + for (int i = 0, j = 0; i < indices.length; i++) { + if (!metadataColumnIndices.contains(i)) { + indices[j] = i; + j++; + } + } + return indices; + } } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java index 641b957d1176..ea400a779235 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java @@ -20,7 +20,9 @@ import java.util.Map; import java.util.concurrent.Callable; +import org.apache.iceberg.SnapshotSummary; 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.ExceptionUtil; /** utility class to accept thread local commit properties */ @@ -35,13 +37,19 @@ private CommitMetadata() {} * running the code wrapped as a caller, and any snapshot committed within the callable object * will be attached with the metadata defined in properties * - * @param properties extra commit metadata to attach to the snapshot committed within callable + * @param properties extra commit metadata to attach to the snapshot committed within callable. + * The prefix will be removed for properties starting with {@link + * SnapshotSummary#EXTRA_METADATA_PREFIX} * @param callable the code to be executed * @param exClass the expected type of exception which would be thrown from callable */ public static R withCommitProperties( Map properties, Callable callable, Class exClass) throws E { - COMMIT_PROPERTIES.set(properties); + Map props = Maps.newHashMap(); + properties.forEach( + (k, v) -> props.put(k.replace(SnapshotSummary.EXTRA_METADATA_PREFIX, ""), v)); + + COMMIT_PROPERTIES.set(props); try { return callable.call(); } catch (Throwable e) { diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/ComputeUpdateIterator.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/ComputeUpdateIterator.java index 23e6a19a17e7..6951c33e51aa 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/ComputeUpdateIterator.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/ComputeUpdateIterator.java @@ -81,15 +81,13 @@ public Row next() { // either a cached record which is not an UPDATE or the next record in the iterator. Row currentRow = currentRow(); - if (currentRow.getString(changeTypeIndex()).equals(DELETE) && rowIterator().hasNext()) { + if (changeType(currentRow).equals(DELETE) && rowIterator().hasNext()) { Row nextRow = rowIterator().next(); cachedRow = nextRow; if (sameLogicalRow(currentRow, nextRow)) { - String nextRowChangeType = nextRow.getString(changeTypeIndex()); - Preconditions.checkState( - nextRowChangeType.equals(INSERT), + changeType(nextRow).equals(INSERT), "Cannot compute updates because there are multiple rows with the same identifier" + " fields([%s]). Please make sure the rows are unique.", String.join(",", identifierFields)); @@ -118,7 +116,7 @@ private Row modify(Row row, int valueIndex, Object value) { } private boolean cachedUpdateRecord() { - return cachedRow != null && cachedRow.getString(changeTypeIndex()).equals(UPDATE_AFTER); + return cachedRow != null && changeType(cachedRow).equals(UPDATE_AFTER); } private Row currentRow() { diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/RemoveCarryoverIterator.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/RemoveCarryoverIterator.java index 70b160e13fee..2e90dc7749d1 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/RemoveCarryoverIterator.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/RemoveCarryoverIterator.java @@ -19,6 +19,8 @@ package org.apache.iceberg.spark; import java.util.Iterator; +import java.util.Set; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.spark.sql.Row; import org.apache.spark.sql.types.StructType; @@ -55,7 +57,7 @@ class RemoveCarryoverIterator extends ChangelogIterator { RemoveCarryoverIterator(Iterator rowIterator, StructType rowType) { super(rowIterator, rowType); - this.indicesToIdentifySameRow = generateIndicesToIdentifySameRow(rowType.size()); + this.indicesToIdentifySameRow = generateIndicesToIdentifySameRow(); } @Override @@ -88,7 +90,7 @@ public Row next() { } // If the current row is a delete row, drain all identical delete rows - if (currentRow.getString(changeTypeIndex()).equals(DELETE) && rowIterator().hasNext()) { + if (changeType(currentRow).equals(DELETE) && rowIterator().hasNext()) { cachedDeletedRow = currentRow; deletedRowCount = 1; @@ -98,8 +100,8 @@ public Row next() { // row is the same record while (nextRow != null && cachedDeletedRow != null - && isSameRecord(cachedDeletedRow, nextRow)) { - if (nextRow.getString(changeTypeIndex()).equals(INSERT)) { + && isSameRecord(cachedDeletedRow, nextRow, indicesToIdentifySameRow)) { + if (changeType(nextRow).equals(INSERT)) { deletedRowCount--; if (deletedRowCount == 0) { cachedDeletedRow = null; @@ -139,25 +141,8 @@ private boolean hasCachedDeleteRow() { return cachedDeletedRow != null; } - private int[] generateIndicesToIdentifySameRow(int columnSize) { - int[] indices = new int[columnSize - 1]; - for (int i = 0; i < indices.length; i++) { - if (i < changeTypeIndex()) { - indices[i] = i; - } else { - indices[i] = i + 1; - } - } - return indices; - } - - private boolean isSameRecord(Row currentRow, Row nextRow) { - for (int idx : indicesToIdentifySameRow) { - if (isDifferentValue(currentRow, nextRow, idx)) { - return false; - } - } - - return true; + private int[] generateIndicesToIdentifySameRow() { + Set metadataColumnIndices = Sets.newHashSet(changeTypeIndex()); + return generateIndicesToIdentifySameRow(rowType().size(), metadataColumnIndices); } } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/RemoveNetCarryoverIterator.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/RemoveNetCarryoverIterator.java new file mode 100644 index 000000000000..e4234755cdcf --- /dev/null +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/RemoveNetCarryoverIterator.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark; + +import java.util.Iterator; +import java.util.Set; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.types.StructType; + +/** + * This class computes the net changes across multiple snapshots. It is different from {@link + * RemoveCarryoverIterator}, which only removes carry-over rows within a single snapshot. It takes a + * row iterator, and assumes the following: + * + *
      + *
    • The row iterator is partitioned by all columns. + *
    • The row iterator is sorted by all columns, change order, and change type. The change order + * is 1-to-1 mapping to snapshot id. + *
    + */ +public class RemoveNetCarryoverIterator extends ChangelogIterator { + + private final int[] indicesToIdentifySameRow; + + private Row cachedNextRow; + private Row cachedRow; + private long cachedRowCount; + + protected RemoveNetCarryoverIterator(Iterator rowIterator, StructType rowType) { + super(rowIterator, rowType); + this.indicesToIdentifySameRow = generateIndicesToIdentifySameRow(); + } + + @Override + public boolean hasNext() { + if (cachedRowCount > 0) { + return true; + } + + if (cachedNextRow != null) { + return true; + } + + return rowIterator().hasNext(); + } + + @Override + public Row next() { + // if there are cached rows, return one of them from the beginning + if (cachedRowCount > 0) { + cachedRowCount--; + return cachedRow; + } + + cachedRow = getCurrentRow(); + // return it directly if there is no more rows + if (!rowIterator().hasNext()) { + return cachedRow; + } + cachedRowCount = 1; + + cachedNextRow = rowIterator().next(); + + // pull rows from the iterator until two consecutive rows are different + while (isSameRecord(cachedRow, cachedNextRow, indicesToIdentifySameRow)) { + if (oppositeChangeType(cachedRow, cachedNextRow)) { + // two rows with opposite change types means no net changes, remove both + cachedRowCount--; + } else { + // two rows with same change types means potential net changes, cache the next row + cachedRowCount++; + } + + // stop pulling rows if there is no more rows or the next row is different + if (cachedRowCount <= 0 || !rowIterator().hasNext()) { + // reset the cached next row if there is no more rows + cachedNextRow = null; + break; + } + + cachedNextRow = rowIterator().next(); + } + + return null; + } + + private Row getCurrentRow() { + Row currentRow; + if (cachedNextRow != null) { + currentRow = cachedNextRow; + cachedNextRow = null; + } else { + currentRow = rowIterator().next(); + } + return currentRow; + } + + private boolean oppositeChangeType(Row currentRow, Row nextRow) { + return (changeType(nextRow).equals(INSERT) && changeType(currentRow).equals(DELETE)) + || (changeType(nextRow).equals(DELETE) && changeType(currentRow).equals(INSERT)); + } + + private int[] generateIndicesToIdentifySameRow() { + Set metadataColumnIndices = + Sets.newHashSet( + rowType().fieldIndex(MetadataColumns.CHANGE_ORDINAL.name()), + rowType().fieldIndex(MetadataColumns.COMMIT_SNAPSHOT_ID.name()), + changeTypeIndex()); + return generateIndicesToIdentifySameRow(rowType().size(), metadataColumnIndices); + } +} diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSessionCatalog.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSessionCatalog.java index c891985b383d..33384e3eff08 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSessionCatalog.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSessionCatalog.java @@ -22,6 +22,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.spark.source.HasIcebergCatalog; @@ -303,7 +304,10 @@ public void renameTable(Identifier from, Identifier to) @Override public final void initialize(String name, CaseInsensitiveStringMap options) { - if (options.containsKey("type") && options.get("type").equalsIgnoreCase("hive")) { + if (options.containsKey(CatalogUtil.ICEBERG_CATALOG_TYPE) + && options + .get(CatalogUtil.ICEBERG_CATALOG_TYPE) + .equalsIgnoreCase(CatalogUtil.ICEBERG_CATALOG_TYPE_HIVE)) { validateHmsUri(options.get(CatalogProperties.URI)); } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java index 90bdbfc1d9ba..37f3f0c6c48d 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java @@ -44,6 +44,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.TableMigrationUtil; +import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.hadoop.HadoopFileIO; import org.apache.iceberg.hadoop.SerializableConfiguration; import org.apache.iceberg.hadoop.Util; @@ -440,12 +441,12 @@ public static void importSparkTable( } else { List sourceTablePartitions = getPartitions(spark, sourceTableIdent, partitionFilter); - Preconditions.checkArgument( - !sourceTablePartitions.isEmpty(), - "Cannot find any partitions in table %s", - sourceTableIdent); - importSparkPartitions( - spark, sourceTablePartitions, targetTable, spec, stagingDir, checkDuplicateFiles); + if (sourceTablePartitions.isEmpty()) { + targetTable.newAppend().commit(); + } else { + importSparkPartitions( + spark, sourceTablePartitions, targetTable, spec, stagingDir, checkDuplicateFiles); + } } } catch (AnalysisException e) { throw SparkExceptionUtil.toUncheckedException( @@ -715,6 +716,43 @@ public static Dataset loadMetadataTable( spark, DataSourceV2Relation.create(metadataTable, Some.empty(), Some.empty(), options)); } + /** + * Determine the write branch. + * + *

    Validate wap config and determine the write branch. + * + * @param spark a Spark Session + * @param branch write branch if there is no WAP branch configured + * @return branch for write operation + */ + public static String determineWriteBranch(SparkSession spark, String branch) { + String wapId = spark.conf().get(SparkSQLProperties.WAP_ID, null); + String wapBranch = spark.conf().get(SparkSQLProperties.WAP_BRANCH, null); + ValidationException.check( + wapId == null || wapBranch == null, + "Cannot set both WAP ID and branch, but got ID [%s] and branch [%s]", + wapId, + wapBranch); + + if (wapBranch != null) { + ValidationException.check( + branch == null, + "Cannot write to both branch and WAP branch, but got branch [%s] and WAP branch [%s]", + branch, + wapBranch); + + return wapBranch; + } + return branch; + } + + public static boolean wapEnabled(Table table) { + return PropertyUtil.propertyAsBoolean( + table.properties(), + TableProperties.WRITE_AUDIT_PUBLISH_ENABLED, + Boolean.getBoolean(TableProperties.WRITE_AUDIT_PUBLISH_ENABLED_DEFAULT)); + } + /** Class representing a table partition. */ public static class SparkPartition implements Serializable { private final Map values; diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkValueConverter.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkValueConverter.java index 687d9f43ade8..c168f7728085 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkValueConverter.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkValueConverter.java @@ -28,6 +28,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.ByteBuffers; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.util.DateTimeUtils; @@ -120,4 +121,39 @@ private static Record convert(Types.StructType struct, Row row) { } return record; } + + public static Object convertToSpark(Type type, Object object) { + if (object == null) { + return null; + } + + switch (type.typeId()) { + case STRUCT: + case LIST: + case MAP: + return new UnsupportedOperationException("Complex types currently not supported"); + case DATE: + return DateTimeUtils.daysToLocalDate((int) object); + case TIMESTAMP: + Types.TimestampType ts = (Types.TimestampType) type.asPrimitiveType(); + if (ts.shouldAdjustToUTC()) { + return DateTimeUtils.microsToInstant((long) object); + } else { + return DateTimeUtils.microsToLocalDateTime((long) object); + } + case BINARY: + return ByteBuffers.toByteArray((ByteBuffer) object); + case INTEGER: + case BOOLEAN: + case LONG: + case FLOAT: + case DOUBLE: + case DECIMAL: + case STRING: + case FIXED: + return object; + default: + throw new UnsupportedOperationException("Not a supported type: " + type); + } + } } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkBinPackPositionDeletesRewriter.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkBinPackPositionDeletesRewriter.java index 51c4cc661f4e..1b36441c3c4d 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkBinPackPositionDeletesRewriter.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkBinPackPositionDeletesRewriter.java @@ -20,6 +20,7 @@ import static org.apache.iceberg.MetadataTableType.POSITION_DELETES; import static org.apache.spark.sql.functions.col; +import static org.apache.spark.sql.functions.lit; import java.util.List; import java.util.Optional; @@ -40,7 +41,9 @@ import org.apache.iceberg.spark.SparkReadOptions; import org.apache.iceberg.spark.SparkTableCache; import org.apache.iceberg.spark.SparkTableUtil; +import org.apache.iceberg.spark.SparkValueConverter; import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.spark.sql.Column; import org.apache.spark.sql.Dataset; @@ -125,10 +128,11 @@ private Dataset dataFiles(Types.StructType partitionType, StructLike partit IntStream.range(0, fields.size()) .mapToObj( i -> { - Class type = fields.get(i).type().typeId().javaClass(); - Object value = partition.get(i, type); - Column col = col("partition." + fields.get(i).name()); - return col.equalTo(value); + Type type = fields.get(i).type(); + Object value = partition.get(i, type.typeId().javaClass()); + Object convertedValue = SparkValueConverter.convertToSpark(type, value); + Column col = col("partition.`" + fields.get(i).name() + "`"); + return col.eqNullSafe(lit(convertedValue)); }) .reduce(Column::and); if (condition.isPresent()) { diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java index 85043d2df3d6..259254aa2d51 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java @@ -21,11 +21,14 @@ import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.function.Predicate; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.ChangelogIterator; import org.apache.iceberg.spark.source.SparkChangelogTable; import org.apache.spark.api.java.function.MapPartitionsFunction; @@ -88,10 +91,22 @@ public class CreateChangelogViewProcedure extends BaseProcedure { ProcedureParameter.optional("options", STRING_MAP); private static final ProcedureParameter COMPUTE_UPDATES_PARAM = ProcedureParameter.optional("compute_updates", DataTypes.BooleanType); + + /** + * Enable or disable the remove carry-over rows. + * + * @deprecated since 1.4.0, will be removed in 1.5.0; The procedure will always remove carry-over + * rows. Please query {@link SparkChangelogTable} instead for the use cases doesn't remove + * carry-over rows. + */ + @Deprecated private static final ProcedureParameter REMOVE_CARRYOVERS_PARAM = ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType); + private static final ProcedureParameter IDENTIFIER_COLUMNS_PARAM = ProcedureParameter.optional("identifier_columns", STRING_ARRAY); + private static final ProcedureParameter NET_CHANGES = + ProcedureParameter.optional("net_changes", DataTypes.BooleanType); private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[] { @@ -101,6 +116,7 @@ public class CreateChangelogViewProcedure extends BaseProcedure { COMPUTE_UPDATES_PARAM, REMOVE_CARRYOVERS_PARAM, IDENTIFIER_COLUMNS_PARAM, + NET_CHANGES, }; private static final StructType OUTPUT_TYPE = @@ -142,10 +158,13 @@ public InternalRow[] call(InternalRow args) { Identifier changelogTableIdent = changelogTableIdent(tableIdent); Dataset df = loadRows(changelogTableIdent, options(input)); + boolean netChanges = input.asBoolean(NET_CHANGES, false); + if (shouldComputeUpdateImages(input)) { + Preconditions.checkArgument(!netChanges, "Not support net changes with update images"); df = computeUpdateImages(identifierColumns(input, tableIdent), df); } else if (shouldRemoveCarryoverRows(input)) { - df = removeCarryoverRows(df); + df = removeCarryoverRows(df, netChanges); } String viewName = viewName(input, tableIdent.name()); @@ -164,6 +183,7 @@ private Dataset computeUpdateImages(String[] identifierColumns, Dataset removeCarryoverRows(Dataset df) { + private Dataset removeCarryoverRows(Dataset df, boolean netChanges) { + Predicate columnsToKeep; + if (netChanges) { + Set metadataColumn = + Sets.newHashSet( + MetadataColumns.CHANGE_TYPE.name(), + MetadataColumns.CHANGE_ORDINAL.name(), + MetadataColumns.COMMIT_SNAPSHOT_ID.name()); + + columnsToKeep = column -> !metadataColumn.contains(column); + } else { + columnsToKeep = column -> !column.equals(MetadataColumns.CHANGE_TYPE.name()); + } + Column[] repartitionSpec = - Arrays.stream(df.columns()) - .filter(c -> !c.equals(MetadataColumns.CHANGE_TYPE.name())) - .map(df::col) - .toArray(Column[]::new); - return applyCarryoverRemoveIterator(df, repartitionSpec); + Arrays.stream(df.columns()).filter(columnsToKeep).map(df::col).toArray(Column[]::new); + return applyCarryoverRemoveIterator(df, repartitionSpec, netChanges); } private String[] identifierColumns(ProcedureInput input, Identifier tableIdent) { @@ -214,7 +244,7 @@ private String viewName(ProcedureInput input, String tableName) { } private Dataset applyChangelogIterator(Dataset df, Column[] repartitionSpec) { - Column[] sortSpec = sortSpec(df, repartitionSpec); + Column[] sortSpec = sortSpec(df, repartitionSpec, false); StructType schema = df.schema(); String[] identifierFields = Arrays.stream(repartitionSpec).map(Column::toString).toArray(String[]::new); @@ -228,22 +258,33 @@ private Dataset applyChangelogIterator(Dataset df, Column[] repartitio RowEncoder.apply(schema)); } - private Dataset applyCarryoverRemoveIterator(Dataset df, Column[] repartitionSpec) { - Column[] sortSpec = sortSpec(df, repartitionSpec); + private Dataset applyCarryoverRemoveIterator( + Dataset df, Column[] repartitionSpec, boolean netChanges) { + Column[] sortSpec = sortSpec(df, repartitionSpec, netChanges); StructType schema = df.schema(); return df.repartition(repartitionSpec) .sortWithinPartitions(sortSpec) .mapPartitions( (MapPartitionsFunction) - rowIterator -> ChangelogIterator.removeCarryovers(rowIterator, schema), + rowIterator -> + netChanges + ? ChangelogIterator.removeNetCarryovers(rowIterator, schema) + : ChangelogIterator.removeCarryovers(rowIterator, schema), RowEncoder.apply(schema)); } - private static Column[] sortSpec(Dataset df, Column[] repartitionSpec) { - Column[] sortSpec = new Column[repartitionSpec.length + 1]; + private static Column[] sortSpec(Dataset df, Column[] repartitionSpec, boolean netChanges) { + Column changeType = df.col(MetadataColumns.CHANGE_TYPE.name()); + Column changeOrdinal = df.col(MetadataColumns.CHANGE_ORDINAL.name()); + Column[] extraColumns = + netChanges ? new Column[] {changeOrdinal, changeType} : new Column[] {changeType}; + + Column[] sortSpec = new Column[repartitionSpec.length + extraColumns.length]; + System.arraycopy(repartitionSpec, 0, sortSpec, 0, repartitionSpec.length); - sortSpec[sortSpec.length - 1] = df.col(MetadataColumns.CHANGE_TYPE.name()); + System.arraycopy(extraColumns, 0, sortSpec, repartitionSpec.length, extraColumns.length); + return sortSpec; } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java index ddeec9c4943b..2653b9eab1f5 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java @@ -30,6 +30,7 @@ import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.MetricsConfig; import org.apache.iceberg.MetricsModes; +import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; import org.apache.iceberg.StructLike; @@ -164,7 +165,9 @@ public Filter[] pushFilters(Filter[] filters) { pushableFilters.add(filter); } - if (expr == null || !ExpressionUtil.selectsPartitions(expr, table, caseSensitive)) { + if (expr == null + || unpartitioned() + || !ExpressionUtil.selectsPartitions(expr, table, caseSensitive)) { postScanFilters.add(filter); } else { LOG.info("Evaluating completely on Iceberg side: {}", filter); @@ -182,6 +185,10 @@ public Filter[] pushFilters(Filter[] filters) { return postScanFilters.toArray(new Filter[0]); } + private boolean unpartitioned() { + return table.specs().values().stream().noneMatch(PartitionSpec::isPartitioned); + } + @Override public Filter[] pushedFilters() { return pushedFilters; @@ -651,7 +658,7 @@ public Scan buildCopyOnWriteScan() { @Override public Statistics estimateStatistics() { - return ((SparkScan) build()).estimateStatistics(); + return ((SupportsReportStatistics) build()).estimateStatistics(); } @Override diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java index 8fb583123cbc..74c3667f6f65 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java @@ -57,6 +57,7 @@ import org.apache.iceberg.spark.SparkFilters; import org.apache.iceberg.spark.SparkReadOptions; import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.SparkTableUtil; import org.apache.iceberg.spark.SparkUtil; import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.SnapshotUtil; @@ -372,6 +373,10 @@ public void deleteWhere(Filter[] filters) { .set("spark.app.id", sparkSession().sparkContext().applicationId()) .deleteFromRowFilter(deleteExpr); + if (SparkTableUtil.wapEnabled(table())) { + branch = SparkTableUtil.determineWriteBranch(sparkSession(), branch); + } + if (branch != null) { deleteFiles.toBranch(branch); } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestChangelogIterator.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestChangelogIterator.java index bf98bebb9d50..0539598f147e 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestChangelogIterator.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestChangelogIterator.java @@ -49,7 +49,17 @@ public class TestChangelogIterator extends SparkTestHelperBase { new StructField("name", DataTypes.StringType, false, Metadata.empty()), new StructField("data", DataTypes.StringType, true, Metadata.empty()), new StructField( - MetadataColumns.CHANGE_TYPE.name(), DataTypes.StringType, false, Metadata.empty()) + MetadataColumns.CHANGE_TYPE.name(), DataTypes.StringType, false, Metadata.empty()), + new StructField( + MetadataColumns.CHANGE_ORDINAL.name(), + DataTypes.IntegerType, + false, + Metadata.empty()), + new StructField( + MetadataColumns.COMMIT_SNAPSHOT_ID.name(), + DataTypes.LongType, + false, + Metadata.empty()) }); private static final String[] IDENTIFIER_FIELDS = new String[] {"id", "name"}; @@ -93,18 +103,18 @@ private List toOriginalRows(RowType rowType, int index) { switch (rowType) { case DELETED: return Lists.newArrayList( - new GenericRowWithSchema(new Object[] {index, "b", "data", DELETE}, null)); + new GenericRowWithSchema(new Object[] {index, "b", "data", DELETE, 0, 0}, null)); case INSERTED: return Lists.newArrayList( - new GenericRowWithSchema(new Object[] {index, "c", "data", INSERT}, null)); + new GenericRowWithSchema(new Object[] {index, "c", "data", INSERT, 0, 0}, null)); case CARRY_OVER: return Lists.newArrayList( - new GenericRowWithSchema(new Object[] {index, "d", "data", DELETE}, null), - new GenericRowWithSchema(new Object[] {index, "d", "data", INSERT}, null)); + new GenericRowWithSchema(new Object[] {index, "d", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {index, "d", "data", INSERT, 0, 0}, null)); case UPDATED: return Lists.newArrayList( - new GenericRowWithSchema(new Object[] {index, "a", "data", DELETE}, null), - new GenericRowWithSchema(new Object[] {index, "a", "new_data", INSERT}, null)); + new GenericRowWithSchema(new Object[] {index, "a", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {index, "a", "new_data", INSERT, 0, 0}, null)); default: throw new IllegalArgumentException("Unknown row type: " + rowType); } @@ -114,18 +124,18 @@ private List toExpectedRows(RowType rowType, int order) { switch (rowType) { case DELETED: List rows = Lists.newArrayList(); - rows.add(new Object[] {order, "b", "data", DELETE}); + rows.add(new Object[] {order, "b", "data", DELETE, 0, 0}); return rows; case INSERTED: List insertedRows = Lists.newArrayList(); - insertedRows.add(new Object[] {order, "c", "data", INSERT}); + insertedRows.add(new Object[] {order, "c", "data", INSERT, 0, 0}); return insertedRows; case CARRY_OVER: return Lists.newArrayList(); case UPDATED: return Lists.newArrayList( - new Object[] {order, "a", "data", UPDATE_BEFORE}, - new Object[] {order, "a", "new_data", UPDATE_AFTER}); + new Object[] {order, "a", "data", UPDATE_BEFORE, 0, 0}, + new Object[] {order, "a", "new_data", UPDATE_AFTER, 0, 0}); default: throw new IllegalArgumentException("Unknown row type: " + rowType); } @@ -146,16 +156,16 @@ private void permute(List arr, int start, List pm) { public void testRowsWithNullValue() { final List rowsWithNull = Lists.newArrayList( - new GenericRowWithSchema(new Object[] {2, null, null, DELETE}, null), - new GenericRowWithSchema(new Object[] {3, null, null, INSERT}, null), - new GenericRowWithSchema(new Object[] {4, null, null, DELETE}, null), - new GenericRowWithSchema(new Object[] {4, null, null, INSERT}, null), + new GenericRowWithSchema(new Object[] {2, null, null, DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {3, null, null, INSERT, 0, 0}, null), + new GenericRowWithSchema(new Object[] {4, null, null, DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {4, null, null, INSERT, 0, 0}, null), // mixed null and non-null value in non-identifier columns - new GenericRowWithSchema(new Object[] {5, null, null, DELETE}, null), - new GenericRowWithSchema(new Object[] {5, null, "data", INSERT}, null), + new GenericRowWithSchema(new Object[] {5, null, null, DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {5, null, "data", INSERT, 0, 0}, null), // mixed null and non-null value in identifier columns - new GenericRowWithSchema(new Object[] {6, null, null, DELETE}, null), - new GenericRowWithSchema(new Object[] {6, "name", null, INSERT}, null)); + new GenericRowWithSchema(new Object[] {6, null, null, DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {6, "name", null, INSERT, 0, 0}, null)); Iterator iterator = ChangelogIterator.computeUpdates(rowsWithNull.iterator(), SCHEMA, IDENTIFIER_FIELDS); @@ -164,12 +174,12 @@ public void testRowsWithNullValue() { assertEquals( "Rows should match", Lists.newArrayList( - new Object[] {2, null, null, DELETE}, - new Object[] {3, null, null, INSERT}, - new Object[] {5, null, null, UPDATE_BEFORE}, - new Object[] {5, null, "data", UPDATE_AFTER}, - new Object[] {6, null, null, DELETE}, - new Object[] {6, "name", null, INSERT}), + new Object[] {2, null, null, DELETE, 0, 0}, + new Object[] {3, null, null, INSERT, 0, 0}, + new Object[] {5, null, null, UPDATE_BEFORE, 0, 0}, + new Object[] {5, null, "data", UPDATE_AFTER, 0, 0}, + new Object[] {6, null, null, DELETE, 0, 0}, + new Object[] {6, "name", null, INSERT, 0, 0}), rowsToJava(result)); } @@ -178,10 +188,10 @@ public void testUpdatedRowsWithDuplication() { List rowsWithDuplication = Lists.newArrayList( // two rows with same identifier fields(id, name) - new GenericRowWithSchema(new Object[] {1, "a", "data", DELETE}, null), - new GenericRowWithSchema(new Object[] {1, "a", "data", DELETE}, null), - new GenericRowWithSchema(new Object[] {1, "a", "new_data", INSERT}, null), - new GenericRowWithSchema(new Object[] {1, "a", "new_data", INSERT}, null)); + new GenericRowWithSchema(new Object[] {1, "a", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "a", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "a", "new_data", INSERT, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "a", "new_data", INSERT, 0, 0}, null)); Iterator iterator = ChangelogIterator.computeUpdates(rowsWithDuplication.iterator(), SCHEMA, IDENTIFIER_FIELDS); @@ -194,9 +204,9 @@ public void testUpdatedRowsWithDuplication() { // still allow extra insert rows rowsWithDuplication = Lists.newArrayList( - new GenericRowWithSchema(new Object[] {1, "a", "data", DELETE}, null), - new GenericRowWithSchema(new Object[] {1, "a", "new_data1", INSERT}, null), - new GenericRowWithSchema(new Object[] {1, "a", "new_data2", INSERT}, null)); + new GenericRowWithSchema(new Object[] {1, "a", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "a", "new_data1", INSERT, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "a", "new_data2", INSERT, 0, 0}, null)); Iterator iterator1 = ChangelogIterator.computeUpdates(rowsWithDuplication.iterator(), SCHEMA, IDENTIFIER_FIELDS); @@ -204,9 +214,9 @@ public void testUpdatedRowsWithDuplication() { assertEquals( "Rows should match.", Lists.newArrayList( - new Object[] {1, "a", "data", UPDATE_BEFORE}, - new Object[] {1, "a", "new_data1", UPDATE_AFTER}, - new Object[] {1, "a", "new_data2", INSERT}), + new Object[] {1, "a", "data", UPDATE_BEFORE, 0, 0}, + new Object[] {1, "a", "new_data1", UPDATE_AFTER, 0, 0}, + new Object[] {1, "a", "new_data2", INSERT, 0, 0}), rowsToJava(Lists.newArrayList(iterator1))); } @@ -216,32 +226,28 @@ public void testCarryRowsRemoveWithDuplicates() { List rowsWithDuplication = Lists.newArrayList( // keep all delete rows for id 0 and id 1 since there is no insert row for them - new GenericRowWithSchema(new Object[] {0, "a", "data", DELETE}, null), - new GenericRowWithSchema(new Object[] {0, "a", "data", DELETE}, null), - new GenericRowWithSchema(new Object[] {0, "a", "data", DELETE}, null), - new GenericRowWithSchema(new Object[] {1, "a", "old_data", DELETE}, null), - new GenericRowWithSchema(new Object[] {1, "a", "old_data", DELETE}, null), + new GenericRowWithSchema(new Object[] {0, "a", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {0, "a", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {0, "a", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "a", "old_data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "a", "old_data", DELETE, 0, 0}, null), // the same number of delete and insert rows for id 2 - new GenericRowWithSchema(new Object[] {2, "a", "data", DELETE}, null), - new GenericRowWithSchema(new Object[] {2, "a", "data", DELETE}, null), - new GenericRowWithSchema(new Object[] {2, "a", "data", INSERT}, null), - new GenericRowWithSchema(new Object[] {2, "a", "data", INSERT}, null), - new GenericRowWithSchema(new Object[] {3, "a", "new_data", INSERT}, null)); - - Iterator iterator = - ChangelogIterator.removeCarryovers(rowsWithDuplication.iterator(), SCHEMA); - List result = Lists.newArrayList(iterator); + new GenericRowWithSchema(new Object[] {2, "a", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {2, "a", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {2, "a", "data", INSERT, 0, 0}, null), + new GenericRowWithSchema(new Object[] {2, "a", "data", INSERT, 0, 0}, null), + new GenericRowWithSchema(new Object[] {3, "a", "new_data", INSERT, 0, 0}, null)); - assertEquals( - "Rows should match.", + List expectedRows = Lists.newArrayList( - new Object[] {0, "a", "data", DELETE}, - new Object[] {0, "a", "data", DELETE}, - new Object[] {0, "a", "data", DELETE}, - new Object[] {1, "a", "old_data", DELETE}, - new Object[] {1, "a", "old_data", DELETE}, - new Object[] {3, "a", "new_data", INSERT}), - rowsToJava(result)); + new Object[] {0, "a", "data", DELETE, 0, 0}, + new Object[] {0, "a", "data", DELETE, 0, 0}, + new Object[] {0, "a", "data", DELETE, 0, 0}, + new Object[] {1, "a", "old_data", DELETE, 0, 0}, + new Object[] {1, "a", "old_data", DELETE, 0, 0}, + new Object[] {3, "a", "new_data", INSERT, 0, 0}); + + validateIterators(rowsWithDuplication, expectedRows); } @Test @@ -249,45 +255,39 @@ public void testCarryRowsRemoveLessInsertRows() { // less insert rows than delete rows List rowsWithDuplication = Lists.newArrayList( - new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE}, null), - new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE}, null), - new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT}, null), - new GenericRowWithSchema(new Object[] {2, "d", "data", INSERT}, null)); + new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT, 0, 0}, null), + new GenericRowWithSchema(new Object[] {2, "d", "data", INSERT, 0, 0}, null)); - Iterator iterator = - ChangelogIterator.removeCarryovers(rowsWithDuplication.iterator(), SCHEMA); - List result = Lists.newArrayList(iterator); - - assertEquals( - "Rows should match.", + List expectedRows = Lists.newArrayList( - new Object[] {1, "d", "data", DELETE}, new Object[] {2, "d", "data", INSERT}), - rowsToJava(result)); + new Object[] {1, "d", "data", DELETE, 0, 0}, + new Object[] {2, "d", "data", INSERT, 0, 0}); + + validateIterators(rowsWithDuplication, expectedRows); } @Test public void testCarryRowsRemoveMoreInsertRows() { List rowsWithDuplication = Lists.newArrayList( - new GenericRowWithSchema(new Object[] {0, "d", "data", DELETE}, null), - new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE}, null), - new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE}, null), - new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE}, null), + new GenericRowWithSchema(new Object[] {0, "d", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE, 0, 0}, null), // more insert rows than delete rows, should keep extra insert rows - new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT}, null), - new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT}, null), - new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT}, null), - new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT}, null)); - - Iterator iterator = - ChangelogIterator.removeCarryovers(rowsWithDuplication.iterator(), SCHEMA); - List result = Lists.newArrayList(iterator); + new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT, 0, 0}, null)); - assertEquals( - "Rows should match.", + List expectedRows = Lists.newArrayList( - new Object[] {0, "d", "data", DELETE}, new Object[] {1, "d", "data", INSERT}), - rowsToJava(result)); + new Object[] {0, "d", "data", DELETE, 0, 0}, + new Object[] {1, "d", "data", INSERT, 0, 0}); + + validateIterators(rowsWithDuplication, expectedRows); } @Test @@ -296,17 +296,64 @@ public void testCarryRowsRemoveNoInsertRows() { List rowsWithDuplication = Lists.newArrayList( // next two rows are identical - new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE}, null), - new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE}, null)); + new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE, 0, 0}, null)); + + List expectedRows = + Lists.newArrayList( + new Object[] {1, "d", "data", DELETE, 0, 0}, + new Object[] {1, "d", "data", DELETE, 0, 0}); + validateIterators(rowsWithDuplication, expectedRows); + } + + private void validateIterators(List rowsWithDuplication, List expectedRows) { Iterator iterator = ChangelogIterator.removeCarryovers(rowsWithDuplication.iterator(), SCHEMA); List result = Lists.newArrayList(iterator); - assertEquals( - "Duplicate rows should not be removed", + assertEquals("Rows should match.", expectedRows, rowsToJava(result)); + + iterator = ChangelogIterator.removeNetCarryovers(rowsWithDuplication.iterator(), SCHEMA); + result = Lists.newArrayList(iterator); + + assertEquals("Rows should match.", expectedRows, rowsToJava(result)); + } + + @Test + public void testRemoveNetCarryovers() { + List rowsWithDuplication = Lists.newArrayList( - new Object[] {1, "d", "data", DELETE}, new Object[] {1, "d", "data", DELETE}), - rowsToJava(result)); + // this row are different from other rows, it is a net change, should be kept + new GenericRowWithSchema(new Object[] {0, "d", "data", DELETE, 0, 0}, null), + // a pair of delete and insert rows, should be removed + new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT, 0, 0}, null), + // 2 delete rows and 2 insert rows, should be removed + new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE, 1, 1}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE, 1, 1}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT, 1, 1}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT, 1, 1}, null), + // a pair of insert and delete rows across snapshots, should be removed + new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT, 2, 2}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE, 3, 3}, null), + // extra insert rows, they are net changes, should be kept + new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT, 4, 4}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT, 4, 4}, null), + // different key, net changes, should be kept + new GenericRowWithSchema(new Object[] {2, "d", "data", DELETE, 4, 4}, null)); + + List expectedRows = + Lists.newArrayList( + new Object[] {0, "d", "data", DELETE, 0, 0}, + new Object[] {1, "d", "data", INSERT, 4, 4}, + new Object[] {1, "d", "data", INSERT, 4, 4}, + new Object[] {2, "d", "data", DELETE, 4, 4}); + + Iterator iterator = + ChangelogIterator.removeNetCarryovers(rowsWithDuplication.iterator(), SCHEMA); + List result = Lists.newArrayList(iterator); + + assertEquals("Rows should match.", expectedRows, rowsToJava(result)); } } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java index 4dba479edd69..7a63da6068fb 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java @@ -94,6 +94,7 @@ import org.apache.iceberg.spark.SparkTableUtil; import org.apache.iceberg.spark.SparkTestBase; import org.apache.iceberg.spark.actions.RewriteDataFilesSparkAction.RewriteExecutionContext; +import org.apache.iceberg.spark.data.TestHelpers; import org.apache.iceberg.spark.source.ThreeColumnRecord; import org.apache.iceberg.types.Comparators; import org.apache.iceberg.types.Conversions; @@ -254,9 +255,7 @@ public void testBinPackWithDeletes() throws Exception { shouldHaveFiles(table, 8); table.refresh(); - CloseableIterable tasks = table.newScan().planFiles(); - List dataFiles = - Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); + List dataFiles = TestHelpers.dataFiles(table); int total = (int) dataFiles.stream().mapToLong(ContentFile::recordCount).sum(); RowDelta rowDelta = table.newRowDelta(); @@ -300,9 +299,7 @@ public void testBinPackWithDeleteAllData() { shouldHaveFiles(table, 1); table.refresh(); - CloseableIterable tasks = table.newScan().planFiles(); - List dataFiles = - Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); + List dataFiles = TestHelpers.dataFiles(table); int total = (int) dataFiles.stream().mapToLong(ContentFile::recordCount).sum(); RowDelta rowDelta = table.newRowDelta(); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java index 5c0aa2e6aa43..9267fae85f7c 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java @@ -32,7 +32,6 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; -import org.apache.iceberg.FileScanTask; import org.apache.iceberg.Files; import org.apache.iceberg.MetadataTableType; import org.apache.iceberg.MetadataTableUtils; @@ -60,6 +59,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.SparkCatalogConfig; import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.apache.iceberg.spark.data.TestHelpers; import org.apache.iceberg.spark.source.FourColumnRecord; import org.apache.iceberg.spark.source.ThreeColumnRecord; import org.apache.iceberg.types.Types; @@ -136,7 +136,7 @@ public void testEmptyTable() { @Test public void testUnpartitioned() throws Exception { Table table = createTableUnpartitioned(2, SCALE); - List dataFiles = dataFiles(table); + List dataFiles = TestHelpers.dataFiles(table); writePosDeletesForFiles(table, 2, DELETES_SCALE, dataFiles); Assert.assertEquals(2, dataFiles.size()); @@ -170,7 +170,7 @@ public void testUnpartitioned() throws Exception { public void testRewriteAll() throws Exception { Table table = createTablePartitioned(4, 2, SCALE); - List dataFiles = dataFiles(table); + List dataFiles = TestHelpers.dataFiles(table); writePosDeletesForFiles(table, 2, DELETES_SCALE, dataFiles); Assert.assertEquals(4, dataFiles.size()); @@ -206,7 +206,7 @@ public void testRewriteAll() throws Exception { public void testRewriteToSmallerTarget() throws Exception { Table table = createTablePartitioned(4, 2, SCALE); - List dataFiles = dataFiles(table); + List dataFiles = TestHelpers.dataFiles(table); writePosDeletesForFiles(table, 2, DELETES_SCALE, dataFiles); Assert.assertEquals(4, dataFiles.size()); @@ -243,7 +243,7 @@ public void testRewriteToSmallerTarget() throws Exception { public void testRemoveDanglingDeletes() throws Exception { Table table = createTablePartitioned(4, 2, SCALE); - List dataFiles = dataFiles(table); + List dataFiles = TestHelpers.dataFiles(table); writePosDeletesForFiles( table, 2, @@ -288,7 +288,7 @@ public void testRemoveDanglingDeletes() throws Exception { public void testSomePartitionsDanglingDeletes() throws Exception { Table table = createTablePartitioned(4, 2, SCALE); - List dataFiles = dataFiles(table); + List dataFiles = TestHelpers.dataFiles(table); writePosDeletesForFiles(table, 2, DELETES_SCALE, dataFiles); Assert.assertEquals(4, dataFiles.size()); @@ -340,7 +340,7 @@ public void testSomePartitionsDanglingDeletes() throws Exception { @Test public void testPartitionEvolutionAdd() throws Exception { Table table = createTableUnpartitioned(2, SCALE); - List unpartitionedDataFiles = dataFiles(table); + List unpartitionedDataFiles = TestHelpers.dataFiles(table); writePosDeletesForFiles(table, 2, DELETES_SCALE, unpartitionedDataFiles); Assert.assertEquals(2, unpartitionedDataFiles.size()); @@ -354,7 +354,8 @@ public void testPartitionEvolutionAdd() throws Exception { table.updateSpec().addField("c1").commit(); writeRecords(table, 2, SCALE, 2); - List partitionedDataFiles = except(dataFiles(table), unpartitionedDataFiles); + List partitionedDataFiles = + except(TestHelpers.dataFiles(table), unpartitionedDataFiles); writePosDeletesForFiles(table, 2, DELETES_SCALE, partitionedDataFiles); Assert.assertEquals(2, partitionedDataFiles.size()); @@ -391,7 +392,7 @@ public void testPartitionEvolutionAdd() throws Exception { @Test public void testPartitionEvolutionRemove() throws Exception { Table table = createTablePartitioned(2, 2, SCALE); - List dataFilesUnpartitioned = dataFiles(table); + List dataFilesUnpartitioned = TestHelpers.dataFiles(table); writePosDeletesForFiles(table, 2, DELETES_SCALE, dataFilesUnpartitioned); Assert.assertEquals(2, dataFilesUnpartitioned.size()); @@ -401,7 +402,8 @@ public void testPartitionEvolutionRemove() throws Exception { table.updateSpec().removeField("c1").commit(); writeRecords(table, 2, SCALE); - List dataFilesPartitioned = except(dataFiles(table), dataFilesUnpartitioned); + List dataFilesPartitioned = + except(TestHelpers.dataFiles(table), dataFilesUnpartitioned); writePosDeletesForFiles(table, 2, DELETES_SCALE, dataFilesPartitioned); Assert.assertEquals(2, dataFilesPartitioned.size()); @@ -438,7 +440,7 @@ public void testPartitionEvolutionRemove() throws Exception { @Test public void testSchemaEvolution() throws Exception { Table table = createTablePartitioned(2, 2, SCALE); - List dataFiles = dataFiles(table); + List dataFiles = TestHelpers.dataFiles(table); writePosDeletesForFiles(table, 2, DELETES_SCALE, dataFiles); Assert.assertEquals(2, dataFiles.size()); @@ -450,7 +452,7 @@ public void testSchemaEvolution() throws Exception { int newColId = table.schema().findField("c4").fieldId(); List newSchemaDataFiles = - dataFiles(table).stream() + TestHelpers.dataFiles(table).stream() .filter(f -> f.upperBounds().containsKey(newColId)) .collect(Collectors.toList()); writePosDeletesForFiles(table, 2, DELETES_SCALE, newSchemaDataFiles); @@ -679,11 +681,6 @@ private void writePosDeletesForFiles( } } - private List dataFiles(Table table) { - CloseableIterable tasks = table.newScan().includeColumnStats().planFiles(); - return Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); - } - private List deleteFiles(Table table) { Table deletesTable = MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java index 4fbaffad9208..82d0f0daf595 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java @@ -50,6 +50,7 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.TableScan; +import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.relocated.com.google.common.collect.Streams; @@ -782,22 +783,18 @@ public static List deleteManifests(Table table) { return table.currentSnapshot().deleteManifests(table.io()); } - public static Set dataFiles(Table table) { + public static List dataFiles(Table table) { return dataFiles(table, null); } - public static Set dataFiles(Table table, String branch) { - Set dataFiles = Sets.newHashSet(); + public static List dataFiles(Table table, String branch) { TableScan scan = table.newScan(); if (branch != null) { scan.useRef(branch); } - for (FileScanTask task : scan.planFiles()) { - dataFiles.add(task.file()); - } - - return dataFiles; + CloseableIterable tasks = scan.includeColumnStats().planFiles(); + return Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); } public static Set deleteFiles(Table table) { diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java index 342d8085b178..7e254960e759 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java @@ -33,10 +33,12 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.hadoop.HadoopTables; import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.math.LongMath; @@ -53,6 +55,7 @@ import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.assertj.core.api.Assertions; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -428,8 +431,14 @@ public void testExtraSnapshotMetadataWithSQL() throws InterruptedException, IOEx Thread writerThread = new Thread( () -> { - Map properties = Maps.newHashMap(); - properties.put("writer-thread", String.valueOf(Thread.currentThread().getName())); + Map properties = + ImmutableMap.of( + "writer-thread", + String.valueOf(Thread.currentThread().getName()), + SnapshotSummary.EXTRA_METADATA_PREFIX + "extra-key", + "someValue", + SnapshotSummary.EXTRA_METADATA_PREFIX + "another-key", + "anotherValue"); CommitMetadata.withCommitProperties( properties, () -> { @@ -445,8 +454,10 @@ public void testExtraSnapshotMetadataWithSQL() throws InterruptedException, IOEx List snapshots = Lists.newArrayList(table.snapshots()); Assert.assertEquals(2, snapshots.size()); Assert.assertNull(snapshots.get(0).summary().get("writer-thread")); - Assert.assertEquals( - "test-extra-commit-message-writer-thread", snapshots.get(1).summary().get("writer-thread")); + Assertions.assertThat(snapshots.get(1).summary()) + .containsEntry("writer-thread", "test-extra-commit-message-writer-thread") + .containsEntry("extra-key", "someValue") + .containsEntry("another-key", "anotherValue"); } @Test @@ -462,8 +473,14 @@ public void testExtraSnapshotMetadataWithDelete() Thread writerThread = new Thread( () -> { - Map properties = Maps.newHashMap(); - properties.put("writer-thread", String.valueOf(Thread.currentThread().getName())); + Map properties = + ImmutableMap.of( + "writer-thread", + String.valueOf(Thread.currentThread().getName()), + SnapshotSummary.EXTRA_METADATA_PREFIX + "extra-key", + "someValue", + SnapshotSummary.EXTRA_METADATA_PREFIX + "another-key", + "anotherValue"); CommitMetadata.withCommitProperties( properties, () -> { @@ -480,7 +497,9 @@ public void testExtraSnapshotMetadataWithDelete() List snapshots = Lists.newArrayList(table.snapshots()); Assert.assertEquals(2, snapshots.size()); Assert.assertNull(snapshots.get(0).summary().get("writer-thread")); - Assert.assertEquals( - "test-extra-commit-message-delete-thread", snapshots.get(1).summary().get("writer-thread")); + Assertions.assertThat(snapshots.get(1).summary()) + .containsEntry("writer-thread", "test-extra-commit-message-delete-thread") + .containsEntry("extra-key", "someValue") + .containsEntry("another-key", "anotherValue"); } } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index 5c2327c6c508..94b3836884ea 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -28,6 +28,7 @@ import java.io.UncheckedIOException; import java.time.LocalDateTime; import java.time.ZoneOffset; +import java.util.Arrays; import java.util.Comparator; import java.util.List; import java.util.StringJoiner; @@ -48,6 +49,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.actions.DeleteOrphanFiles; +import org.apache.iceberg.actions.RewriteManifests; import org.apache.iceberg.avro.Avro; import org.apache.iceberg.avro.AvroSchemaUtil; import org.apache.iceberg.catalog.TableIdentifier; @@ -1227,7 +1229,7 @@ public void testAllManifestsTable() { @Test public void testUnpartitionedPartitionsTable() { TableIdentifier tableIdentifier = TableIdentifier.of("db", "unpartitioned_partitions_test"); - createTable(tableIdentifier, SCHEMA, PartitionSpec.unpartitioned()); + Table table = createTable(tableIdentifier, SCHEMA, PartitionSpec.unpartitioned()); Dataset df = spark.createDataFrame(Lists.newArrayList(new SimpleRecord(1, "a")), SimpleRecord.class); @@ -1242,6 +1244,11 @@ public void testUnpartitionedPartitionsTable() { Types.StructType.of( required(2, "record_count", Types.LongType.get(), "Count of records in data files"), required(3, "file_count", Types.IntegerType.get(), "Count of data files"), + required( + 11, + "total_data_file_size_in_bytes", + Types.LongType.get(), + "Total size in bytes of data files"), required( 5, "position_delete_record_count", @@ -1261,7 +1268,17 @@ public void testUnpartitionedPartitionsTable() { 8, "equality_delete_file_count", Types.IntegerType.get(), - "Count of equality delete files")); + "Count of equality delete files"), + optional( + 9, + "last_updated_at", + Types.TimestampType.withZone(), + "Commit time of snapshot that last updated this partition"), + optional( + 10, + "last_updated_snapshot_id", + Types.LongType.get(), + "Id of snapshot that last updated this partition")); Table partitionsTable = loadTable(tableIdentifier, "partitions"); @@ -1274,8 +1291,13 @@ public void testUnpartitionedPartitionsTable() { new GenericRecordBuilder(AvroSchemaUtil.convert(partitionsTable.schema(), "partitions")); GenericData.Record expectedRow = builder + .set("last_updated_at", table.currentSnapshot().timestampMillis() * 1000) + .set("last_updated_snapshot_id", table.currentSnapshot().snapshotId()) .set("record_count", 1L) .set("file_count", 1) + .set( + "total_data_file_size_in_bytes", + totalSizeInBytes(table.currentSnapshot().addedDataFiles(table.io()))) .set("position_delete_record_count", 0L) .set("position_delete_file_count", 0) .set("equality_delete_record_count", 0L) @@ -1319,6 +1341,9 @@ public void testPartitionsTable() { .mode("append") .save(loadLocation(tableIdentifier)); + table.refresh(); + long secondCommitId = table.currentSnapshot().snapshotId(); + List actual = spark .read() @@ -1339,22 +1364,32 @@ public void testPartitionsTable() { .set("partition", partitionBuilder.set("id", 1).build()) .set("record_count", 1L) .set("file_count", 1) + .set( + "total_data_file_size_in_bytes", + totalSizeInBytes(table.snapshot(firstCommitId).addedDataFiles(table.io()))) .set("position_delete_record_count", 0L) .set("position_delete_file_count", 0) .set("equality_delete_record_count", 0L) .set("equality_delete_file_count", 0) .set("spec_id", 0) + .set("last_updated_at", table.snapshot(firstCommitId).timestampMillis() * 1000) + .set("last_updated_snapshot_id", firstCommitId) .build()); expected.add( builder .set("partition", partitionBuilder.set("id", 2).build()) .set("record_count", 1L) .set("file_count", 1) + .set( + "total_data_file_size_in_bytes", + totalSizeInBytes(table.snapshot(secondCommitId).addedDataFiles(table.io()))) .set("position_delete_record_count", 0L) .set("position_delete_file_count", 0) .set("equality_delete_record_count", 0L) .set("equality_delete_file_count", 0) .set("spec_id", 0) + .set("last_updated_at", table.snapshot(secondCommitId).timestampMillis() * 1000) + .set("last_updated_snapshot_id", secondCommitId) .build()); Assert.assertEquals("Partitions table should have two rows", 2, expected.size()); @@ -1397,11 +1432,157 @@ public void testPartitionsTable() { .load(loadLocation(tableIdentifier, "partitions")) .filter("partition.id < 2 or record_count=1") .collectAsList(); - Assert.assertEquals("Actual results should have one row", 2, nonFiltered.size()); + Assert.assertEquals("Actual results should have two row", 2, nonFiltered.size()); + for (int i = 0; i < 2; i += 1) { + TestHelpers.assertEqualsSafe( + partitionsTable.schema().asStruct(), expected.get(i), actual.get(i)); + } + } + + @Test + public void testPartitionsTableLastUpdatedSnapshot() { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "partitions_test"); + Table table = createTable(tableIdentifier, SCHEMA, SPEC); + Table partitionsTable = loadTable(tableIdentifier, "partitions"); + Dataset df1 = + spark.createDataFrame( + Lists.newArrayList(new SimpleRecord(1, "1"), new SimpleRecord(2, "2")), + SimpleRecord.class); + Dataset df2 = + spark.createDataFrame(Lists.newArrayList(new SimpleRecord(2, "20")), SimpleRecord.class); + + df1.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + table.refresh(); + long firstCommitId = table.currentSnapshot().snapshotId(); + + // add a second file + df2.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + table.refresh(); + long secondCommitId = table.currentSnapshot().snapshotId(); + + // check if rewrite manifest does not override metadata about data file's creating snapshot + RewriteManifests.Result rewriteManifestResult = + SparkActions.get().rewriteManifests(table).execute(); + Assert.assertEquals( + "rewrite replaced 2 manifests", + 2, + Iterables.size(rewriteManifestResult.rewrittenManifests())); + Assert.assertEquals( + "rewrite added 1 manifests", 1, Iterables.size(rewriteManifestResult.addedManifests())); + + List actual = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "partitions")) + .orderBy("partition.id") + .collectAsList(); + + List dataFiles = TestHelpers.dataFiles(table); + assertDataFilePartitions(dataFiles, Arrays.asList(1, 2, 2)); + + GenericRecordBuilder builder = + new GenericRecordBuilder(AvroSchemaUtil.convert(partitionsTable.schema(), "partitions")); + GenericRecordBuilder partitionBuilder = + new GenericRecordBuilder( + AvroSchemaUtil.convert( + partitionsTable.schema().findType("partition").asStructType(), "partition")); + List expected = Lists.newArrayList(); + expected.add( + builder + .set("partition", partitionBuilder.set("id", 1).build()) + .set("record_count", 1L) + .set("file_count", 1) + .set("total_data_file_size_in_bytes", dataFiles.get(0).fileSizeInBytes()) + .set("position_delete_record_count", 0L) + .set("position_delete_file_count", 0) + .set("equality_delete_record_count", 0L) + .set("equality_delete_file_count", 0) + .set("spec_id", 0) + .set("last_updated_at", table.snapshot(firstCommitId).timestampMillis() * 1000) + .set("last_updated_snapshot_id", firstCommitId) + .build()); + expected.add( + builder + .set("partition", partitionBuilder.set("id", 2).build()) + .set("record_count", 2L) + .set("file_count", 2) + .set( + "total_data_file_size_in_bytes", + dataFiles.get(1).fileSizeInBytes() + dataFiles.get(2).fileSizeInBytes()) + .set("position_delete_record_count", 0L) + .set("position_delete_file_count", 0) + .set("equality_delete_record_count", 0L) + .set("equality_delete_file_count", 0) + .set("spec_id", 0) + .set("last_updated_at", table.snapshot(secondCommitId).timestampMillis() * 1000) + .set("last_updated_snapshot_id", secondCommitId) + .build()); + + Assert.assertEquals("Partitions table should have two rows", 2, expected.size()); + Assert.assertEquals("Actual results should have two rows", 2, actual.size()); for (int i = 0; i < 2; i += 1) { TestHelpers.assertEqualsSafe( partitionsTable.schema().asStruct(), expected.get(i), actual.get(i)); } + + // check predicate push down + List filtered = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "partitions")) + .filter("partition.id < 2") + .collectAsList(); + Assert.assertEquals("Actual results should have one row", 1, filtered.size()); + TestHelpers.assertEqualsSafe( + partitionsTable.schema().asStruct(), expected.get(0), filtered.get(0)); + + // check for snapshot expiration + // if snapshot with firstCommitId is expired, + // we expect the partition of id=1 will no longer have last updated timestamp and snapshotId + SparkActions.get().expireSnapshots(table).expireSnapshotId(firstCommitId).execute(); + GenericData.Record newPartitionRecord = + builder + .set("partition", partitionBuilder.set("id", 1).build()) + .set("record_count", 1L) + .set("file_count", 1) + .set("total_data_file_size_in_bytes", dataFiles.get(0).fileSizeInBytes()) + .set("position_delete_record_count", 0L) + .set("position_delete_file_count", 0) + .set("equality_delete_record_count", 0L) + .set("equality_delete_file_count", 0) + .set("spec_id", 0) + .set("last_updated_at", null) + .set("last_updated_snapshot_id", null) + .build(); + expected.remove(0); + expected.add(0, newPartitionRecord); + + List actualAfterSnapshotExpiration = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "partitions")) + .collectAsList(); + Assert.assertEquals( + "Actual results should have two row", 2, actualAfterSnapshotExpiration.size()); + for (int i = 0; i < 2; i += 1) { + TestHelpers.assertEqualsSafe( + partitionsTable.schema().asStruct(), + expected.get(i), + actualAfterSnapshotExpiration.get(i)); + } } @Test @@ -1421,6 +1602,7 @@ public void testPartitionsTableDeleteStats() { .save(loadLocation(tableIdentifier)); table.refresh(); + long firstCommitId = table.currentSnapshot().snapshotId(); // add a second file df2.select("id", "data") @@ -1433,6 +1615,8 @@ public void testPartitionsTableDeleteStats() { table.updateProperties().set(TableProperties.FORMAT_VERSION, "2").commit(); DeleteFile deleteFile = writePosDeleteFile(table); table.newRowDelta().addDeletes(deleteFile).commit(); + table.refresh(); + long posDeleteCommitId = table.currentSnapshot().snapshotId(); List actual = spark @@ -1455,23 +1639,34 @@ public void testPartitionsTableDeleteStats() { .set("partition", partitionBuilder.set("id", 1).build()) .set("record_count", 1L) .set("file_count", 1) + .set( + "total_data_file_size_in_bytes", + totalSizeInBytes(table.snapshot(firstCommitId).addedDataFiles(table.io()))) .set("position_delete_record_count", 0L) .set("position_delete_file_count", 0) .set("equality_delete_record_count", 0L) .set("equality_delete_file_count", 0) .set("spec_id", 0) + .set("last_updated_at", table.snapshot(firstCommitId).timestampMillis() * 1000) + .set("last_updated_snapshot_id", firstCommitId) .build()); expected.add( builder .set("partition", partitionBuilder.set("id", 2).build()) .set("record_count", 1L) .set("file_count", 1) + .set( + "total_data_file_size_in_bytes", + totalSizeInBytes(table.snapshot(firstCommitId).addedDataFiles(table.io()))) .set("position_delete_record_count", 1L) // should be incremented now .set("position_delete_file_count", 1) // should be incremented now .set("equality_delete_record_count", 0L) .set("equality_delete_file_count", 0) .set("spec_id", 0) + .set("last_updated_at", table.snapshot(posDeleteCommitId).timestampMillis() * 1000) + .set("last_updated_snapshot_id", posDeleteCommitId) .build()); + for (int i = 0; i < 2; i += 1) { TestHelpers.assertEqualsSafe( partitionsTable.schema().asStruct(), expected.get(i), actual.get(i)); @@ -1480,6 +1675,8 @@ public void testPartitionsTableDeleteStats() { // test equality delete DeleteFile eqDeleteFile = writeEqDeleteFile(table); table.newRowDelta().addDeletes(eqDeleteFile).commit(); + table.refresh(); + long eqDeleteCommitId = table.currentSnapshot().snapshotId(); actual = spark .read() @@ -1500,6 +1697,8 @@ public void testPartitionsTableDeleteStats() { .set("equality_delete_record_count", 1L) // should be incremented now .set("equality_delete_file_count", 1) // should be incremented now .set("spec_id", 0) + .set("last_updated_at", table.snapshot(eqDeleteCommitId).timestampMillis() * 1000) + .set("last_updated_snapshot_id", eqDeleteCommitId) .build()); for (int i = 0; i < 2; i += 1) { TestHelpers.assertEqualsSafe( @@ -2057,4 +2256,23 @@ private DeleteFile writeEqDeleteFile(Table table) { throw new RuntimeException(e); } } + + private long totalSizeInBytes(Iterable dataFiles) { + return Lists.newArrayList(dataFiles).stream().mapToLong(DataFile::fileSizeInBytes).sum(); + } + + private void assertDataFilePartitions( + List dataFiles, List expectedPartitionIds) { + Assert.assertEquals( + "Table should have " + expectedPartitionIds.size() + " data files", + expectedPartitionIds.size(), + dataFiles.size()); + + for (int i = 0; i < dataFiles.size(); ++i) { + Assert.assertEquals( + "Data file should have partition of id " + expectedPartitionIds.get(i), + expectedPartitionIds.get(i).intValue(), + dataFiles.get(i).partition().get(0, Integer.class).intValue()); + } + } } diff --git a/spark/v3.4/build.gradle b/spark/v3.4/build.gradle index a6514a9dffea..0ef9aff4ed83 100644 --- a/spark/v3.4/build.gradle +++ b/spark/v3.4/build.gradle @@ -17,7 +17,7 @@ * under the License. */ -String sparkVersion = '3.4.0' +String sparkVersion = '3.4.1' String sparkMajorVersion = '3.4' String scalaVersion = System.getProperty("scalaVersion") != null ? System.getProperty("scalaVersion") : System.getProperty("defaultScalaVersion") @@ -31,9 +31,9 @@ configure(sparkProjects) { configurations { all { resolutionStrategy { - force "com.fasterxml.jackson.module:jackson-module-scala_${scalaVersion}:2.13.4" - force 'com.fasterxml.jackson.core:jackson-databind:2.13.4.2' - force 'com.fasterxml.jackson.core:jackson-core:2.13.4' + force "com.fasterxml.jackson.module:jackson-module-scala_${scalaVersion}:2.14.2" + force 'com.fasterxml.jackson.core:jackson-databind:2.14.2' + force 'com.fasterxml.jackson.core:jackson-core:2.14.2' } } } @@ -165,8 +165,8 @@ project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVer testImplementation "org.apache.parquet:parquet-hadoop" // Required because we remove antlr plugin dependencies from the compile configuration, see note above - runtimeOnly "org.antlr:antlr4-runtime:4.8" - antlr "org.antlr:antlr4:4.8" + runtimeOnly "org.antlr:antlr4-runtime:4.9.3" + antlr "org.antlr:antlr4:4.9.3" } generateGrammarSource { @@ -230,6 +230,7 @@ project(":iceberg-spark:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersio integrationImplementation "org.apache.spark:spark-hive_${scalaVersion}:${sparkVersion}" integrationImplementation 'org.junit.vintage:junit-vintage-engine' integrationImplementation 'org.slf4j:slf4j-simple' + integrationImplementation 'org.assertj:assertj-core' integrationImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') integrationImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') integrationImplementation project(path: ":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}", configuration: 'testArtifacts') diff --git a/spark/v3.4/spark-extensions/src/jmh/java/org/apache/iceberg/spark/PlanningBenchmark.java b/spark/v3.4/spark-extensions/src/jmh/java/org/apache/iceberg/spark/PlanningBenchmark.java new file mode 100644 index 000000000000..6e75b6aa00c4 --- /dev/null +++ b/spark/v3.4/spark-extensions/src/jmh/java/org/apache/iceberg/spark/PlanningBenchmark.java @@ -0,0 +1,301 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark; + +import static org.apache.spark.sql.functions.lit; + +import com.google.errorprone.annotations.FormatMethod; +import com.google.errorprone.annotations.FormatString; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.RowLevelOperationMode; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.data.RandomData; +import org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.parser.ParseException; +import org.apache.spark.sql.connector.read.InputPartition; +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation; +import org.apache.spark.sql.types.StructType; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Timeout; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +/** + * A benchmark that evaluates the job planning performance. + * + *

    To run this benchmark for spark-3.4: + * ./gradlew -DsparkVersions=3.4 :iceberg-spark:iceberg-spark-extensions-3.4_2.12:jmh + * -PjmhIncludeRegex=PlanningBenchmark + * -PjmhOutputPath=benchmark/iceberg-planning-benchmark.txt + * + */ +@Fork(1) +@State(Scope.Benchmark) +@Warmup(iterations = 3) +@Measurement(iterations = 5) +@Timeout(time = 20, timeUnit = TimeUnit.MINUTES) +@BenchmarkMode(Mode.SingleShotTime) +public class PlanningBenchmark { + + private static final String TABLE_NAME = "test_table"; + private static final String PARTITION_COLUMN = "ss_ticket_number"; + private static final int PARTITION_VALUE = 10; + private static final String SORT_KEY_COLUMN = "ss_sold_date_sk"; + private static final int SORT_KEY_VALUE = 5; + + private static final String SORT_KEY_PREDICATE = + String.format("%s = %s", SORT_KEY_COLUMN, SORT_KEY_VALUE); + private static final String PARTITION_AND_SORT_KEY_PREDICATE = + String.format( + "%s = %d AND %s = %d", + PARTITION_COLUMN, PARTITION_VALUE, SORT_KEY_COLUMN, SORT_KEY_VALUE); + + private static final int NUM_PARTITIONS = 30; + private static final int NUM_DATA_FILES_PER_PARTITION = 50_000; + private static final int NUM_DELETE_FILES_PER_PARTITION = 50; + private static final int NUM_ROWS_PER_DATA_FILE = 500; + + private final Configuration hadoopConf = new Configuration(); + private SparkSession spark; + private Table table; + + @Setup + public void setupBenchmark() throws NoSuchTableException, ParseException { + setupSpark(); + initTable(); + initDataAndDeletes(); + } + + @TearDown + public void tearDownBenchmark() { + dropTable(); + tearDownSpark(); + } + + @Benchmark + @Threads(1) + public void localPlanningWithPartitionAndMinMaxFilter(Blackhole blackhole) { + InputPartition[] partitions = planInputPartitions(PARTITION_AND_SORT_KEY_PREDICATE); + blackhole.consume(partitions); + } + + @Benchmark + @Threads(1) + public void localPlanningWithMinMaxFilter(Blackhole blackhole) { + InputPartition[] partitions = planInputPartitions(SORT_KEY_PREDICATE); + blackhole.consume(partitions); + } + + @Benchmark + @Threads(1) + public void localPlanningWithoutFilter(Blackhole blackhole) { + InputPartition[] partitions = planInputPartitions("true"); + blackhole.consume(partitions); + } + + private void setupSpark() { + this.spark = + SparkSession.builder() + .config("spark.ui.enabled", false) + .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .config("spark.sql.extensions", IcebergSparkSessionExtensions.class.getName()) + .config("spark.sql.catalog.spark_catalog", SparkSessionCatalog.class.getName()) + .config("spark.sql.catalog.spark_catalog.type", "hadoop") + .config("spark.sql.catalog.spark_catalog.warehouse", newWarehouseDir()) + .master("local[*]") + .getOrCreate(); + } + + private void tearDownSpark() { + spark.stop(); + } + + private void initTable() throws NoSuchTableException, ParseException { + sql( + "CREATE TABLE %s ( " + + " `ss_sold_date_sk` INT, " + + " `ss_sold_time_sk` INT, " + + " `ss_item_sk` INT, " + + " `ss_customer_sk` STRING, " + + " `ss_cdemo_sk` STRING, " + + " `ss_hdemo_sk` STRING, " + + " `ss_addr_sk` STRING, " + + " `ss_store_sk` STRING, " + + " `ss_promo_sk` STRING, " + + " `ss_ticket_number` INT, " + + " `ss_quantity` STRING, " + + " `ss_wholesale_cost` STRING, " + + " `ss_list_price` STRING, " + + " `ss_sales_price` STRING, " + + " `ss_ext_discount_amt` STRING, " + + " `ss_ext_sales_price` STRING, " + + " `ss_ext_wholesale_cost` STRING, " + + " `ss_ext_list_price` STRING, " + + " `ss_ext_tax` STRING, " + + " `ss_coupon_amt` STRING, " + + " `ss_net_paid` STRING, " + + " `ss_net_paid_inc_tax` STRING, " + + " `ss_net_profit` STRING " + + ")" + + "USING iceberg " + + "PARTITIONED BY (%s) " + + "TBLPROPERTIES (" + + " '%s' '%s'," + + " '%s' '%d')", + TABLE_NAME, + PARTITION_COLUMN, + TableProperties.DELETE_MODE, + RowLevelOperationMode.MERGE_ON_READ.modeName(), + TableProperties.FORMAT_VERSION, + 2); + + this.table = Spark3Util.loadIcebergTable(spark, TABLE_NAME); + } + + private void dropTable() { + sql("DROP TABLE IF EXISTS %s PURGE", TABLE_NAME); + } + + private DataFile loadAddedDataFile() { + table.refresh(); + + Iterable dataFiles = table.currentSnapshot().addedDataFiles(table.io()); + return Iterables.getOnlyElement(dataFiles); + } + + private DeleteFile loadAddedDeleteFile() { + table.refresh(); + + Iterable deleteFiles = table.currentSnapshot().addedDeleteFiles(table.io()); + return Iterables.getOnlyElement(deleteFiles); + } + + private void initDataAndDeletes() throws NoSuchTableException { + for (int partitionOrdinal = 0; partitionOrdinal < NUM_PARTITIONS; partitionOrdinal++) { + Dataset inputDF = + randomDataDF(table.schema(), NUM_ROWS_PER_DATA_FILE) + .drop(PARTITION_COLUMN) + .withColumn(PARTITION_COLUMN, lit(partitionOrdinal)); + appendAsFile(inputDF); + + DataFile dataFile = loadAddedDataFile(); + + sql( + "DELETE FROM %s WHERE ss_item_sk IS NULL AND %s = %d", + TABLE_NAME, PARTITION_COLUMN, partitionOrdinal); + + DeleteFile deleteFile = loadAddedDeleteFile(); + + AppendFiles append = table.newFastAppend(); + + for (int fileOrdinal = 0; fileOrdinal < NUM_DATA_FILES_PER_PARTITION; fileOrdinal++) { + DataFile replicaDataFile = + DataFiles.builder(table.spec()) + .copy(dataFile) + .withPath("replica-" + fileOrdinal + "-" + dataFile.path()) + .build(); + append.appendFile(replicaDataFile); + } + + append.commit(); + + RowDelta rowDelta = table.newRowDelta(); + + for (int fileOrdinal = 0; fileOrdinal < NUM_DELETE_FILES_PER_PARTITION; fileOrdinal++) { + DeleteFile replicaDeleteFile = + FileMetadata.deleteFileBuilder(table.spec()) + .copy(deleteFile) + .withPath("replica-" + fileOrdinal + "-" + deleteFile.path()) + .build(); + rowDelta.addDeletes(replicaDeleteFile); + } + + rowDelta.commit(); + + Dataset sortedInputDF = + randomDataDF(table.schema(), NUM_ROWS_PER_DATA_FILE) + .drop(SORT_KEY_COLUMN) + .withColumn(SORT_KEY_COLUMN, lit(SORT_KEY_VALUE)) + .drop(PARTITION_COLUMN) + .withColumn(PARTITION_COLUMN, lit(partitionOrdinal)); + appendAsFile(sortedInputDF); + } + } + + private void appendAsFile(Dataset df) throws NoSuchTableException { + df.coalesce(1).writeTo(TABLE_NAME).append(); + } + + private String newWarehouseDir() { + return hadoopConf.get("hadoop.tmp.dir") + UUID.randomUUID(); + } + + private Dataset randomDataDF(Schema schema, int numRows) { + Iterable rows = RandomData.generateSpark(schema, numRows, 0); + JavaSparkContext context = JavaSparkContext.fromSparkContext(spark.sparkContext()); + JavaRDD rowRDD = context.parallelize(Lists.newArrayList(rows)); + StructType rowSparkType = SparkSchemaUtil.convert(schema); + return spark.internalCreateDataFrame(JavaRDD.toRDD(rowRDD), rowSparkType, false); + } + + private InputPartition[] planInputPartitions(String predicate) { + DataSourceV2ScanRelation relation = + (DataSourceV2ScanRelation) + spark + .sql(String.format("SELECT * FROM %s WHERE %s", TABLE_NAME, predicate)) + .queryExecution() + .optimizedPlan() + .collectLeaves() + .head(); + return relation.scan().toBatch().planInputPartitions(); + } + + @FormatMethod + private void sql(@FormatString String query, Object... args) { + spark.sql(String.format(query, args)); + } +} diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala index f758cb08fd3d..2e438de2b8cd 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala @@ -116,6 +116,7 @@ class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergS val branchRetention = branchOptionsContext.flatMap(branchOptions => Option(branchOptions.refRetain())) val branchRefAgeMs = branchRetention.map(retain => TimeUnit.valueOf(retain.timeUnit().getText.toUpperCase(Locale.ENGLISH)).toMillis(retain.number().getText.toLong)) + val create = createOrReplaceBranchClause.CREATE() != null val replace = ctx.createReplaceBranchClause().REPLACE() != null val ifNotExists = createOrReplaceBranchClause.EXISTS() != null @@ -130,6 +131,7 @@ class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergS typedVisit[Seq[String]](ctx.multipartIdentifier), branchName.getText, branchOptions, + create, replace, ifNotExists) } @@ -153,12 +155,14 @@ class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergS tagRefAgeMs ) + val create = createTagClause.CREATE() != null val replace = createTagClause.REPLACE() != null val ifNotExists = createTagClause.EXISTS() != null CreateOrReplaceTag(typedVisit[Seq[String]](ctx.multipartIdentifier), tagName, tagOptions, + create, replace, ifNotExists) } diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala index 2a22484499cf..b7981a3c7a0d 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala @@ -25,6 +25,7 @@ case class CreateOrReplaceBranch( table: Seq[String], branch: String, branchOptions: BranchOptions, + create: Boolean, replace: Boolean, ifNotExists: Boolean) extends LeafCommand { diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala index e48f7d8ed04c..6e7db84a90fb 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala @@ -25,6 +25,7 @@ case class CreateOrReplaceTag( table: Seq[String], tag: String, tagOptions: TagOptions, + create: Boolean, replace: Boolean, ifNotExists: Boolean) extends LeafCommand { diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala index 6457875b15a4..2be406e7f344 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala @@ -32,6 +32,7 @@ case class CreateOrReplaceBranchExec( ident: Identifier, branch: String, branchOptions: BranchOptions, + create: Boolean, replace: Boolean, ifNotExists: Boolean) extends LeafV2CommandExec { @@ -47,19 +48,29 @@ case class CreateOrReplaceBranchExec( .map(java.lang.Long.valueOf) .orNull - Preconditions.checkArgument(snapshotId != null, - "Cannot complete create or replace branch operation on %s, main has no snapshot", ident) - val manageSnapshots = iceberg.table().manageSnapshots() - if (!replace) { - val ref = iceberg.table().refs().get(branch) - if (ref != null && ifNotExists) { - return Nil + val refExists = null != iceberg.table().refs().get(branch) + + def safeCreateBranch(): Unit = { + if (snapshotId == null) { + manageSnapshots.createBranch(branch) + } else { + manageSnapshots.createBranch(branch, snapshotId) } + } - manageSnapshots.createBranch(branch, snapshotId) - } else { + if (create && replace && !refExists) { + safeCreateBranch() + } else if (replace) { + Preconditions.checkArgument(snapshotId != null, + "Cannot complete replace branch operation on %s, main has no snapshot", ident) manageSnapshots.replaceBranch(branch, snapshotId) + } else { + if (refExists && ifNotExists) { + return Nil + } + + safeCreateBranch() } if (branchOptions.numSnapshots.nonEmpty) { diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala index 7ca193d1b156..372cd7548632 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala @@ -31,6 +31,7 @@ case class CreateOrReplaceTagExec( ident: Identifier, tag: String, tagOptions: TagOptions, + create: Boolean, replace: Boolean, ifNotExists: Boolean) extends LeafV2CommandExec { @@ -50,15 +51,18 @@ case class CreateOrReplaceTagExec( "Cannot complete create or replace tag operation on %s, main has no snapshot", ident) val manageSnapshot = iceberg.table.manageSnapshots() - if (!replace) { - val ref = iceberg.table().refs().get(tag) - if (ref != null && ifNotExists) { + val refExists = null != iceberg.table().refs().get(tag) + + if (create && replace && !refExists) { + manageSnapshot.createTag(tag, snapshotId) + } else if (replace) { + manageSnapshot.replaceTag(tag, snapshotId) + } else { + if (refExists && ifNotExists) { return Nil } manageSnapshot.createTag(tag, snapshotId) - } else { - manageSnapshot.replaceTag(tag, snapshotId) } if (tagOptions.snapshotRefRetain.nonEmpty) { diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala index aa81e38f29d3..3af5ddd131e9 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala @@ -39,6 +39,7 @@ import org.apache.spark.sql.catalyst.plans.logical.DropTag import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.logical.MergeRows import org.apache.spark.sql.catalyst.plans.logical.NoStatsUnaryNode +import org.apache.spark.sql.catalyst.plans.logical.OrderAwareCoalesce import org.apache.spark.sql.catalyst.plans.logical.ReplaceIcebergData import org.apache.spark.sql.catalyst.plans.logical.ReplacePartitionField import org.apache.spark.sql.catalyst.plans.logical.SetIdentifierFields @@ -47,6 +48,7 @@ import org.apache.spark.sql.catalyst.plans.logical.UpdateRows import org.apache.spark.sql.catalyst.plans.logical.WriteIcebergDelta import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.TableCatalog +import org.apache.spark.sql.execution.OrderAwareCoalesceExec import org.apache.spark.sql.execution.SparkPlan import scala.jdk.CollectionConverters._ @@ -61,11 +63,12 @@ case class ExtendedDataSourceV2Strategy(spark: SparkSession) extends Strategy wi AddPartitionFieldExec(catalog, ident, transform, name) :: Nil case CreateOrReplaceBranch( - IcebergCatalogAndIdentifier(catalog, ident), branch, branchOptions, replace, ifNotExists) => - CreateOrReplaceBranchExec(catalog, ident, branch, branchOptions, replace, ifNotExists) :: Nil + IcebergCatalogAndIdentifier(catalog, ident), branch, branchOptions, create, replace, ifNotExists) => + CreateOrReplaceBranchExec(catalog, ident, branch, branchOptions, create, replace, ifNotExists) :: Nil - case CreateOrReplaceTag(IcebergCatalogAndIdentifier(catalog, ident), tag, tagOptions, replace, ifNotExists) => - CreateOrReplaceTagExec(catalog, ident, tag, tagOptions, replace, ifNotExists) :: Nil + case CreateOrReplaceTag( + IcebergCatalogAndIdentifier(catalog, ident), tag, tagOptions, create, replace, ifNotExists) => + CreateOrReplaceTagExec(catalog, ident, tag, tagOptions, create, replace, ifNotExists) :: Nil case DropBranch(IcebergCatalogAndIdentifier(catalog, ident), branch, ifExists) => DropBranchExec(catalog, ident, branch, ifExists) :: Nil @@ -111,6 +114,9 @@ case class ExtendedDataSourceV2Strategy(spark: SparkSession) extends Strategy wi case NoStatsUnaryNode(child) => planLater(child) :: Nil + case OrderAwareCoalesce(numPartitions, coalescer, child) => + OrderAwareCoalesceExec(numPartitions, coalescer, planLater(child)) :: Nil + case _ => Nil } diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java index e02065bd6347..db8ffe07b8b7 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java @@ -32,7 +32,6 @@ import org.apache.avro.generic.GenericDatumWriter; import org.apache.avro.generic.GenericRecord; import org.apache.avro.io.DatumWriter; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.DataFile; import org.apache.iceberg.TableProperties; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -44,6 +43,7 @@ import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; +import org.assertj.core.api.Assertions; import org.joda.time.DateTime; import org.junit.After; import org.junit.Assert; @@ -708,23 +708,21 @@ public void invalidDataImport() { sql(createIceberg, tableName); - AssertHelpers.assertThrows( - "Should forbid adding of partitioned data to unpartitioned table", - IllegalArgumentException.class, - "Cannot use partition filter with an unpartitioned table", - () -> - scalarSql( - "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('id', 1))", - catalogName, tableName, fileTableDir.getAbsolutePath())); - - AssertHelpers.assertThrows( - "Should forbid adding of partitioned data to unpartitioned table", - IllegalArgumentException.class, - "Cannot add partitioned files to an unpartitioned table", - () -> - scalarSql( - "CALL %s.system.add_files('%s', '`parquet`.`%s`')", - catalogName, tableName, fileTableDir.getAbsolutePath())); + Assertions.assertThatThrownBy( + () -> + scalarSql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('id', 1))", + catalogName, tableName, fileTableDir.getAbsolutePath())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot use partition filter with an unpartitioned table"); + + Assertions.assertThatThrownBy( + () -> + scalarSql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`')", + catalogName, tableName, fileTableDir.getAbsolutePath())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot add partitioned files to an unpartitioned table"); } @Test @@ -736,23 +734,24 @@ public void invalidDataImportPartitioned() { sql(createIceberg, tableName); - AssertHelpers.assertThrows( - "Should forbid adding with a mismatching partition spec", - IllegalArgumentException.class, - "is greater than the number of partitioned columns", - () -> - scalarSql( - "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('x', '1', 'y', '2'))", - catalogName, tableName, fileTableDir.getAbsolutePath())); - - AssertHelpers.assertThrows( - "Should forbid adding with partition spec with incorrect columns", - IllegalArgumentException.class, - "specified partition filter refers to columns that are not partitioned", - () -> - scalarSql( - "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('dept', '2'))", - catalogName, tableName, fileTableDir.getAbsolutePath())); + Assertions.assertThatThrownBy( + () -> + scalarSql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('x', '1', 'y', '2'))", + catalogName, tableName, fileTableDir.getAbsolutePath())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot add data files to target table") + .hasMessageContaining("is greater than the number of partitioned columns"); + + Assertions.assertThatThrownBy( + () -> + scalarSql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('dept', '2'))", + catalogName, tableName, fileTableDir.getAbsolutePath())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot add files to target table") + .hasMessageContaining( + "specified partition filter refers to columns that are not partitioned"); } @Test @@ -808,18 +807,18 @@ public void duplicateDataPartitioned() { + "partition_filter => map('id', 1))", catalogName, tableName, sourceTableName); - AssertHelpers.assertThrows( - "Should not allow adding duplicate files", - IllegalStateException.class, - "Cannot complete import because data files to be imported already" - + " exist within the target table", - () -> - scalarSql( - "CALL %s.system.add_files(" - + "table => '%s', " - + "source_table => '%s', " - + "partition_filter => map('id', 1))", - catalogName, tableName, sourceTableName)); + Assertions.assertThatThrownBy( + () -> + scalarSql( + "CALL %s.system.add_files(" + + "table => '%s', " + + "source_table => '%s', " + + "partition_filter => map('id', 1))", + catalogName, tableName, sourceTableName)) + .isInstanceOf(IllegalStateException.class) + .hasMessageStartingWith( + "Cannot complete import because data files to be imported already" + + " exist within the target table"); } @Test @@ -872,14 +871,15 @@ public void duplicateDataUnpartitioned() { sql("CALL %s.system.add_files('%s', '%s')", catalogName, tableName, sourceTableName); - AssertHelpers.assertThrows( - "Should not allow adding duplicate files", - IllegalStateException.class, - "Cannot complete import because data files to be imported already" - + " exist within the target table", - () -> - scalarSql( - "CALL %s.system.add_files('%s', '%s')", catalogName, tableName, sourceTableName)); + Assertions.assertThatThrownBy( + () -> + scalarSql( + "CALL %s.system.add_files('%s', '%s')", + catalogName, tableName, sourceTableName)) + .isInstanceOf(IllegalStateException.class) + .hasMessageStartingWith( + "Cannot complete import because data files to be imported already" + + " exist within the target table"); } @Test diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java index c993c213dc5e..7a6eb9aa387f 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java @@ -19,9 +19,9 @@ package org.apache.iceberg.spark.extensions; import java.util.Map; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -77,17 +77,14 @@ public void testSetInvalidIdentifierFields() { Table table = validationCatalog.loadTable(tableIdent); Assert.assertTrue( "Table should start without identifier", table.schema().identifierFieldIds().isEmpty()); - AssertHelpers.assertThrows( - "should not allow setting unknown fields", - IllegalArgumentException.class, - "not found in current schema or added columns", - () -> sql("ALTER TABLE %s SET IDENTIFIER FIELDS unknown", tableName)); - - AssertHelpers.assertThrows( - "should not allow setting optional fields", - IllegalArgumentException.class, - "not a required field", - () -> sql("ALTER TABLE %s SET IDENTIFIER FIELDS id2", tableName)); + Assertions.assertThatThrownBy( + () -> sql("ALTER TABLE %s SET IDENTIFIER FIELDS unknown", tableName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageEndingWith("not found in current schema or added columns"); + + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s SET IDENTIFIER FIELDS id2", tableName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageEndingWith("not a required field"); } @Test @@ -140,23 +137,22 @@ public void testDropInvalidIdentifierFields() { Table table = validationCatalog.loadTable(tableIdent); Assert.assertTrue( "Table should start without identifier", table.schema().identifierFieldIds().isEmpty()); - AssertHelpers.assertThrows( - "should not allow dropping unknown fields", - IllegalArgumentException.class, - "field unknown not found", - () -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS unknown", tableName)); + Assertions.assertThatThrownBy( + () -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS unknown", tableName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot complete drop identifier fields operation: field unknown not found"); sql("ALTER TABLE %s SET IDENTIFIER FIELDS id", tableName); - AssertHelpers.assertThrows( - "should not allow dropping a field that is not an identifier", - IllegalArgumentException.class, - "data is not an identifier field", - () -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS data", tableName)); - - AssertHelpers.assertThrows( - "should not allow dropping a nested field that is not an identifier", - IllegalArgumentException.class, - "location.lon is not an identifier field", - () -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS location.lon", tableName)); + Assertions.assertThatThrownBy( + () -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS data", tableName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot complete drop identifier fields operation: data is not an identifier field"); + + Assertions.assertThatThrownBy( + () -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS location.lon", tableName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot complete drop identifier fields operation: location.lon is not an identifier field"); } } diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java index ae591821e21a..3282cdf381ce 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java @@ -20,10 +20,10 @@ import java.util.List; import java.util.Map; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.spark.sql.AnalysisException; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Test; @@ -147,22 +147,17 @@ public void testAncestorOfUsingNamedArgs() { @Test public void testInvalidAncestorOfCases() { - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.ancestors_of()", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with empty table identifier", - IllegalArgumentException.class, - "Cannot handle an empty identifier for parameter 'table'", - () -> sql("CALL %s.system.ancestors_of('')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with invalid arg types", - AnalysisException.class, - "Wrong arg type for snapshot_id: cannot cast", - () -> sql("CALL %s.system.ancestors_of('%s', 1.1)", catalogName, tableIdent)); + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.ancestors_of()", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage("Missing required parameters: [table]"); + + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.ancestors_of('')", catalogName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot handle an empty identifier for parameter 'table'"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.ancestors_of('%s', 1.1)", catalogName, tableIdent)) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith("Wrong arg type for snapshot_id: cannot cast"); } } diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java index 5dd8c751863a..a6bf194b3df5 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java @@ -18,10 +18,12 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; -import org.apache.iceberg.AssertHelpers; +import org.apache.iceberg.Snapshot; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -83,20 +85,33 @@ public void testCreateBranch() throws NoSuchTableException { Assert.assertEquals(TimeUnit.DAYS.toMillis(maxSnapshotAge), ref.maxSnapshotAgeMs().longValue()); Assert.assertEquals(TimeUnit.DAYS.toMillis(maxRefAge), ref.maxRefAgeMs().longValue()); - AssertHelpers.assertThrows( - "Cannot create an existing branch", - IllegalArgumentException.class, - "Ref b1 already exists", - () -> sql("ALTER TABLE %s CREATE BRANCH %s", tableName, branchName)); + Assertions.assertThatThrownBy( + () -> sql("ALTER TABLE %s CREATE BRANCH %s", tableName, branchName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Ref b1 already exists"); } @Test public void testCreateBranchOnEmptyTable() { - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE BRANCH %s", tableName, "b1")) - .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining( - "Cannot complete create or replace branch operation on %s, main has no snapshot", - tableName); + String branchName = "b1"; + sql("ALTER TABLE %s CREATE BRANCH %s", tableName, "b1"); + Table table = validationCatalog.loadTable(tableIdent); + + SnapshotRef mainRef = table.refs().get(SnapshotRef.MAIN_BRANCH); + Assertions.assertThat(mainRef).isNull(); + + SnapshotRef ref = table.refs().get(branchName); + Assertions.assertThat(ref).isNotNull(); + Assertions.assertThat(ref.minSnapshotsToKeep()).isNull(); + Assertions.assertThat(ref.maxSnapshotAgeMs()).isNull(); + Assertions.assertThat(ref.maxRefAgeMs()).isNull(); + + Snapshot snapshot = table.snapshot(ref.snapshotId()); + Assertions.assertThat(snapshot.parentId()).isNull(); + Assertions.assertThat(snapshot.addedDataFiles(table.io())).isEmpty(); + Assertions.assertThat(snapshot.removedDataFiles(table.io())).isEmpty(); + Assertions.assertThat(snapshot.addedDeleteFiles(table.io())).isEmpty(); + Assertions.assertThat(snapshot.removedDeleteFiles(table.io())).isEmpty(); } @Test @@ -179,12 +194,13 @@ public void testCreateBranchUseCustomMinSnapshotsToKeepAndMaxSnapshotAge() Assert.assertEquals(TimeUnit.DAYS.toMillis(maxSnapshotAge), ref.maxSnapshotAgeMs().longValue()); Assert.assertNull(ref.maxRefAgeMs()); - AssertHelpers.assertThrows( - "Illegal statement", - IcebergParseException.class, - "no viable alternative at input 'WITH SNAPSHOT RETENTION'", - () -> - sql("ALTER TABLE %s CREATE BRANCH %s WITH SNAPSHOT RETENTION", tableName, branchName)); + Assertions.assertThatThrownBy( + () -> + sql( + "ALTER TABLE %s CREATE BRANCH %s WITH SNAPSHOT RETENTION", + tableName, branchName)) + .isInstanceOf(IcebergParseException.class) + .hasMessageContaining("no viable alternative at input 'WITH SNAPSHOT RETENTION'"); } @Test @@ -200,26 +216,24 @@ public void testCreateBranchUseCustomMaxRefAge() throws NoSuchTableException { Assert.assertNull(ref.maxSnapshotAgeMs()); Assert.assertEquals(TimeUnit.DAYS.toMillis(maxRefAge), ref.maxRefAgeMs().longValue()); - AssertHelpers.assertThrows( - "Illegal statement", - IcebergParseException.class, - "mismatched input", - () -> sql("ALTER TABLE %s CREATE BRANCH %s RETAIN", tableName, branchName)); - - AssertHelpers.assertThrows( - "Illegal statement", - IcebergParseException.class, - "mismatched input", - () -> sql("ALTER TABLE %s CREATE BRANCH %s RETAIN %s DAYS", tableName, branchName, "abc")); - - AssertHelpers.assertThrows( - "Illegal statement", - IcebergParseException.class, - "mismatched input 'SECONDS' expecting {'DAYS', 'HOURS', 'MINUTES'}", - () -> - sql( - "ALTER TABLE %s CREATE BRANCH %s RETAIN %d SECONDS", - tableName, branchName, maxRefAge)); + Assertions.assertThatThrownBy( + () -> sql("ALTER TABLE %s CREATE BRANCH %s RETAIN", tableName, branchName)) + .isInstanceOf(IcebergParseException.class) + .hasMessageContaining("mismatched input"); + + Assertions.assertThatThrownBy( + () -> + sql("ALTER TABLE %s CREATE BRANCH %s RETAIN %s DAYS", tableName, branchName, "abc")) + .isInstanceOf(IcebergParseException.class) + .hasMessageContaining("mismatched input"); + + Assertions.assertThatThrownBy( + () -> + sql( + "ALTER TABLE %s CREATE BRANCH %s RETAIN %d SECONDS", + tableName, branchName, maxRefAge)) + .isInstanceOf(IcebergParseException.class) + .hasMessageContaining("mismatched input 'SECONDS' expecting {'DAYS', 'HOURS', 'MINUTES'}"); } @Test @@ -241,11 +255,10 @@ public void testDropBranch() throws NoSuchTableException { @Test public void testDropBranchDoesNotExist() { - AssertHelpers.assertThrows( - "Cannot perform drop branch on branch which does not exist", - IllegalArgumentException.class, - "Branch does not exist: nonExistingBranch", - () -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, "nonExistingBranch")); + Assertions.assertThatThrownBy( + () -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, "nonExistingBranch")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Branch does not exist: nonExistingBranch"); } @Test @@ -254,29 +267,23 @@ public void testDropBranchFailsForTag() throws NoSuchTableException { Table table = insertRows(); table.manageSnapshots().createTag(tagName, table.currentSnapshot().snapshotId()).commit(); - AssertHelpers.assertThrows( - "Cannot perform drop branch on tag", - IllegalArgumentException.class, - "Ref b1 is a tag not a branch", - () -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, tagName)); + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, tagName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Ref b1 is a tag not a branch"); } @Test public void testDropBranchNonConformingName() { - AssertHelpers.assertThrows( - "Non-conforming branch name", - IcebergParseException.class, - "mismatched input '123'", - () -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, "123")); + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, "123")) + .isInstanceOf(IcebergParseException.class) + .hasMessageContaining("mismatched input '123'"); } @Test public void testDropMainBranchFails() { - AssertHelpers.assertThrows( - "Cannot drop the main branch", - IllegalArgumentException.class, - "Cannot remove main branch", - () -> sql("ALTER TABLE %s DROP BRANCH main", tableName)); + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s DROP BRANCH main", tableName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot remove main branch"); } @Test @@ -299,4 +306,92 @@ private Table insertRows() throws NoSuchTableException { df.writeTo(tableName).append(); return validationCatalog.loadTable(tableIdent); } + + @Test + public void createOrReplace() throws NoSuchTableException { + Table table = insertRows(); + long first = table.currentSnapshot().snapshotId(); + String branchName = "b1"; + insertRows(); + long second = table.currentSnapshot().snapshotId(); + table.manageSnapshots().createBranch(branchName, second).commit(); + + sql( + "ALTER TABLE %s CREATE OR REPLACE BRANCH %s AS OF VERSION %d", + tableName, branchName, first); + table.refresh(); + assertThat(table.refs().get(branchName).snapshotId()).isEqualTo(second); + } + + @Test + public void testCreateOrReplaceBranchOnEmptyTable() { + String branchName = "b1"; + sql("ALTER TABLE %s CREATE OR REPLACE BRANCH %s", tableName, "b1"); + Table table = validationCatalog.loadTable(tableIdent); + + SnapshotRef mainRef = table.refs().get(SnapshotRef.MAIN_BRANCH); + Assertions.assertThat(mainRef).isNull(); + + SnapshotRef ref = table.refs().get(branchName); + Assertions.assertThat(ref).isNotNull(); + Assertions.assertThat(ref.minSnapshotsToKeep()).isNull(); + Assertions.assertThat(ref.maxSnapshotAgeMs()).isNull(); + Assertions.assertThat(ref.maxRefAgeMs()).isNull(); + + Snapshot snapshot = table.snapshot(ref.snapshotId()); + Assertions.assertThat(snapshot.parentId()).isNull(); + Assertions.assertThat(snapshot.addedDataFiles(table.io())).isEmpty(); + Assertions.assertThat(snapshot.removedDataFiles(table.io())).isEmpty(); + Assertions.assertThat(snapshot.addedDeleteFiles(table.io())).isEmpty(); + Assertions.assertThat(snapshot.removedDeleteFiles(table.io())).isEmpty(); + } + + @Test + public void createOrReplaceWithNonExistingBranch() throws NoSuchTableException { + Table table = insertRows(); + String branchName = "b1"; + insertRows(); + long snapshotId = table.currentSnapshot().snapshotId(); + + sql( + "ALTER TABLE %s CREATE OR REPLACE BRANCH %s AS OF VERSION %d", + tableName, branchName, snapshotId); + table.refresh(); + assertThat(table.refs().get(branchName).snapshotId()).isEqualTo(snapshotId); + } + + @Test + public void replaceBranch() throws NoSuchTableException { + Table table = insertRows(); + long first = table.currentSnapshot().snapshotId(); + String branchName = "b1"; + long expectedMaxRefAgeMs = 1000; + table + .manageSnapshots() + .createBranch(branchName, first) + .setMaxRefAgeMs(branchName, expectedMaxRefAgeMs) + .commit(); + + insertRows(); + long second = table.currentSnapshot().snapshotId(); + + sql("ALTER TABLE %s REPLACE BRANCH %s AS OF VERSION %d", tableName, branchName, second); + table.refresh(); + SnapshotRef ref = table.refs().get(branchName); + assertThat(ref.snapshotId()).isEqualTo(second); + assertThat(ref.maxRefAgeMs()).isEqualTo(expectedMaxRefAgeMs); + } + + @Test + public void replaceBranchDoesNotExist() throws NoSuchTableException { + Table table = insertRows(); + + Assertions.assertThatThrownBy( + () -> + sql( + "ALTER TABLE %s REPLACE BRANCH %s AS OF VERSION %d", + tableName, "someBranch", table.currentSnapshot().snapshotId())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Branch does not exist: someBranch"); + } } diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java index 9c2233ccb791..5c41604afae1 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java @@ -22,7 +22,6 @@ import java.sql.Timestamp; import java.time.Instant; import java.util.List; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.spark.sql.SparkSession; @@ -38,6 +37,7 @@ import org.apache.spark.sql.catalyst.plans.logical.PositionalArgument; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.DataTypes; +import org.assertj.core.api.Assertions; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -144,11 +144,9 @@ public void testCallWithVarSubstitution() throws ParseException { @Test public void testCallParseError() { - AssertHelpers.assertThrows( - "Should fail with a sensible parse error", - IcebergParseException.class, - "missing '(' at 'radish'", - () -> parser.parsePlan("CALL cat.system radish kebab")); + Assertions.assertThatThrownBy(() -> parser.parsePlan("CALL cat.system radish kebab")) + .isInstanceOf(IcebergParseException.class) + .hasMessageContaining("missing '(' at 'radish'"); } @Test diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCherrypickSnapshotProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCherrypickSnapshotProcedure.java index 7309a176b922..c9af319f5c35 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCherrypickSnapshotProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCherrypickSnapshotProcedure.java @@ -22,7 +22,6 @@ import java.util.List; import java.util.Map; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; @@ -32,6 +31,7 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Test; @@ -161,43 +161,36 @@ public void testCherrypickSnapshotRefreshesRelationCache() { public void testCherrypickInvalidSnapshot() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); - AssertHelpers.assertThrows( - "Should reject invalid snapshot id", - ValidationException.class, - "Cannot cherry-pick unknown snapshot ID", - () -> sql("CALL %s.system.cherrypick_snapshot('%s', -1L)", catalogName, tableIdent)); + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.cherrypick_snapshot('%s', -1L)", catalogName, tableIdent)) + .isInstanceOf(ValidationException.class) + .hasMessage("Cannot cherry-pick unknown snapshot ID: -1"); } @Test public void testInvalidCherrypickSnapshotCases() { - AssertHelpers.assertThrows( - "Should not allow mixed args", - AnalysisException.class, - "Named and positional arguments cannot be mixed", - () -> sql("CALL %s.system.cherrypick_snapshot('n', table => 't', 1L)", catalogName)); - - AssertHelpers.assertThrows( - "Should not resolve procedures in arbitrary namespaces", - NoSuchProcedureException.class, - "not found", - () -> sql("CALL %s.custom.cherrypick_snapshot('n', 't', 1L)", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.cherrypick_snapshot('t')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with empty table identifier", - IllegalArgumentException.class, - "Cannot handle an empty identifier", - () -> sql("CALL %s.system.cherrypick_snapshot('', 1L)", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with invalid arg types", - AnalysisException.class, - "Wrong arg type for snapshot_id: cannot cast", - () -> sql("CALL %s.system.cherrypick_snapshot('t', 2.2)", catalogName)); + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.cherrypick_snapshot('n', table => 't', 1L)", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage("Named and positional arguments cannot be mixed"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.custom.cherrypick_snapshot('n', 't', 1L)", catalogName)) + .isInstanceOf(NoSuchProcedureException.class) + .hasMessage("Procedure custom.cherrypick_snapshot not found"); + + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.cherrypick_snapshot('t')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage("Missing required parameters: [snapshot_id]"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.cherrypick_snapshot('', 1L)", catalogName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot handle an empty identifier for argument table"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.cherrypick_snapshot('t', 2.2)", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith("Wrong arg type for snapshot_id: cannot cast"); } } diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestConflictValidation.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestConflictValidation.java index 10c86015e2e8..aaa275db9c16 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestConflictValidation.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestConflictValidation.java @@ -20,7 +20,6 @@ import java.util.List; import java.util.Map; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.IsolationLevel; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; @@ -29,8 +28,8 @@ import org.apache.iceberg.spark.source.SimpleRecord; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.functions; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -69,21 +68,17 @@ public void testOverwriteFilterSerializableIsolation() throws Exception { // Validating from previous snapshot finds conflicts Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); - AssertHelpers.assertThrows( - "Conflicting new data files should throw exception", - ValidationException.class, - "Found conflicting files that can contain records matching ref(name=\"id\") == 1:", - () -> { - try { - conflictingDf - .writeTo(tableName) - .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) - .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SERIALIZABLE.toString()) - .overwrite(functions.col("id").equalTo(1)); - } catch (NoSuchTableException e) { - throw new RuntimeException(e); - } - }); + Assertions.assertThatThrownBy( + () -> + conflictingDf + .writeTo(tableName) + .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) + .option( + SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SERIALIZABLE.toString()) + .overwrite(functions.col("id").equalTo(1))) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith( + "Found conflicting files that can contain records matching ref(name=\"id\") == 1:"); // Validating from latest snapshot should succeed table.refresh(); @@ -111,21 +106,16 @@ public void testOverwriteFilterSerializableIsolation2() throws Exception { // Validating from previous snapshot finds conflicts List conflictingRecords = Lists.newArrayList(new SimpleRecord(1, "a")); Dataset conflictingDf = spark.createDataFrame(conflictingRecords, SimpleRecord.class); - AssertHelpers.assertThrows( - "Conflicting new delete files should throw exception", - ValidationException.class, - "Found new conflicting delete files that can apply to records matching ref(name=\"id\") == 1:", - () -> { - try { - conflictingDf - .writeTo(tableName) - .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) - .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SNAPSHOT.toString()) - .overwrite(functions.col("id").equalTo(1)); - } catch (NoSuchTableException e) { - throw new RuntimeException(e); - } - }); + Assertions.assertThatThrownBy( + () -> + conflictingDf + .writeTo(tableName) + .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) + .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SNAPSHOT.toString()) + .overwrite(functions.col("id").equalTo(1))) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith( + "Found new conflicting delete files that can apply to records matching ref(name=\"id\") == 1:"); // Validating from latest snapshot should succeed table.refresh(); @@ -149,21 +139,17 @@ public void testOverwriteFilterSerializableIsolation3() throws Exception { // Validating from previous snapshot finds conflicts List conflictingRecords = Lists.newArrayList(new SimpleRecord(1, "a")); Dataset conflictingDf = spark.createDataFrame(conflictingRecords, SimpleRecord.class); - AssertHelpers.assertThrows( - "Conflicting deleted data files should throw exception", - ValidationException.class, - "Found conflicting deleted files that can contain records matching ref(name=\"id\") == 1:", - () -> { - try { - conflictingDf - .writeTo(tableName) - .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) - .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SERIALIZABLE.toString()) - .overwrite(functions.col("id").equalTo(1)); - } catch (NoSuchTableException e) { - throw new RuntimeException(e); - } - }); + Assertions.assertThatThrownBy( + () -> + conflictingDf + .writeTo(tableName) + .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) + .option( + SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SERIALIZABLE.toString()) + .overwrite(functions.col("id").equalTo(1))) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith( + "Found conflicting deleted files that can contain records matching ref(name=\"id\") == 1:"); // Validating from latest snapshot should succeed table.refresh(); @@ -184,20 +170,17 @@ public void testOverwriteFilterNoSnapshotIdValidation() throws Exception { // Validating from no snapshot id defaults to beginning snapshot id and finds conflicts Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); - AssertHelpers.assertThrows( - "Conflicting new data files should throw exception", - ValidationException.class, - "Found conflicting files that can contain records matching ref(name=\"id\") == 1:", - () -> { - try { - conflictingDf - .writeTo(tableName) - .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SERIALIZABLE.toString()) - .overwrite(functions.col("id").equalTo(1)); - } catch (NoSuchTableException e) { - throw new RuntimeException(e); - } - }); + + Assertions.assertThatThrownBy( + () -> + conflictingDf + .writeTo(tableName) + .option( + SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SERIALIZABLE.toString()) + .overwrite(functions.col("id").equalTo(1))) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith( + "Found conflicting files that can contain records matching ref(name=\"id\") == 1:"); // Validating from latest snapshot should succeed table.refresh(); @@ -225,21 +208,16 @@ public void testOverwriteFilterSnapshotIsolation() throws Exception { // Validating from previous snapshot finds conflicts List conflictingRecords = Lists.newArrayList(new SimpleRecord(1, "a")); Dataset conflictingDf = spark.createDataFrame(conflictingRecords, SimpleRecord.class); - AssertHelpers.assertThrows( - "Conflicting new delete files should throw exception", - ValidationException.class, - "Found new conflicting delete files that can apply to records matching ref(name=\"id\") == 1:", - () -> { - try { - conflictingDf - .writeTo(tableName) - .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) - .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SNAPSHOT.toString()) - .overwrite(functions.col("id").equalTo(1)); - } catch (NoSuchTableException e) { - throw new RuntimeException(e); - } - }); + Assertions.assertThatThrownBy( + () -> + conflictingDf + .writeTo(tableName) + .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) + .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SNAPSHOT.toString()) + .overwrite(functions.col("id").equalTo(1))) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith( + "Found new conflicting delete files that can apply to records matching ref(name=\"id\") == 1:"); // Validating from latest snapshot should succeed table.refresh(); @@ -278,21 +256,17 @@ public void testOverwritePartitionSerializableIsolation() throws Exception { // Validating from previous snapshot finds conflicts Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); - AssertHelpers.assertThrows( - "Conflicting deleted data files should throw exception", - ValidationException.class, - "Found conflicting files that can contain records matching partitions [id=1]", - () -> { - try { - conflictingDf - .writeTo(tableName) - .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) - .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SERIALIZABLE.toString()) - .overwritePartitions(); - } catch (NoSuchTableException e) { - throw new RuntimeException(e); - } - }); + Assertions.assertThatThrownBy( + () -> + conflictingDf + .writeTo(tableName) + .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) + .option( + SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SERIALIZABLE.toString()) + .overwritePartitions()) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith( + "Found conflicting files that can contain records matching partitions [id=1]"); // Validating from latest snapshot should succeed table.refresh(); @@ -318,21 +292,16 @@ public void testOverwritePartitionSnapshotIsolation() throws Exception { // Validating from previous snapshot finds conflicts Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); - AssertHelpers.assertThrows( - "Conflicting deleted data files should throw exception", - ValidationException.class, - "Found new conflicting delete files that can apply to records matching [id=1]", - () -> { - try { - conflictingDf - .writeTo(tableName) - .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) - .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SNAPSHOT.toString()) - .overwritePartitions(); - } catch (NoSuchTableException e) { - throw new RuntimeException(e); - } - }); + Assertions.assertThatThrownBy( + () -> + conflictingDf + .writeTo(tableName) + .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) + .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SNAPSHOT.toString()) + .overwritePartitions()) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith( + "Found new conflicting delete files that can apply to records matching [id=1]"); // Validating from latest snapshot should succeed table.refresh(); @@ -357,21 +326,16 @@ public void testOverwritePartitionSnapshotIsolation2() throws Exception { spark.createDataFrame(records, SimpleRecord.class).coalesce(1).writeTo(tableName).append(); Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); - AssertHelpers.assertThrows( - "Conflicting deleted data files should throw exception", - ValidationException.class, - "Found conflicting deleted files that can apply to records matching [id=1]", - () -> { - try { - conflictingDf - .writeTo(tableName) - .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) - .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SNAPSHOT.toString()) - .overwritePartitions(); - } catch (NoSuchTableException e) { - throw new RuntimeException(e); - } - }); + Assertions.assertThatThrownBy( + () -> + conflictingDf + .writeTo(tableName) + .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) + .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SNAPSHOT.toString()) + .overwritePartitions()) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith( + "Found conflicting deleted files that can apply to records matching [id=1]"); // Validating from latest snapshot should succeed table.refresh(); @@ -409,20 +373,16 @@ public void testOverwritePartitionNoSnapshotIdValidation() throws Exception { // Validating from null snapshot is equivalent to validating from beginning Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); - AssertHelpers.assertThrows( - "Conflicting deleted data files should throw exception", - ValidationException.class, - "Found conflicting files that can contain records matching partitions [id=1]", - () -> { - try { - conflictingDf - .writeTo(tableName) - .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SERIALIZABLE.toString()) - .overwritePartitions(); - } catch (NoSuchTableException e) { - throw new RuntimeException(e); - } - }); + Assertions.assertThatThrownBy( + () -> + conflictingDf + .writeTo(tableName) + .option( + SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SERIALIZABLE.toString()) + .overwritePartitions()) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith( + "Found conflicting files that can contain records matching partitions [id=1]"); // Validating from latest snapshot should succeed table.refresh(); diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java index dc12b0145d50..0a4a1073c3f9 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.extensions; +import static org.junit.Assert.assertThrows; + import java.util.List; import java.util.Map; import org.apache.iceberg.ChangelogOperation; @@ -45,13 +47,13 @@ public void removeTable() { sql("DROP TABLE IF EXISTS %s", tableName); } - public void createTableWith2Columns() { + public void createTableWithTwoColumns() { sql("CREATE TABLE %s (id INT, data STRING) USING iceberg", tableName); sql("ALTER TABLE %s SET TBLPROPERTIES ('format-version'='%d')", tableName, 1); sql("ALTER TABLE %s ADD PARTITION FIELD data", tableName); } - private void createTableWith3Columns() { + private void createTableWithThreeColumns() { sql("CREATE TABLE %s (id INT, data STRING, age INT) USING iceberg", tableName); sql("ALTER TABLE %s SET TBLPROPERTIES ('format-version'='%d')", tableName, 1); sql("ALTER TABLE %s ADD PARTITION FIELD id", tableName); @@ -65,7 +67,7 @@ private void createTableWithIdentifierField() { @Test public void testCustomizedViewName() { - createTableWith2Columns(); + createTableWithTwoColumns(); sql("INSERT INTO %s VALUES (1, 'a')", tableName); sql("INSERT INTO %s VALUES (2, 'b')", tableName); @@ -98,7 +100,7 @@ public void testCustomizedViewName() { @Test public void testNoSnapshotIdInput() { - createTableWith2Columns(); + createTableWithTwoColumns(); sql("INSERT INTO %s VALUES (1, 'a')", tableName); Table table = validationCatalog.loadTable(tableIdent); Snapshot snap0 = table.currentSnapshot(); @@ -129,7 +131,7 @@ public void testNoSnapshotIdInput() { @Test public void testTimestampsBasedQuery() { - createTableWith2Columns(); + createTableWithTwoColumns(); long beginning = System.currentTimeMillis(); sql("INSERT INTO %s VALUES (1, 'a')", tableName); @@ -189,7 +191,7 @@ public void testTimestampsBasedQuery() { @Test public void testWithCarryovers() { - createTableWith2Columns(); + createTableWithTwoColumns(); sql("INSERT INTO %s VALUES (1, 'a')", tableName); Table table = validationCatalog.loadTable(tableIdent); Snapshot snap0 = table.currentSnapshot(); @@ -224,7 +226,7 @@ public void testWithCarryovers() { @Test public void testUpdate() { - createTableWith2Columns(); + createTableWithTwoColumns(); sql("ALTER TABLE %s DROP PARTITION FIELD data", tableName); sql("ALTER TABLE %s ADD PARTITION FIELD id", tableName); @@ -283,7 +285,7 @@ public void testUpdateWithIdentifierField() { @Test public void testUpdateWithFilter() { - createTableWith2Columns(); + createTableWithTwoColumns(); sql("ALTER TABLE %s DROP PARTITION FIELD data", tableName); sql("ALTER TABLE %s ADD PARTITION FIELD id", tableName); @@ -315,7 +317,7 @@ public void testUpdateWithFilter() { @Test public void testUpdateWithMultipleIdentifierColumns() { - createTableWith3Columns(); + createTableWithThreeColumns(); sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11)", tableName); Table table = validationCatalog.loadTable(tableIdent); @@ -347,7 +349,7 @@ public void testUpdateWithMultipleIdentifierColumns() { @Test public void testRemoveCarryOvers() { - createTableWith3Columns(); + createTableWithThreeColumns(); sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11), (2, 'e', 12)", tableName); Table table = validationCatalog.loadTable(tableIdent); @@ -381,7 +383,7 @@ public void testRemoveCarryOvers() { @Test public void testRemoveCarryOversWithoutUpdatedRows() { - createTableWith3Columns(); + createTableWithThreeColumns(); sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11), (2, 'e', 12)", tableName); Table table = validationCatalog.loadTable(tableIdent); @@ -411,9 +413,74 @@ public void testRemoveCarryOversWithoutUpdatedRows() { sql("select * from %s order by _change_ordinal, id, data", viewName)); } + @Test + public void testNetChangesWithRemoveCarryOvers() { + // partitioned by id + createTableWithThreeColumns(); + + // insert rows: (1, 'a', 12) (2, 'b', 11) (2, 'e', 12) + sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11), (2, 'e', 12)", tableName); + Table table = validationCatalog.loadTable(tableIdent); + Snapshot snap1 = table.currentSnapshot(); + + // delete rows: (2, 'b', 11) (2, 'e', 12) + // insert rows: (3, 'c', 13) (2, 'd', 11) (2, 'e', 12) + sql("INSERT OVERWRITE %s VALUES (3, 'c', 13), (2, 'd', 11), (2, 'e', 12)", tableName); + table.refresh(); + Snapshot snap2 = table.currentSnapshot(); + + // delete rows: (2, 'd', 11) (2, 'e', 12) (3, 'c', 13) + // insert rows: (3, 'c', 15) (2, 'e', 12) + sql("INSERT OVERWRITE %s VALUES (3, 'c', 15), (2, 'e', 12)", tableName); + table.refresh(); + Snapshot snap3 = table.currentSnapshot(); + + // test with all snapshots + List returns = + sql( + "CALL %s.system.create_changelog_view(table => '%s', net_changes => true)", + catalogName, tableName); + + String viewName = (String) returns.get(0)[0]; + + assertEquals( + "Rows should match", + ImmutableList.of( + row(1, "a", 12, INSERT, 0, snap1.snapshotId()), + row(3, "c", 15, INSERT, 2, snap3.snapshotId()), + row(2, "e", 12, INSERT, 2, snap3.snapshotId())), + sql("select * from %s order by _change_ordinal, data", viewName)); + + // test with snap2 and snap3 + sql( + "CALL %s.system.create_changelog_view(table => '%s', " + + "options => map('start-snapshot-id','%s'), " + + "net_changes => true)", + catalogName, tableName, snap1.snapshotId()); + + assertEquals( + "Rows should match", + ImmutableList.of( + row(2, "b", 11, DELETE, 0, snap2.snapshotId()), + row(3, "c", 15, INSERT, 1, snap3.snapshotId())), + sql("select * from %s order by _change_ordinal, data", viewName)); + } + + @Test + public void testNetChangesWithComputeUpdates() { + createTableWithTwoColumns(); + assertThrows( + "Should fail because net_changes is not supported with computing updates", + IllegalArgumentException.class, + () -> + sql( + "CALL %s.system.create_changelog_view(table => '%s', identifier_columns => array('id'), net_changes => true)", + catalogName, tableName)); + } + @Test public void testNotRemoveCarryOvers() { - createTableWith3Columns(); + createTableWithThreeColumns(); sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11), (2, 'e', 12)", tableName); Table table = validationCatalog.loadTable(tableIdent); diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java index 2010f5affd12..744c47f3d579 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java @@ -27,12 +27,12 @@ import static org.apache.iceberg.TableProperties.SPLIT_OPEN_FILE_COST; import static org.apache.iceberg.TableProperties.SPLIT_SIZE; import static org.apache.spark.sql.functions.lit; +import static org.assertj.core.api.Assumptions.assumeThat; import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -43,12 +43,12 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import org.apache.iceberg.AppendFiles; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.DataFile; import org.apache.iceberg.DistributionMode; import org.apache.iceberg.ManifestFile; import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -288,11 +288,11 @@ public void testDeleteFileThenMetadataDelete() throws Exception { // Metadata Delete Table table = Spark3Util.loadIcebergTable(spark, tableName); - Set dataFilesBefore = TestHelpers.dataFiles(table, branch); + List dataFilesBefore = TestHelpers.dataFiles(table, branch); sql("DELETE FROM %s AS t WHERE t.id = 1", commitTarget()); - Set dataFilesAfter = TestHelpers.dataFiles(table, branch); + List dataFilesAfter = TestHelpers.dataFiles(table, branch); Assert.assertTrue( "Data file should have been removed", dataFilesBefore.size() > dataFilesAfter.size()); @@ -522,11 +522,10 @@ public void testDeleteWithNonDeterministicCondition() { sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware')", tableName); createBranchIfNeeded(); - AssertHelpers.assertThrows( - "Should complain about non-deterministic expressions", - AnalysisException.class, - "nondeterministic expressions are only allowed", - () -> sql("DELETE FROM %s WHERE id = 1 AND rand() > 0.5", commitTarget())); + Assertions.assertThatThrownBy( + () -> sql("DELETE FROM %s WHERE id = 1 AND rand() > 0.5", commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith("nondeterministic expressions are only allowed"); } @Test @@ -813,11 +812,9 @@ public void testDeleteOnNonIcebergTableNotSupported() { sql("CREATE TABLE parquet_table (c1 INT, c2 INT) USING parquet"); - AssertHelpers.assertThrows( - "Delete is supported only for Iceberg tables", - AnalysisException.class, - "does not support DELETE", - () -> sql("DELETE FROM parquet_table WHERE c1 = -100")); + Assertions.assertThatThrownBy(() -> sql("DELETE FROM parquet_table WHERE c1 = -100")) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("does not support DELETE"); } @Test @@ -1271,6 +1268,42 @@ public void testDeleteToWapBranchWithTableBranchIdentifier() throws NoSuchTableE branch))); } + @Test + public void testDeleteToCustomWapBranchWithoutWhereClause() throws NoSuchTableException { + assumeThat(branch) + .as("Run only if custom WAP branch is not main") + .isNotNull() + .isNotEqualTo(SnapshotRef.MAIN_BRANCH); + + createAndInitPartitionedTable(); + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s' = 'true')", + tableName, TableProperties.WRITE_AUDIT_PUBLISH_ENABLED); + append(tableName, new Employee(0, "hr"), new Employee(1, "hr"), new Employee(2, "hr")); + createBranchIfNeeded(); + + withSQLConf( + ImmutableMap.of(SparkSQLProperties.WAP_BRANCH, branch), + () -> { + sql("DELETE FROM %s t WHERE id=1", tableName); + Assertions.assertThat(spark.table(tableName).count()).isEqualTo(2L); + Assertions.assertThat(spark.table(tableName + ".branch_" + branch).count()).isEqualTo(2L); + Assertions.assertThat(spark.table(tableName + ".branch_main").count()) + .as("Should not modify main branch") + .isEqualTo(3L); + }); + withSQLConf( + ImmutableMap.of(SparkSQLProperties.WAP_BRANCH, branch), + () -> { + sql("DELETE FROM %s t", tableName); + Assertions.assertThat(spark.table(tableName).count()).isEqualTo(0L); + Assertions.assertThat(spark.table(tableName + ".branch_" + branch).count()).isEqualTo(0L); + Assertions.assertThat(spark.table(tableName + ".branch_main").count()) + .as("Should not modify main branch") + .isEqualTo(3L); + }); + } + // TODO: multiple stripes for ORC protected void createAndInitPartitionedTable() { diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java index efb3d43668f1..c4b93c6d6a44 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java @@ -33,7 +33,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.GenericBlobMetadata; import org.apache.iceberg.GenericStatisticsFile; import org.apache.iceberg.Snapshot; @@ -164,44 +163,36 @@ public void testExpireSnapshotsGCDisabled() { sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'false')", tableName, GC_ENABLED); - AssertHelpers.assertThrows( - "Should reject call", - ValidationException.class, - "Cannot expire snapshots: GC is disabled", - () -> sql("CALL %s.system.expire_snapshots('%s')", catalogName, tableIdent)); + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.expire_snapshots('%s')", catalogName, tableIdent)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Cannot expire snapshots: GC is disabled"); } @Test public void testInvalidExpireSnapshotsCases() { - AssertHelpers.assertThrows( - "Should not allow mixed args", - AnalysisException.class, - "Named and positional arguments cannot be mixed", - () -> sql("CALL %s.system.expire_snapshots('n', table => 't')", catalogName)); - - AssertHelpers.assertThrows( - "Should not resolve procedures in arbitrary namespaces", - NoSuchProcedureException.class, - "not found", - () -> sql("CALL %s.custom.expire_snapshots('n', 't')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.expire_snapshots()", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with invalid arg types", - AnalysisException.class, - "Wrong arg type", - () -> sql("CALL %s.system.expire_snapshots('n', 2.2)", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with empty table identifier", - IllegalArgumentException.class, - "Cannot handle an empty identifier", - () -> sql("CALL %s.system.expire_snapshots('')", catalogName)); + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.expire_snapshots('n', table => 't')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage("Named and positional arguments cannot be mixed"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.custom.expire_snapshots('n', 't')", catalogName)) + .isInstanceOf(NoSuchProcedureException.class) + .hasMessage("Procedure custom.expire_snapshots not found"); + + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.expire_snapshots()", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage("Missing required parameters: [table]"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.expire_snapshots('n', 2.2)", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage("Wrong arg type for older_than: cannot cast DecimalType(2,1) to TimestampType"); + + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.expire_snapshots('')", catalogName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot handle an empty identifier for argument table"); } @Test @@ -219,14 +210,13 @@ public void testResolvingTableInAnotherCatalog() throws IOException { "CREATE TABLE %s.%s (id bigint NOT NULL, data string) USING iceberg", anotherCatalog, tableIdent); - AssertHelpers.assertThrows( - "Should reject calls for a table in another catalog", - IllegalArgumentException.class, - "Cannot run procedure in catalog", - () -> - sql( - "CALL %s.system.expire_snapshots('%s')", - catalogName, anotherCatalog + "." + tableName)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.expire_snapshots('%s')", + catalogName, anotherCatalog + "." + tableName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot run procedure in catalog"); } @Test @@ -256,23 +246,21 @@ public void testConcurrentExpireSnapshots() { public void testConcurrentExpireSnapshotsWithInvalidInput() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); - AssertHelpers.assertThrows( - "Should throw an error when max_concurrent_deletes = 0", - IllegalArgumentException.class, - "max_concurrent_deletes should have value > 0", - () -> - sql( - "CALL %s.system.expire_snapshots(table => '%s', max_concurrent_deletes => %s)", - catalogName, tableIdent, 0)); - - AssertHelpers.assertThrows( - "Should throw an error when max_concurrent_deletes < 0 ", - IllegalArgumentException.class, - "max_concurrent_deletes should have value > 0", - () -> - sql( - "CALL %s.system.expire_snapshots(table => '%s', max_concurrent_deletes => %s)", - catalogName, tableIdent, -1)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.expire_snapshots(table => '%s', max_concurrent_deletes => %s)", + catalogName, tableIdent, 0)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("max_concurrent_deletes should have value > 0, value: 0"); + + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.expire_snapshots(table => '%s', max_concurrent_deletes => %s)", + catalogName, tableIdent, -1)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("max_concurrent_deletes should have value > 0, value: -1"); } @Test @@ -405,19 +393,18 @@ public void testExpireSnapshotShouldFailForCurrentSnapshot() { Table table = validationCatalog.loadTable(tableIdent); Assert.assertEquals("Should be 2 snapshots", 2, Iterables.size(table.snapshots())); - AssertHelpers.assertThrows( - "Should reject call", - IllegalArgumentException.class, - "Cannot expire", - () -> - sql( - "CALL %s.system.expire_snapshots(" - + "table => '%s'," - + "snapshot_ids => ARRAY(%d, %d))", - catalogName, - tableIdent, - table.currentSnapshot().snapshotId(), - table.currentSnapshot().parentId())); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.expire_snapshots(" + + "table => '%s'," + + "snapshot_ids => ARRAY(%d, %d))", + catalogName, + tableIdent, + table.currentSnapshot().snapshotId(), + table.currentSnapshot().parentId())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot expire"); } @Test diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java index 819a6f8c7572..e32b0f22ac23 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java @@ -25,7 +25,6 @@ import java.util.List; import java.util.Locale; import java.util.Map; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.RowDelta; import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Table; @@ -37,6 +36,7 @@ import org.apache.iceberg.spark.source.SparkTable; import org.apache.iceberg.spark.source.TestSparkCatalog; import org.apache.spark.sql.connector.catalog.Identifier; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Test; import org.junit.runners.Parameterized; @@ -123,11 +123,10 @@ public void testCommitUnknownException() { TestSparkCatalog.setTable(ident, sparkTable); // Although an exception is thrown here, write and commit have succeeded - AssertHelpers.assertThrows( - "Should throw a Commit State Unknown Exception", - CommitStateUnknownException.class, - "Datacenter on Fire", - () -> sql("DELETE FROM %s WHERE id = 2", "dummy_catalog.default.table")); + Assertions.assertThatThrownBy( + () -> sql("DELETE FROM %s WHERE id = 2", "dummy_catalog.default.table")) + .isInstanceOf(CommitStateUnknownException.class) + .hasMessageStartingWith("Datacenter on Fire"); // Since write and commit succeeded, the rows should be readable assertEquals( diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java index 8b2950b74f8d..2fc1044f7d4a 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java @@ -20,11 +20,11 @@ import java.io.IOException; import java.util.Map; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.spark.sql.AnalysisException; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -103,7 +103,7 @@ public void testMigrateWithOptions() throws IOException { ImmutableList.of(row(1L, "a"), row(1L, "a")), sql("SELECT * FROM %s ORDER BY id", tableName)); - sql("DROP TABLE IF EXISTS %s", tableName + "_BACKUP_"); + sql("DROP TABLE IF EXISTS %s", tableName + "_BACKUP_"); } @Test @@ -131,14 +131,13 @@ public void testMigrateWithInvalidMetricsConfig() throws IOException { "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", tableName, location); - AssertHelpers.assertThrows( - "Should reject invalid metrics config", - ValidationException.class, - "Invalid metrics config", - () -> { - String props = "map('write.metadata.metrics.column.x', 'X')"; - sql("CALL %s.system.migrate('%s', %s)", catalogName, tableName, props); - }); + Assertions.assertThatThrownBy( + () -> { + String props = "map('write.metadata.metrics.column.x', 'X')"; + sql("CALL %s.system.migrate('%s', %s)", catalogName, tableName, props); + }) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Invalid metrics config"); } @Test @@ -166,22 +165,56 @@ public void testMigrateWithConflictingProps() throws IOException { @Test public void testInvalidMigrateCases() { - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.migrate()", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with invalid arg types", - AnalysisException.class, - "Wrong arg type", - () -> sql("CALL %s.system.migrate(map('foo','bar'))", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with empty table identifier", - IllegalArgumentException.class, - "Cannot handle an empty identifier", - () -> sql("CALL %s.system.migrate('')", catalogName)); + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.migrate()", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage("Missing required parameters: [table]"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.migrate(map('foo','bar'))", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith("Wrong arg type for table"); + + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.migrate('')", catalogName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot handle an empty identifier for argument table"); + } + + @Test + public void testMigratePartitionWithSpecialCharacter() throws IOException { + Assume.assumeTrue(catalogName.equals("spark_catalog")); + String location = temp.newFolder().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string, dt date) USING parquet " + + "PARTITIONED BY (data, dt) LOCATION '%s'", + tableName, location); + sql("INSERT INTO TABLE %s VALUES (1, '2023/05/30', date '2023-05-30')", tableName); + Object result = scalarSql("CALL %s.system.migrate('%s')", catalogName, tableName); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1L, "2023/05/30", java.sql.Date.valueOf("2023-05-30"))), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @Test + public void testMigrateEmptyPartitionedTable() throws Exception { + Assume.assumeTrue(catalogName.equals("spark_catalog")); + String location = temp.newFolder().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet PARTITIONED BY (id) LOCATION '%s'", + tableName, location); + Object result = scalarSql("CALL %s.system.migrate('%s')", catalogName, tableName); + Assert.assertEquals(0L, result); + } + + @Test + public void testMigrateEmptyTable() throws Exception { + Assume.assumeTrue(catalogName.equals("spark_catalog")); + String location = temp.newFolder().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + tableName, location); + Object result = scalarSql("CALL %s.system.migrate('%s')", catalogName, tableName); + Assert.assertEquals(0L, result); } } diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestPublishChangesProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestPublishChangesProcedure.java index 2b74cd475fae..94c4af3ad56d 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestPublishChangesProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestPublishChangesProcedure.java @@ -22,7 +22,6 @@ import java.util.List; import java.util.Map; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; @@ -32,6 +31,7 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Test; @@ -157,37 +157,32 @@ public void testApplyWapChangesRefreshesRelationCache() { public void testApplyInvalidWapId() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); - AssertHelpers.assertThrows( - "Should reject invalid wap id", - ValidationException.class, - "Cannot apply unknown WAP ID", - () -> sql("CALL %s.system.publish_changes('%s', 'not_valid')", catalogName, tableIdent)); + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.publish_changes('%s', 'not_valid')", catalogName, tableIdent)) + .isInstanceOf(ValidationException.class) + .hasMessage("Cannot apply unknown WAP ID 'not_valid'"); } @Test public void testInvalidApplyWapChangesCases() { - AssertHelpers.assertThrows( - "Should not allow mixed args", - AnalysisException.class, - "Named and positional arguments cannot be mixed", - () -> sql("CALL %s.system.publish_changes('n', table => 't', 'not_valid')", catalogName)); - - AssertHelpers.assertThrows( - "Should not resolve procedures in arbitrary namespaces", - NoSuchProcedureException.class, - "not found", - () -> sql("CALL %s.custom.publish_changes('n', 't', 'not_valid')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.publish_changes('t')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with empty table identifier", - IllegalArgumentException.class, - "Cannot handle an empty identifier", - () -> sql("CALL %s.system.publish_changes('', 'not_valid')", catalogName)); + Assertions.assertThatThrownBy( + () -> + sql("CALL %s.system.publish_changes('n', table => 't', 'not_valid')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage("Named and positional arguments cannot be mixed"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.custom.publish_changes('n', 't', 'not_valid')", catalogName)) + .isInstanceOf(NoSuchProcedureException.class) + .hasMessage("Procedure custom.publish_changes not found"); + + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.publish_changes('t')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage("Missing required parameters: [wap_id]"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.publish_changes('', 'not_valid')", catalogName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot handle an empty identifier for argument table"); } } diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java index 4cf6b10cb293..b29981775076 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java @@ -34,7 +34,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; import org.apache.iceberg.Files; @@ -228,11 +227,11 @@ public void testRemoveOrphanFilesGCDisabled() { sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'false')", tableName, GC_ENABLED); - AssertHelpers.assertThrows( - "Should reject call", - ValidationException.class, - "Cannot delete orphan files: GC is disabled", - () -> sql("CALL %s.system.remove_orphan_files('%s')", catalogName, tableIdent)); + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.remove_orphan_files('%s')", catalogName, tableIdent)) + .isInstanceOf(ValidationException.class) + .hasMessage( + "Cannot delete orphan files: GC is disabled (deleting files may corrupt other tables)"); // reset the property to enable the table purging in removeTable. sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'true')", tableName, GC_ENABLED); @@ -259,35 +258,28 @@ public void testRemoveOrphanFilesWap() { @Test public void testInvalidRemoveOrphanFilesCases() { - AssertHelpers.assertThrows( - "Should not allow mixed args", - AnalysisException.class, - "Named and positional arguments cannot be mixed", - () -> sql("CALL %s.system.remove_orphan_files('n', table => 't')", catalogName)); - - AssertHelpers.assertThrows( - "Should not resolve procedures in arbitrary namespaces", - NoSuchProcedureException.class, - "not found", - () -> sql("CALL %s.custom.remove_orphan_files('n', 't')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.remove_orphan_files()", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with invalid arg types", - AnalysisException.class, - "Wrong arg type", - () -> sql("CALL %s.system.remove_orphan_files('n', 2.2)", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with empty table identifier", - IllegalArgumentException.class, - "Cannot handle an empty identifier", - () -> sql("CALL %s.system.remove_orphan_files('')", catalogName)); + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.remove_orphan_files('n', table => 't')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage("Named and positional arguments cannot be mixed"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.custom.remove_orphan_files('n', 't')", catalogName)) + .isInstanceOf(NoSuchProcedureException.class) + .hasMessage("Procedure custom.remove_orphan_files not found"); + + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.remove_orphan_files()", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage("Missing required parameters: [table]"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.remove_orphan_files('n', 2.2)", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith("Wrong arg type for older_than"); + + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.remove_orphan_files('')", catalogName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot handle an empty identifier for argument table"); } @Test @@ -352,63 +344,58 @@ public void testConcurrentRemoveOrphanFiles() throws IOException { public void testConcurrentRemoveOrphanFilesWithInvalidInput() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); - AssertHelpers.assertThrows( - "Should throw an error when max_concurrent_deletes = 0", - IllegalArgumentException.class, - "max_concurrent_deletes should have value > 0", - () -> - sql( - "CALL %s.system.remove_orphan_files(table => '%s', max_concurrent_deletes => %s)", - catalogName, tableIdent, 0)); - - AssertHelpers.assertThrows( - "Should throw an error when max_concurrent_deletes < 0 ", - IllegalArgumentException.class, - "max_concurrent_deletes should have value > 0", - () -> - sql( - "CALL %s.system.remove_orphan_files(table => '%s', max_concurrent_deletes => %s)", - catalogName, tableIdent, -1)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.remove_orphan_files(table => '%s', max_concurrent_deletes => %s)", + catalogName, tableIdent, 0)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("max_concurrent_deletes should have value > 0, value: 0"); + + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.remove_orphan_files(table => '%s', max_concurrent_deletes => %s)", + catalogName, tableIdent, -1)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("max_concurrent_deletes should have value > 0, value: -1"); String tempViewName = "file_list_test"; spark.emptyDataFrame().createOrReplaceTempView(tempViewName); - AssertHelpers.assertThrows( - "Should throw an error if file_list_view is missing required columns", - IllegalArgumentException.class, - "does not exist. Available:", - () -> - sql( - "CALL %s.system.remove_orphan_files(table => '%s', file_list_view => '%s')", - catalogName, tableIdent, tempViewName)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.remove_orphan_files(table => '%s', file_list_view => '%s')", + catalogName, tableIdent, tempViewName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("file_path does not exist. Available: "); spark .createDataset(Lists.newArrayList(), Encoders.tuple(Encoders.INT(), Encoders.TIMESTAMP())) .toDF("file_path", "last_modified") .createOrReplaceTempView(tempViewName); - AssertHelpers.assertThrows( - "Should throw an error if file_path has wrong type", - IllegalArgumentException.class, - "Invalid file_path column", - () -> - sql( - "CALL %s.system.remove_orphan_files(table => '%s', file_list_view => '%s')", - catalogName, tableIdent, tempViewName)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.remove_orphan_files(table => '%s', file_list_view => '%s')", + catalogName, tableIdent, tempViewName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid file_path column: IntegerType is not a string"); spark .createDataset(Lists.newArrayList(), Encoders.tuple(Encoders.STRING(), Encoders.STRING())) .toDF("file_path", "last_modified") .createOrReplaceTempView(tempViewName); - AssertHelpers.assertThrows( - "Should throw an error if last_modified has wrong type", - IllegalArgumentException.class, - "Invalid last_modified column", - () -> - sql( - "CALL %s.system.remove_orphan_files(table => '%s', file_list_view => '%s')", - catalogName, tableIdent, tempViewName)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.remove_orphan_files(table => '%s', file_list_view => '%s')", + catalogName, tableIdent, tempViewName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid last_modified column: StringType is not a timestamp"); } @Test @@ -611,16 +598,15 @@ public void testRemoveOrphanFilesProcedureWithPrefixMode() Assert.assertEquals(0, orphanFiles.size()); // Test with no equal schemes - AssertHelpers.assertThrows( - "Should complain about removing orphan files", - ValidationException.class, - "Conflicting authorities/schemes: [(file1, file)]", - () -> - sql( - "CALL %s.system.remove_orphan_files(" - + "table => '%s'," - + "file_list_view => '%s')", - catalogName, tableIdent, fileListViewName)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.remove_orphan_files(" + + "table => '%s'," + + "file_list_view => '%s')", + catalogName, tableIdent, fileListViewName)) + .isInstanceOf(ValidationException.class) + .hasMessageEndingWith("Conflicting authorities/schemes: [(file1, file)]."); // Drop table in afterEach has purge and fails due to invalid scheme "file1" used in this test // Dropping the table here diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestReplaceBranch.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestReplaceBranch.java index b63826e543b8..eb167ed25be4 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestReplaceBranch.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestReplaceBranch.java @@ -21,7 +21,6 @@ import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -30,6 +29,7 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -74,11 +74,13 @@ public void testReplaceBranchFailsForTag() throws NoSuchTableException { df.writeTo(tableName).append(); long second = table.currentSnapshot().snapshotId(); - AssertHelpers.assertThrows( - "Cannot perform replace branch on tags", - IllegalArgumentException.class, - "Ref tag1 is a tag not a branch", - () -> sql("ALTER TABLE %s REPLACE BRANCH %s AS OF VERSION %d", tableName, tagName, second)); + Assertions.assertThatThrownBy( + () -> + sql( + "ALTER TABLE %s REPLACE BRANCH %s AS OF VERSION %d", + tableName, tagName, second)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Ref tag1 is a tag not a branch"); } @Test @@ -126,14 +128,13 @@ public void testReplaceBranchDoesNotExist() throws NoSuchTableException { df.writeTo(tableName).append(); Table table = validationCatalog.loadTable(tableIdent); - AssertHelpers.assertThrows( - "Cannot perform replace branch on branch which does not exist", - IllegalArgumentException.class, - "Branch does not exist", - () -> - sql( - "ALTER TABLE %s REPLACE BRANCH %s AS OF VERSION %d", - tableName, "someBranch", table.currentSnapshot().snapshotId())); + Assertions.assertThatThrownBy( + () -> + sql( + "ALTER TABLE %s REPLACE BRANCH %s AS OF VERSION %d", + tableName, "someBranch", table.currentSnapshot().snapshotId())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Branch does not exist: someBranch"); } @Test diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRequiredDistributionAndOrdering.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRequiredDistributionAndOrdering.java index adb4fab41922..fcdf9bf992bb 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRequiredDistributionAndOrdering.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRequiredDistributionAndOrdering.java @@ -236,6 +236,23 @@ public void testDefaultSortOnStringBucketedColumn() { assertEquals("Rows must match", expected, sql("SELECT * FROM %s ORDER BY c1", tableName)); } + @Test + public void testDefaultSortOnBinaryBucketedColumn() { + sql( + "CREATE TABLE %s (c1 INT, c2 Binary) " + + "USING iceberg " + + "PARTITIONED BY (bucket(2, c2))", + tableName); + + sql("INSERT INTO %s VALUES (1, X'A1B1'), (2, X'A2B2')", tableName); + + byte[] bytes1 = new byte[] {-95, -79}; + byte[] bytes2 = new byte[] {-94, -78}; + List expected = ImmutableList.of(row(1, bytes1), row(2, bytes2)); + + assertEquals("Rows must match", expected, sql("SELECT * FROM %s ORDER BY c1", tableName)); + } + @Test public void testDefaultSortOnDecimalTruncatedColumn() { sql( diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java index 3ed47d54d374..801396718137 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java @@ -24,7 +24,6 @@ import java.util.List; import java.util.Map; import java.util.stream.IntStream; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.TableProperties; import org.apache.iceberg.catalog.TableIdentifier; @@ -42,6 +41,7 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; import org.apache.spark.sql.internal.SQLConf; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -184,6 +184,41 @@ public void testRewriteDataFilesWithSortStrategy() { assertEquals("Data after compaction should not change", expectedRecords, actualRecords); } + @Test + public void testRewriteDataFilesWithSortStrategyAndMultipleShufflePartitionsPerFile() { + createTable(); + insertData(10 /* file count */); + + List output = + sql( + "CALL %s.system.rewrite_data_files(" + + " table => '%s', " + + " strategy => 'sort', " + + " sort_order => 'c1', " + + " options => map('shuffle-partitions-per-file', '2'))", + catalogName, tableIdent); + + assertEquals( + "Action should rewrite 10 data files and add 1 data files", + row(10, 1), + Arrays.copyOf(output.get(0), 2)); + + // as there is only one small output file, validate the query ordering (it will not change) + ImmutableList expectedRows = + ImmutableList.of( + row(1, "foo", null), + row(1, "foo", null), + row(1, "foo", null), + row(1, "foo", null), + row(1, "foo", null), + row(2, "bar", null), + row(2, "bar", null), + row(2, "bar", null), + row(2, "bar", null), + row(2, "bar", null)); + assertEquals("Should have expected rows", expectedRows, sql("SELECT * FROM %s", tableName)); + } + @Test public void testRewriteDataFilesWithZOrder() { createTable(); @@ -225,6 +260,42 @@ public void testRewriteDataFilesWithZOrder() { assertEquals("Should have expected rows", expectedRows, sql("SELECT * FROM %s", tableName)); } + @Test + public void testRewriteDataFilesWithZOrderAndMultipleShufflePartitionsPerFile() { + createTable(); + insertData(10 /* file count */); + + List output = + sql( + "CALL %s.system.rewrite_data_files(" + + " table => '%s', " + + "strategy => 'sort', " + + " sort_order => 'zorder(c1, c2)', " + + " options => map('shuffle-partitions-per-file', '2'))", + catalogName, tableIdent); + + assertEquals( + "Action should rewrite 10 data files and add 1 data files", + row(10, 1), + Arrays.copyOf(output.get(0), 2)); + + // due to z-ordering, the data will be written in the below order + // as there is only one small output file, validate the query ordering (it will not change) + ImmutableList expectedRows = + ImmutableList.of( + row(2, "bar", null), + row(2, "bar", null), + row(2, "bar", null), + row(2, "bar", null), + row(2, "bar", null), + row(1, "foo", null), + row(1, "foo", null), + row(1, "foo", null), + row(1, "foo", null), + row(1, "foo", null)); + assertEquals("Should have expected rows", expectedRows, sql("SELECT * FROM %s", tableName)); + } + @Test public void testRewriteDataFilesWithFilter() { createTable(); @@ -427,136 +498,121 @@ public void testRewriteDataFilesWithInvalidInputs() { insertData(2); // Test for invalid strategy - AssertHelpers.assertThrows( - "Should reject calls with unsupported strategy error message", - IllegalArgumentException.class, - "unsupported strategy: temp. Only binpack or sort is supported", - () -> - sql( - "CALL %s.system.rewrite_data_files(table => '%s', options => map('min-input-files','2'), " - + "strategy => 'temp')", - catalogName, tableIdent)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_data_files(table => '%s', options => map('min-input-files','2'), " + + "strategy => 'temp')", + catalogName, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("unsupported strategy: temp. Only binpack or sort is supported"); // Test for sort_order with binpack strategy - AssertHelpers.assertThrows( - "Should reject calls with error message", - IllegalArgumentException.class, - "Must use only one rewriter type (bin-pack, sort, zorder)", - () -> - sql( - "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'binpack', " - + "sort_order => 'c1 ASC NULLS FIRST')", - catalogName, tableIdent)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'binpack', " + + "sort_order => 'c1 ASC NULLS FIRST')", + catalogName, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Must use only one rewriter type (bin-pack, sort, zorder)"); // Test for sort strategy without any (default/user defined) sort_order - AssertHelpers.assertThrows( - "Should reject calls with error message", - IllegalArgumentException.class, - "Cannot sort data without a valid sort order", - () -> - sql( - "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort')", - catalogName, tableIdent)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort')", + catalogName, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot sort data without a valid sort order"); // Test for sort_order with invalid null order - AssertHelpers.assertThrows( - "Should reject calls with error message", - IllegalArgumentException.class, - "Unable to parse sortOrder:", - () -> - sql( - "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " - + "sort_order => 'c1 ASC none')", - catalogName, tableIdent)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " + + "sort_order => 'c1 ASC none')", + catalogName, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Unable to parse sortOrder: c1 ASC none"); // Test for sort_order with invalid sort direction - AssertHelpers.assertThrows( - "Should reject calls with error message", - IllegalArgumentException.class, - "Unable to parse sortOrder:", - () -> - sql( - "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " - + "sort_order => 'c1 none NULLS FIRST')", - catalogName, tableIdent)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " + + "sort_order => 'c1 none NULLS FIRST')", + catalogName, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Unable to parse sortOrder: c1 none NULLS FIRST"); // Test for sort_order with invalid column name - AssertHelpers.assertThrows( - "Should reject calls with error message", - ValidationException.class, - "Cannot find field 'col1' in struct:" - + " struct<1: c1: optional int, 2: c2: optional string, 3: c3: optional string>", - () -> - sql( - "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " - + "sort_order => 'col1 DESC NULLS FIRST')", - catalogName, tableIdent)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " + + "sort_order => 'col1 DESC NULLS FIRST')", + catalogName, tableIdent)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Cannot find field 'col1' in struct:"); // Test with invalid filter column col1 - AssertHelpers.assertThrows( - "Should reject calls with error message", - IllegalArgumentException.class, - "Cannot parse predicates in where option: col1 = 3", - () -> - sql( - "CALL %s.system.rewrite_data_files(table => '%s', " + "where => 'col1 = 3')", - catalogName, tableIdent)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_data_files(table => '%s', " + "where => 'col1 = 3')", + catalogName, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse predicates in where option: col1 = 3"); // Test for z_order with invalid column name - AssertHelpers.assertThrows( - "Should reject calls with error message", - IllegalArgumentException.class, - "Cannot find column 'col1' in table schema (case sensitive = false): " - + "struct<1: c1: optional int, 2: c2: optional string, 3: c3: optional string>", - () -> - sql( - "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " - + "sort_order => 'zorder(col1)')", - catalogName, tableIdent)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " + + "sort_order => 'zorder(col1)')", + catalogName, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot find column 'col1' in table schema (case sensitive = false): " + + "struct<1: c1: optional int, 2: c2: optional string, 3: c3: optional string>"); // Test for z_order with sort_order - AssertHelpers.assertThrows( - "Should reject calls with error message", - IllegalArgumentException.class, - "Cannot mix identity sort columns and a Zorder sort expression:" + " c1,zorder(c2,c3)", - () -> - sql( - "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " - + "sort_order => 'c1,zorder(c2,c3)')", - catalogName, tableIdent)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " + + "sort_order => 'c1,zorder(c2,c3)')", + catalogName, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot mix identity sort columns and a Zorder sort expression:" + " c1,zorder(c2,c3)"); } @Test public void testInvalidCasesForRewriteDataFiles() { - AssertHelpers.assertThrows( - "Should not allow mixed args", - AnalysisException.class, - "Named and positional arguments cannot be mixed", - () -> sql("CALL %s.system.rewrite_data_files('n', table => 't')", catalogName)); - - AssertHelpers.assertThrows( - "Should not resolve procedures in arbitrary namespaces", - NoSuchProcedureException.class, - "not found", - () -> sql("CALL %s.custom.rewrite_data_files('n', 't')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.rewrite_data_files()", catalogName)); - - AssertHelpers.assertThrows( - "Should reject duplicate arg names name", - AnalysisException.class, - "Duplicate procedure argument: table", - () -> sql("CALL %s.system.rewrite_data_files(table => 't', table => 't')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with empty table identifier", - IllegalArgumentException.class, - "Cannot handle an empty identifier", - () -> sql("CALL %s.system.rewrite_data_files('')", catalogName)); + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.rewrite_data_files('n', table => 't')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage("Named and positional arguments cannot be mixed"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.custom.rewrite_data_files('n', 't')", catalogName)) + .isInstanceOf(NoSuchProcedureException.class) + .hasMessage("Procedure custom.rewrite_data_files not found"); + + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.rewrite_data_files()", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage("Missing required parameters: [table]"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.rewrite_data_files(table => 't', table => 't')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessageEndingWith("Duplicate procedure argument: table"); + + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.rewrite_data_files('')", catalogName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot handle an empty identifier for argument table"); } @Test diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java index 40625b5e3450..dc22fc4b704c 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java @@ -24,7 +24,6 @@ import java.sql.Timestamp; import java.util.List; import java.util.Map; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -32,6 +31,7 @@ import org.apache.spark.sql.RowFactory; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -276,41 +276,33 @@ public void testRewriteManifestsCaseInsensitiveArgs() { @Test public void testInvalidRewriteManifestsCases() { - AssertHelpers.assertThrows( - "Should not allow mixed args", - AnalysisException.class, - "Named and positional arguments cannot be mixed", - () -> sql("CALL %s.system.rewrite_manifests('n', table => 't')", catalogName)); - - AssertHelpers.assertThrows( - "Should not resolve procedures in arbitrary namespaces", - NoSuchProcedureException.class, - "not found", - () -> sql("CALL %s.custom.rewrite_manifests('n', 't')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.rewrite_manifests()", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with invalid arg types", - AnalysisException.class, - "Wrong arg type", - () -> sql("CALL %s.system.rewrite_manifests('n', 2.2)", catalogName)); - - AssertHelpers.assertThrows( - "Should reject duplicate arg names name", - AnalysisException.class, - "Duplicate procedure argument: table", - () -> sql("CALL %s.system.rewrite_manifests(table => 't', tAbLe => 't')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with empty table identifier", - IllegalArgumentException.class, - "Cannot handle an empty identifier", - () -> sql("CALL %s.system.rewrite_manifests('')", catalogName)); + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.rewrite_manifests('n', table => 't')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage("Named and positional arguments cannot be mixed"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.custom.rewrite_manifests('n', 't')", catalogName)) + .isInstanceOf(NoSuchProcedureException.class) + .hasMessage("Procedure custom.rewrite_manifests not found"); + + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.rewrite_manifests()", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage("Missing required parameters: [table]"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.rewrite_manifests('n', 2.2)", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith("Wrong arg type for use_caching"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.rewrite_manifests(table => 't', tAbLe => 't')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage("Could not build name to arg map: Duplicate procedure argument: table"); + + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.rewrite_manifests('')", catalogName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot handle an empty identifier for argument table"); } @Test diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java new file mode 100644 index 000000000000..4c5082cabbb6 --- /dev/null +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java @@ -0,0 +1,426 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.extensions; + +import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; +import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT; +import static org.apache.spark.sql.functions.expr; +import static org.apache.spark.sql.functions.lit; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.Closeable; +import java.io.IOException; +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Timestamp; +import java.time.LocalDateTime; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Files; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.MetadataTableUtils; +import org.apache.iceberg.PositionDeletesScanTask; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.ScanTask; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.RewritePositionDeleteFiles.FileGroupRewriteResult; +import org.apache.iceberg.actions.RewritePositionDeleteFiles.Result; +import org.apache.iceberg.actions.SizeBasedFileRewriter; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.deletes.PositionDeleteWriter; +import org.apache.iceberg.encryption.EncryptedFiles; +import org.apache.iceberg.encryption.EncryptedOutputFile; +import org.apache.iceberg.encryption.EncryptionKeyMetadata; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.actions.SparkActions; +import org.apache.iceberg.util.Pair; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.types.StructType; +import org.junit.After; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runners.Parameterized; + +public class TestRewritePositionDeleteFiles extends SparkExtensionsTestBase { + + private static final Map CATALOG_PROPS = + ImmutableMap.of( + "type", "hive", + "default-namespace", "default", + "cache-enabled", "false"); + + private static final String PARTITION_COL = "partition_col"; + private static final int NUM_DATA_FILES = 5; + private static final int ROWS_PER_DATA_FILE = 100; + private static final int DELETE_FILES_PER_PARTITION = 2; + private static final int DELETE_FILE_SIZE = 10; + + @Parameterized.Parameters( + name = "formatVersion = {0}, catalogName = {1}, implementation = {2}, config = {3}") + public static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HIVE.catalogName(), + SparkCatalogConfig.HIVE.implementation(), + CATALOG_PROPS + } + }; + } + + @Rule public TemporaryFolder temp = new TemporaryFolder(); + + public TestRewritePositionDeleteFiles( + String catalogName, String implementation, Map config) { + super(catalogName, implementation, config); + } + + @After + public void cleanup() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @Test + public void testDatePartition() throws Exception { + createTable("date"); + Date baseDate = Date.valueOf("2023-01-01"); + insertData(i -> Date.valueOf(baseDate.toLocalDate().plusDays(i))); + testDanglingDelete(); + } + + @Test + public void testBooleanPartition() throws Exception { + createTable("boolean"); + insertData(i -> i % 2 == 0, 2); + testDanglingDelete(2); + } + + @Test + public void testTimestampPartition() throws Exception { + createTable("timestamp"); + Timestamp baseTimestamp = Timestamp.valueOf("2023-01-01 15:30:00"); + insertData(i -> Timestamp.valueOf(baseTimestamp.toLocalDateTime().plusDays(i))); + testDanglingDelete(); + } + + @Test + public void testTimestampNtz() throws Exception { + createTable("timestamp_ntz"); + LocalDateTime baseTimestamp = Timestamp.valueOf("2023-01-01 15:30:00").toLocalDateTime(); + insertData(baseTimestamp::plusDays); + testDanglingDelete(); + } + + @Test + public void testBytePartition() throws Exception { + createTable("byte"); + insertData(i -> i); + testDanglingDelete(); + } + + @Test + public void testDecimalPartition() throws Exception { + createTable("decimal(18, 10)"); + BigDecimal baseDecimal = new BigDecimal("1.0"); + insertData(i -> baseDecimal.add(new BigDecimal(i))); + testDanglingDelete(); + } + + @Test + public void testBinaryPartition() throws Exception { + createTable("binary"); + insertData(i -> java.nio.ByteBuffer.allocate(4).putInt(i).array()); + testDanglingDelete(); + } + + @Test + public void testCharPartition() throws Exception { + createTable("char(10)"); + insertData(Object::toString); + testDanglingDelete(); + } + + @Test + public void testVarcharPartition() throws Exception { + createTable("varchar(10)"); + insertData(Object::toString); + testDanglingDelete(); + } + + @Test + public void testIntPartition() throws Exception { + createTable("int"); + insertData(i -> i); + testDanglingDelete(); + } + + @Test + public void testDaysPartitionTransform() throws Exception { + createTable("timestamp", PARTITION_COL, String.format("days(%s)", PARTITION_COL)); + Timestamp baseTimestamp = Timestamp.valueOf("2023-01-01 15:30:00"); + insertData(i -> Timestamp.valueOf(baseTimestamp.toLocalDateTime().plusDays(i))); + testDanglingDelete(); + } + + @Test + public void testNullTransform() throws Exception { + createTable("int"); + insertData(i -> i == 0 ? null : 1, 2); + testDanglingDelete(2); + } + + @Test + public void testPartitionColWithDot() throws Exception { + String partitionColWithDot = "`partition.col`"; + createTable("int", partitionColWithDot, partitionColWithDot); + insertData(partitionColWithDot, i -> i, NUM_DATA_FILES); + testDanglingDelete(partitionColWithDot, NUM_DATA_FILES); + } + + private void testDanglingDelete() throws Exception { + testDanglingDelete(NUM_DATA_FILES); + } + + private void testDanglingDelete(int numDataFiles) throws Exception { + testDanglingDelete(PARTITION_COL, numDataFiles); + } + + private void testDanglingDelete(String partitionCol, int numDataFiles) throws Exception { + Table table = Spark3Util.loadIcebergTable(spark, tableName); + + List dataFiles = dataFiles(table); + assertThat(dataFiles).hasSize(numDataFiles); + + SparkActions.get(spark) + .rewriteDataFiles(table) + .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .execute(); + + // write dangling delete files for 'old data files' + writePosDeletesForFiles(table, dataFiles); + List deleteFiles = deleteFiles(table); + assertThat(deleteFiles).hasSize(numDataFiles * DELETE_FILES_PER_PARTITION); + + List expectedRecords = records(tableName, partitionCol); + + Result result = + SparkActions.get(spark) + .rewritePositionDeletes(table) + .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .execute(); + + List newDeleteFiles = deleteFiles(table); + assertThat(newDeleteFiles).as("Remaining dangling deletes").isEmpty(); + checkResult(result, deleteFiles, Lists.newArrayList(), numDataFiles); + + List actualRecords = records(tableName, partitionCol); + assertEquals("Rows must match", expectedRecords, actualRecords); + } + + private void createTable(String partitionType) { + createTable(partitionType, PARTITION_COL, PARTITION_COL); + } + + private void createTable(String partitionType, String partitionCol, String partitionTransform) { + sql( + "CREATE TABLE %s (id long, %s %s, c1 string, c2 string) " + + "USING iceberg " + + "PARTITIONED BY (%s) " + + "TBLPROPERTIES('format-version'='2')", + tableName, partitionCol, partitionType, partitionTransform); + } + + private void insertData(Function partitionValueFunction) throws Exception { + insertData(partitionValueFunction, NUM_DATA_FILES); + } + + private void insertData(Function partitionValueFunction, int numDataFiles) + throws Exception { + insertData(PARTITION_COL, partitionValueFunction, numDataFiles); + } + + private void insertData( + String partitionCol, Function partitionValue, int numDataFiles) throws Exception { + for (int i = 0; i < numDataFiles; i++) { + Dataset df = + spark + .range(0, ROWS_PER_DATA_FILE) + .withColumn(partitionCol, lit(partitionValue.apply(i))) + .withColumn("c1", expr("CAST(id AS STRING)")) + .withColumn("c2", expr("CAST(id AS STRING)")); + appendAsFile(df); + } + } + + private void appendAsFile(Dataset df) throws Exception { + // ensure the schema is precise + StructType sparkSchema = spark.table(tableName).schema(); + spark.createDataFrame(df.rdd(), sparkSchema).coalesce(1).writeTo(tableName).append(); + } + + private void writePosDeletesForFiles(Table table, List files) throws IOException { + + Map> filesByPartition = + files.stream().collect(Collectors.groupingBy(ContentFile::partition)); + List deleteFiles = + Lists.newArrayListWithCapacity(DELETE_FILES_PER_PARTITION * filesByPartition.size()); + + for (Map.Entry> filesByPartitionEntry : + filesByPartition.entrySet()) { + + StructLike partition = filesByPartitionEntry.getKey(); + List partitionFiles = filesByPartitionEntry.getValue(); + + int deletesForPartition = partitionFiles.size() * DELETE_FILE_SIZE; + assertThat(deletesForPartition % DELETE_FILE_SIZE) + .as("Number of delete files per partition modulo number of data files in this partition") + .isEqualTo(0); + int deleteFileSize = deletesForPartition / DELETE_FILES_PER_PARTITION; + + int counter = 0; + List> deletes = Lists.newArrayList(); + for (DataFile partitionFile : partitionFiles) { + for (int deletePos = 0; deletePos < DELETE_FILE_SIZE; deletePos++) { + deletes.add(Pair.of(partitionFile.path(), (long) deletePos)); + counter++; + if (counter == deleteFileSize) { + // Dump to file and reset variables + OutputFile output = Files.localOutput(temp.newFile()); + deleteFiles.add(writeDeleteFile(table, output, partition, deletes)); + counter = 0; + deletes.clear(); + } + } + } + } + + RowDelta rowDelta = table.newRowDelta(); + deleteFiles.forEach(rowDelta::addDeletes); + rowDelta.commit(); + } + + private DeleteFile writeDeleteFile( + Table table, OutputFile out, StructLike partition, List> deletes) + throws IOException { + FileFormat format = defaultFormat(table.properties()); + FileAppenderFactory factory = new GenericAppenderFactory(table.schema(), table.spec()); + + PositionDeleteWriter writer = + factory.newPosDeleteWriter(encrypt(out), format, partition); + PositionDelete posDelete = PositionDelete.create(); + try (Closeable toClose = writer) { + for (Pair delete : deletes) { + writer.write(posDelete.set(delete.first(), delete.second(), null)); + } + } + + return writer.toDeleteFile(); + } + + private static EncryptedOutputFile encrypt(OutputFile out) { + return EncryptedFiles.encryptedOutput(out, EncryptionKeyMetadata.EMPTY); + } + + private static FileFormat defaultFormat(Map properties) { + String formatString = properties.getOrDefault(DEFAULT_FILE_FORMAT, DEFAULT_FILE_FORMAT_DEFAULT); + return FileFormat.fromString(formatString); + } + + private List records(String table, String partitionCol) { + return rowsToJava( + spark.read().format("iceberg").load(table).sort(partitionCol, "id").collectAsList()); + } + + private long size(List deleteFiles) { + return deleteFiles.stream().mapToLong(DeleteFile::fileSizeInBytes).sum(); + } + + private List dataFiles(Table table) { + CloseableIterable tasks = table.newScan().includeColumnStats().planFiles(); + return Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); + } + + private List deleteFiles(Table table) { + Table deletesTable = + MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES); + CloseableIterable tasks = deletesTable.newBatchScan().planFiles(); + return Lists.newArrayList( + CloseableIterable.transform(tasks, t -> ((PositionDeletesScanTask) t).file())); + } + + private void checkResult( + Result result, + List rewrittenDeletes, + List newDeletes, + int expectedGroups) { + assertThat(result.rewrittenDeleteFilesCount()) + .as("Rewritten delete files") + .isEqualTo(rewrittenDeletes.size()); + assertThat(result.addedDeleteFilesCount()) + .as("Added delete files") + .isEqualTo(newDeletes.size()); + assertThat(result.rewrittenBytesCount()) + .as("Rewritten delete bytes") + .isEqualTo(size(rewrittenDeletes)); + assertThat(result.addedBytesCount()).as("New Delete byte count").isEqualTo(size(newDeletes)); + + assertThat(result.rewriteResults()).as("Rewritten group count").hasSize(expectedGroups); + assertThat( + result.rewriteResults().stream() + .mapToInt(FileGroupRewriteResult::rewrittenDeleteFilesCount) + .sum()) + .as("Rewritten delete file count in all groups") + .isEqualTo(rewrittenDeletes.size()); + assertThat( + result.rewriteResults().stream() + .mapToInt(FileGroupRewriteResult::addedDeleteFilesCount) + .sum()) + .as("Added delete file count in all groups") + .isEqualTo(newDeletes.size()); + assertThat( + result.rewriteResults().stream() + .mapToLong(FileGroupRewriteResult::rewrittenBytesCount) + .sum()) + .as("Rewritten delete bytes in all groups") + .isEqualTo(size(rewrittenDeletes)); + assertThat( + result.rewriteResults().stream() + .mapToLong(FileGroupRewriteResult::addedBytesCount) + .sum()) + .as("Added delete bytes in all groups") + .isEqualTo(size(newDeletes)); + } +} diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToSnapshotProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToSnapshotProcedure.java index af94b456d02e..1b4cd2b4a3f6 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToSnapshotProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToSnapshotProcedure.java @@ -20,7 +20,6 @@ import java.util.List; import java.util.Map; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; @@ -29,6 +28,7 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assume; import org.junit.Test; @@ -240,58 +240,49 @@ public void testRollbackToSnapshotWithoutExplicitCatalog() { public void testRollbackToInvalidSnapshot() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); - AssertHelpers.assertThrows( - "Should reject invalid snapshot id", - ValidationException.class, - "Cannot roll back to unknown snapshot id", - () -> sql("CALL %s.system.rollback_to_snapshot('%s', -1L)", catalogName, tableIdent)); + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.rollback_to_snapshot('%s', -1L)", catalogName, tableIdent)) + .isInstanceOf(ValidationException.class) + .hasMessage("Cannot roll back to unknown snapshot id: -1"); } @Test public void testInvalidRollbackToSnapshotCases() { - AssertHelpers.assertThrows( - "Should not allow mixed args", - AnalysisException.class, - "Named and positional arguments cannot be mixed", - () -> - sql( - "CALL %s.system.rollback_to_snapshot(namespace => 'n1', table => 't', 1L)", - catalogName)); - - AssertHelpers.assertThrows( - "Should not resolve procedures in arbitrary namespaces", - NoSuchProcedureException.class, - "not found", - () -> sql("CALL %s.custom.rollback_to_snapshot('n', 't', 1L)", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.rollback_to_snapshot('t')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.rollback_to_snapshot(1L)", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.rollback_to_snapshot(table => 't')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with invalid arg types", - AnalysisException.class, - "Wrong arg type for snapshot_id: cannot cast", - () -> sql("CALL %s.system.rollback_to_snapshot('t', 2.2)", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with empty table identifier", - IllegalArgumentException.class, - "Cannot handle an empty identifier", - () -> sql("CALL %s.system.rollback_to_snapshot('', 1L)", catalogName)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.rollback_to_snapshot(namespace => 'n1', table => 't', 1L)", + catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage("Named and positional arguments cannot be mixed"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.custom.rollback_to_snapshot('n', 't', 1L)", catalogName)) + .isInstanceOf(NoSuchProcedureException.class) + .hasMessage("Procedure custom.rollback_to_snapshot not found"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.rollback_to_snapshot('t')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage("Missing required parameters: [snapshot_id]"); + + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_snapshot(1L)", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage("Missing required parameters: [snapshot_id]"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.rollback_to_snapshot(table => 't')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage("Missing required parameters: [snapshot_id]"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.rollback_to_snapshot('t', 2.2)", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage("Wrong arg type for snapshot_id: cannot cast DecimalType(2,1) to LongType"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.rollback_to_snapshot('', 1L)", catalogName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot handle an empty identifier for argument table"); } } diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java index 6da3853bbe24..80af6e7f5f4c 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java @@ -21,7 +21,6 @@ import java.time.LocalDateTime; import java.util.List; import java.util.Map; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -29,6 +28,7 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assume; import org.junit.Test; @@ -255,50 +255,48 @@ public void testRollbackToTimestampWithoutExplicitCatalog() { public void testInvalidRollbackToTimestampCases() { String timestamp = "TIMESTAMP '2007-12-03T10:15:30'"; - AssertHelpers.assertThrows( - "Should not allow mixed args", - AnalysisException.class, - "Named and positional arguments cannot be mixed", - () -> - sql( - "CALL %s.system.rollback_to_timestamp(namespace => 'n1', 't', %s)", - catalogName, timestamp)); - - AssertHelpers.assertThrows( - "Should not resolve procedures in arbitrary namespaces", - NoSuchProcedureException.class, - "not found", - () -> sql("CALL %s.custom.rollback_to_timestamp('n', 't', %s)", catalogName, timestamp)); - - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.rollback_to_timestamp('t')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.rollback_to_timestamp(timestamp => %s)", catalogName, timestamp)); - - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.rollback_to_timestamp(table => 't')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with extra args", - AnalysisException.class, - "Too many arguments", - () -> - sql("CALL %s.system.rollback_to_timestamp('n', 't', %s, 1L)", catalogName, timestamp)); - - AssertHelpers.assertThrows( - "Should reject calls with invalid arg types", - AnalysisException.class, - "Wrong arg type for timestamp: cannot cast", - () -> sql("CALL %s.system.rollback_to_timestamp('t', 2.2)", catalogName)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.rollback_to_timestamp(namespace => 'n1', 't', %s)", + catalogName, timestamp)) + .isInstanceOf(AnalysisException.class) + .hasMessage("Named and positional arguments cannot be mixed"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.custom.rollback_to_timestamp('n', 't', %s)", catalogName, timestamp)) + .isInstanceOf(NoSuchProcedureException.class) + .hasMessage("Procedure custom.rollback_to_timestamp not found"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.rollback_to_timestamp('t')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage("Missing required parameters: [timestamp]"); + + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.rollback_to_timestamp(timestamp => %s)", + catalogName, timestamp)) + .isInstanceOf(AnalysisException.class) + .hasMessage("Missing required parameters: [table]"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.rollback_to_timestamp(table => 't')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage("Missing required parameters: [timestamp]"); + + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.rollback_to_timestamp('n', 't', %s, 1L)", + catalogName, timestamp)) + .isInstanceOf(AnalysisException.class) + .hasMessage("Too many arguments for procedure"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.rollback_to_timestamp('t', 2.2)", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage("Wrong arg type for timestamp: cannot cast DecimalType(2,1) to TimestampType"); } } diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetCurrentSnapshotProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetCurrentSnapshotProcedure.java index 8a8a974bbebe..51db8d321059 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetCurrentSnapshotProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetCurrentSnapshotProcedure.java @@ -22,15 +22,14 @@ import java.util.List; import java.util.Map; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assume; import org.junit.Test; @@ -190,67 +189,54 @@ public void tesSetCurrentSnapshotWithoutExplicitCatalog() { public void testSetCurrentSnapshotToInvalidSnapshot() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); - Namespace namespace = tableIdent.namespace(); - String tableName = tableIdent.name(); - - AssertHelpers.assertThrows( - "Should reject invalid snapshot id", - ValidationException.class, - "Cannot roll back to unknown snapshot id", - () -> sql("CALL %s.system.set_current_snapshot('%s', -1L)", catalogName, tableIdent)); + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.set_current_snapshot('%s', -1L)", catalogName, tableIdent)) + .isInstanceOf(ValidationException.class) + .hasMessage("Cannot roll back to unknown snapshot id: -1"); } @Test public void testInvalidRollbackToSnapshotCases() { - AssertHelpers.assertThrows( - "Should not allow mixed args", - AnalysisException.class, - "Named and positional arguments cannot be mixed", - () -> - sql( - "CALL %s.system.set_current_snapshot(namespace => 'n1', table => 't', 1L)", - catalogName)); - - AssertHelpers.assertThrows( - "Should not resolve procedures in arbitrary namespaces", - NoSuchProcedureException.class, - "not found", - () -> sql("CALL %s.custom.set_current_snapshot('n', 't', 1L)", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.set_current_snapshot('t')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.set_current_snapshot(1L)", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.set_current_snapshot(snapshot_id => 1L)", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.set_current_snapshot(table => 't')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with invalid arg types", - AnalysisException.class, - "Wrong arg type for snapshot_id: cannot cast", - () -> sql("CALL %s.system.set_current_snapshot('t', 2.2)", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with empty table identifier", - IllegalArgumentException.class, - "Cannot handle an empty identifier", - () -> sql("CALL %s.system.set_current_snapshot('', 1L)", catalogName)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.set_current_snapshot(namespace => 'n1', table => 't', 1L)", + catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage("Named and positional arguments cannot be mixed"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.custom.set_current_snapshot('n', 't', 1L)", catalogName)) + .isInstanceOf(NoSuchProcedureException.class) + .hasMessage("Procedure custom.set_current_snapshot not found"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.set_current_snapshot('t')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage("Missing required parameters: [snapshot_id]"); + + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.set_current_snapshot(1L)", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage("Missing required parameters: [snapshot_id]"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.set_current_snapshot(snapshot_id => 1L)", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage("Missing required parameters: [table]"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.set_current_snapshot(table => 't')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage("Missing required parameters: [snapshot_id]"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.set_current_snapshot('t', 2.2)", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage("Wrong arg type for snapshot_id: cannot cast DecimalType(2,1) to LongType"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.set_current_snapshot('', 1L)", catalogName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot handle an empty identifier for argument table"); } } diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java index ed64ef331580..6e2bf99c5422 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java @@ -20,11 +20,11 @@ import java.io.IOException; import java.util.Map; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.spark.sql.AnalysisException; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -195,37 +195,30 @@ public void testInvalidSnapshotsCases() throws IOException { "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", sourceName, location); - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.snapshot('foo')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with invalid arg types", - AnalysisException.class, - "Wrong arg type", - () -> sql("CALL %s.system.snapshot('n', 't', map('foo', 'bar'))", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with invalid map args", - AnalysisException.class, - "The `map` requires 2n (n > 0) parameters but the actual number is 3", - () -> - sql( - "CALL %s.system.snapshot('%s', 'fable', 'loc', map(2, 1, 1))", - catalogName, sourceName)); - - AssertHelpers.assertThrows( - "Should reject calls with empty table identifier", - IllegalArgumentException.class, - "Cannot handle an empty identifier", - () -> sql("CALL %s.system.snapshot('', 'dest')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with empty table identifier", - IllegalArgumentException.class, - "Cannot handle an empty identifier", - () -> sql("CALL %s.system.snapshot('src', '')", catalogName)); + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.snapshot('foo')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage("Missing required parameters: [table]"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.snapshot('n', 't', map('foo', 'bar'))", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith("Wrong arg type for location"); + + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.snapshot('%s', 'fable', 'loc', map(2, 1, 1))", + catalogName, sourceName)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "The `map` requires 2n (n > 0) parameters but the actual number is 3"); + + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.snapshot('', 'dest')", catalogName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot handle an empty identifier for argument source_table"); + + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.snapshot('src', '')", catalogName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot handle an empty identifier for argument table"); } } diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestTagDDL.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestTagDDL.java index ec3148de6cb5..866a965e33e6 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestTagDDL.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestTagDDL.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.List; import java.util.Locale; import java.util.Map; @@ -364,6 +366,18 @@ public void testDropTagIfExists() throws NoSuchTableException { Assert.assertNull("The tag needs to be dropped.", table.refs().get(tagName)); } + @Test + public void createOrReplaceWithNonExistingTag() throws NoSuchTableException { + Table table = insertRows(); + String tagName = "t1"; + insertRows(); + long snapshotId = table.currentSnapshot().snapshotId(); + + sql("ALTER TABLE %s CREATE OR REPLACE TAG %s AS OF VERSION %d", tableName, tagName, snapshotId); + table.refresh(); + assertThat(table.refs().get(tagName).snapshotId()).isEqualTo(snapshotId); + } + private Table insertRows() throws NoSuchTableException { List records = ImmutableList.of(new SimpleRecord(1, "a"), new SimpleRecord(2, "b")); diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java index 37b38621b1a1..15484f45f895 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java @@ -21,7 +21,6 @@ import java.util.List; import java.util.Map; import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.TableProperties; import org.apache.iceberg.hadoop.HadoopFileIO; @@ -38,7 +37,7 @@ import org.apache.spark.SparkException; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Rule; import org.junit.Test; @@ -105,23 +104,17 @@ public void testBatchAppend() throws Exception { new SimpleRecord(4, "b")); Dataset inputDF = spark.createDataFrame(records, SimpleRecord.class); - AssertHelpers.assertThrows( - "Write must fail", - SparkException.class, - "Encountered records that belong to already closed files", - () -> { - try { - // incoming records are not ordered by partitions so the job must fail - inputDF - .coalesce(1) - .sortWithinPartitions("id") - .writeTo(tableName) - .option(SparkWriteOptions.USE_TABLE_DISTRIBUTION_AND_ORDERING, "false") - .append(); - } catch (NoSuchTableException e) { - throw new RuntimeException(e); - } - }); + Assertions.assertThatThrownBy( + () -> + // incoming records are not ordered by partitions so the job must fail + inputDF + .coalesce(1) + .sortWithinPartitions("id") + .writeTo(tableName) + .option(SparkWriteOptions.USE_TABLE_DISTRIBUTION_AND_ORDERING, "false") + .append()) + .isInstanceOf(SparkException.class) + .hasMessageContaining("Encountered records that belong to already closed files"); assertEquals("Should be no records", sql("SELECT * FROM %s", tableName), ImmutableList.of()); diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/ChangelogIterator.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/ChangelogIterator.java index 5f30c5fd4e56..cc44b1f3992c 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/ChangelogIterator.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/ChangelogIterator.java @@ -20,8 +20,10 @@ import java.util.Iterator; import java.util.Objects; +import java.util.Set; import org.apache.iceberg.ChangelogOperation; import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Iterators; import org.apache.spark.sql.Row; import org.apache.spark.sql.types.StructType; @@ -35,9 +37,11 @@ public abstract class ChangelogIterator implements Iterator { private final Iterator rowIterator; private final int changeTypeIndex; + private final StructType rowType; protected ChangelogIterator(Iterator rowIterator, StructType rowType) { this.rowIterator = rowIterator; + this.rowType = rowType; this.changeTypeIndex = rowType.fieldIndex(MetadataColumns.CHANGE_TYPE.name()); } @@ -45,6 +49,16 @@ protected int changeTypeIndex() { return changeTypeIndex; } + protected StructType rowType() { + return rowType; + } + + protected String changeType(Row row) { + String changeType = row.getString(changeTypeIndex()); + Preconditions.checkNotNull(changeType, "Change type should not be null"); + return changeType; + } + protected Iterator rowIterator() { return rowIterator; } @@ -79,7 +93,35 @@ public static Iterator removeCarryovers(Iterator rowIterator, StructTy return Iterators.filter(changelogIterator, Objects::nonNull); } + public static Iterator removeNetCarryovers(Iterator rowIterator, StructType rowType) { + ChangelogIterator changelogIterator = new RemoveNetCarryoverIterator(rowIterator, rowType); + return Iterators.filter(changelogIterator, Objects::nonNull); + } + + protected boolean isSameRecord(Row currentRow, Row nextRow, int[] indicesToIdentifySameRow) { + for (int idx : indicesToIdentifySameRow) { + if (isDifferentValue(currentRow, nextRow, idx)) { + return false; + } + } + + return true; + } + protected boolean isDifferentValue(Row currentRow, Row nextRow, int idx) { return !Objects.equals(nextRow.get(idx), currentRow.get(idx)); } + + protected static int[] generateIndicesToIdentifySameRow( + int totalColumnCount, Set metadataColumnIndices) { + int[] indices = new int[totalColumnCount - metadataColumnIndices.size()]; + + for (int i = 0, j = 0; i < indices.length; i++) { + if (!metadataColumnIndices.contains(i)) { + indices[j] = i; + j++; + } + } + return indices; + } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java index 641b957d1176..ea400a779235 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java @@ -20,7 +20,9 @@ import java.util.Map; import java.util.concurrent.Callable; +import org.apache.iceberg.SnapshotSummary; 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.ExceptionUtil; /** utility class to accept thread local commit properties */ @@ -35,13 +37,19 @@ private CommitMetadata() {} * running the code wrapped as a caller, and any snapshot committed within the callable object * will be attached with the metadata defined in properties * - * @param properties extra commit metadata to attach to the snapshot committed within callable + * @param properties extra commit metadata to attach to the snapshot committed within callable. + * The prefix will be removed for properties starting with {@link + * SnapshotSummary#EXTRA_METADATA_PREFIX} * @param callable the code to be executed * @param exClass the expected type of exception which would be thrown from callable */ public static R withCommitProperties( Map properties, Callable callable, Class exClass) throws E { - COMMIT_PROPERTIES.set(properties); + Map props = Maps.newHashMap(); + properties.forEach( + (k, v) -> props.put(k.replace(SnapshotSummary.EXTRA_METADATA_PREFIX, ""), v)); + + COMMIT_PROPERTIES.set(props); try { return callable.call(); } catch (Throwable e) { diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/ComputeUpdateIterator.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/ComputeUpdateIterator.java index 23e6a19a17e7..6951c33e51aa 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/ComputeUpdateIterator.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/ComputeUpdateIterator.java @@ -81,15 +81,13 @@ public Row next() { // either a cached record which is not an UPDATE or the next record in the iterator. Row currentRow = currentRow(); - if (currentRow.getString(changeTypeIndex()).equals(DELETE) && rowIterator().hasNext()) { + if (changeType(currentRow).equals(DELETE) && rowIterator().hasNext()) { Row nextRow = rowIterator().next(); cachedRow = nextRow; if (sameLogicalRow(currentRow, nextRow)) { - String nextRowChangeType = nextRow.getString(changeTypeIndex()); - Preconditions.checkState( - nextRowChangeType.equals(INSERT), + changeType(nextRow).equals(INSERT), "Cannot compute updates because there are multiple rows with the same identifier" + " fields([%s]). Please make sure the rows are unique.", String.join(",", identifierFields)); @@ -118,7 +116,7 @@ private Row modify(Row row, int valueIndex, Object value) { } private boolean cachedUpdateRecord() { - return cachedRow != null && cachedRow.getString(changeTypeIndex()).equals(UPDATE_AFTER); + return cachedRow != null && changeType(cachedRow).equals(UPDATE_AFTER); } private Row currentRow() { diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/RemoveCarryoverIterator.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/RemoveCarryoverIterator.java index 70b160e13fee..2e90dc7749d1 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/RemoveCarryoverIterator.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/RemoveCarryoverIterator.java @@ -19,6 +19,8 @@ package org.apache.iceberg.spark; import java.util.Iterator; +import java.util.Set; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.spark.sql.Row; import org.apache.spark.sql.types.StructType; @@ -55,7 +57,7 @@ class RemoveCarryoverIterator extends ChangelogIterator { RemoveCarryoverIterator(Iterator rowIterator, StructType rowType) { super(rowIterator, rowType); - this.indicesToIdentifySameRow = generateIndicesToIdentifySameRow(rowType.size()); + this.indicesToIdentifySameRow = generateIndicesToIdentifySameRow(); } @Override @@ -88,7 +90,7 @@ public Row next() { } // If the current row is a delete row, drain all identical delete rows - if (currentRow.getString(changeTypeIndex()).equals(DELETE) && rowIterator().hasNext()) { + if (changeType(currentRow).equals(DELETE) && rowIterator().hasNext()) { cachedDeletedRow = currentRow; deletedRowCount = 1; @@ -98,8 +100,8 @@ public Row next() { // row is the same record while (nextRow != null && cachedDeletedRow != null - && isSameRecord(cachedDeletedRow, nextRow)) { - if (nextRow.getString(changeTypeIndex()).equals(INSERT)) { + && isSameRecord(cachedDeletedRow, nextRow, indicesToIdentifySameRow)) { + if (changeType(nextRow).equals(INSERT)) { deletedRowCount--; if (deletedRowCount == 0) { cachedDeletedRow = null; @@ -139,25 +141,8 @@ private boolean hasCachedDeleteRow() { return cachedDeletedRow != null; } - private int[] generateIndicesToIdentifySameRow(int columnSize) { - int[] indices = new int[columnSize - 1]; - for (int i = 0; i < indices.length; i++) { - if (i < changeTypeIndex()) { - indices[i] = i; - } else { - indices[i] = i + 1; - } - } - return indices; - } - - private boolean isSameRecord(Row currentRow, Row nextRow) { - for (int idx : indicesToIdentifySameRow) { - if (isDifferentValue(currentRow, nextRow, idx)) { - return false; - } - } - - return true; + private int[] generateIndicesToIdentifySameRow() { + Set metadataColumnIndices = Sets.newHashSet(changeTypeIndex()); + return generateIndicesToIdentifySameRow(rowType().size(), metadataColumnIndices); } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/RemoveNetCarryoverIterator.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/RemoveNetCarryoverIterator.java new file mode 100644 index 000000000000..941e4a4731e2 --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/RemoveNetCarryoverIterator.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark; + +import java.util.Iterator; +import java.util.Set; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.types.StructType; + +/** + * This class computes the net changes across multiple snapshots. It is different from {@link + * org.apache.iceberg.spark.RemoveCarryoverIterator}, which only removes carry-over rows within a + * single snapshot. It takes a row iterator, and assumes the following: + * + *

      + *
    • The row iterator is partitioned by all columns. + *
    • The row iterator is sorted by all columns, change order, and change type. The change order + * is 1-to-1 mapping to snapshot id. + *
    + */ +public class RemoveNetCarryoverIterator extends ChangelogIterator { + + private final int[] indicesToIdentifySameRow; + + private Row cachedNextRow; + private Row cachedRow; + private long cachedRowCount; + + protected RemoveNetCarryoverIterator(Iterator rowIterator, StructType rowType) { + super(rowIterator, rowType); + this.indicesToIdentifySameRow = generateIndicesToIdentifySameRow(); + } + + @Override + public boolean hasNext() { + if (cachedRowCount > 0) { + return true; + } + + if (cachedNextRow != null) { + return true; + } + + return rowIterator().hasNext(); + } + + @Override + public Row next() { + // if there are cached rows, return one of them from the beginning + if (cachedRowCount > 0) { + cachedRowCount--; + return cachedRow; + } + + cachedRow = getCurrentRow(); + // return it directly if there is no more rows + if (!rowIterator().hasNext()) { + return cachedRow; + } + cachedRowCount = 1; + + cachedNextRow = rowIterator().next(); + + // pull rows from the iterator until two consecutive rows are different + while (isSameRecord(cachedRow, cachedNextRow, indicesToIdentifySameRow)) { + if (oppositeChangeType(cachedRow, cachedNextRow)) { + // two rows with opposite change types means no net changes, remove both + cachedRowCount--; + } else { + // two rows with same change types means potential net changes, cache the next row + cachedRowCount++; + } + + // stop pulling rows if there is no more rows or the next row is different + if (cachedRowCount <= 0 || !rowIterator().hasNext()) { + // reset the cached next row if there is no more rows + cachedNextRow = null; + break; + } + + cachedNextRow = rowIterator().next(); + } + + return null; + } + + private Row getCurrentRow() { + Row currentRow; + if (cachedNextRow != null) { + currentRow = cachedNextRow; + cachedNextRow = null; + } else { + currentRow = rowIterator().next(); + } + return currentRow; + } + + private boolean oppositeChangeType(Row currentRow, Row nextRow) { + return (changeType(nextRow).equals(INSERT) && changeType(currentRow).equals(DELETE)) + || (changeType(nextRow).equals(DELETE) && changeType(currentRow).equals(INSERT)); + } + + private int[] generateIndicesToIdentifySameRow() { + Set metadataColumnIndices = + Sets.newHashSet( + rowType().fieldIndex(MetadataColumns.CHANGE_ORDINAL.name()), + rowType().fieldIndex(MetadataColumns.COMMIT_SNAPSHOT_ID.name()), + changeTypeIndex()); + return generateIndicesToIdentifySameRow(rowType().size(), metadataColumnIndices); + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java index 421e7a07a162..85e368d8cf69 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java @@ -267,4 +267,12 @@ public boolean aggregatePushDownEnabled() { .defaultValue(SparkSQLProperties.AGGREGATE_PUSH_DOWN_ENABLED_DEFAULT) .parse(); } + + public boolean adaptiveSplitSizeEnabled() { + return confParser + .booleanConf() + .tableProperty(TableProperties.ADAPTIVE_SPLIT_SIZE_ENABLED) + .defaultValue(TableProperties.ADAPTIVE_SPLIT_SIZE_ENABLED_DEFAULT) + .parse(); + } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkSessionCatalog.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkSessionCatalog.java index c891985b383d..33384e3eff08 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkSessionCatalog.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkSessionCatalog.java @@ -22,6 +22,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.spark.source.HasIcebergCatalog; @@ -303,7 +304,10 @@ public void renameTable(Identifier from, Identifier to) @Override public final void initialize(String name, CaseInsensitiveStringMap options) { - if (options.containsKey("type") && options.get("type").equalsIgnoreCase("hive")) { + if (options.containsKey(CatalogUtil.ICEBERG_CATALOG_TYPE) + && options + .get(CatalogUtil.ICEBERG_CATALOG_TYPE) + .equalsIgnoreCase(CatalogUtil.ICEBERG_CATALOG_TYPE_HIVE)) { validateHmsUri(options.get(CatalogProperties.URI)); } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java index b8c00b713a23..88b752c3c6dd 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java @@ -44,6 +44,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.TableMigrationUtil; +import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.hadoop.HadoopFileIO; import org.apache.iceberg.hadoop.SerializableConfiguration; import org.apache.iceberg.hadoop.Util; @@ -403,12 +404,12 @@ public static void importSparkTable( } else { List sourceTablePartitions = getPartitions(spark, sourceTableIdent, partitionFilter); - Preconditions.checkArgument( - !sourceTablePartitions.isEmpty(), - "Cannot find any partitions in table %s", - sourceTableIdent); - importSparkPartitions( - spark, sourceTablePartitions, targetTable, spec, stagingDir, checkDuplicateFiles); + if (sourceTablePartitions.isEmpty()) { + targetTable.newAppend().commit(); + } else { + importSparkPartitions( + spark, sourceTablePartitions, targetTable, spec, stagingDir, checkDuplicateFiles); + } } } catch (AnalysisException e) { throw SparkExceptionUtil.toUncheckedException( @@ -666,6 +667,43 @@ public static Dataset loadMetadataTable( spark, DataSourceV2Relation.create(metadataTable, Some.empty(), Some.empty(), options)); } + /** + * Determine the write branch. + * + *

    Validate wap config and determine the write branch. + * + * @param spark a Spark Session + * @param branch write branch if there is no WAP branch configured + * @return branch for write operation + */ + public static String determineWriteBranch(SparkSession spark, String branch) { + String wapId = spark.conf().get(SparkSQLProperties.WAP_ID, null); + String wapBranch = spark.conf().get(SparkSQLProperties.WAP_BRANCH, null); + ValidationException.check( + wapId == null || wapBranch == null, + "Cannot set both WAP ID and branch, but got ID [%s] and branch [%s]", + wapId, + wapBranch); + + if (wapBranch != null) { + ValidationException.check( + branch == null, + "Cannot write to both branch and WAP branch, but got branch [%s] and WAP branch [%s]", + branch, + wapBranch); + + return wapBranch; + } + return branch; + } + + public static boolean wapEnabled(Table table) { + return PropertyUtil.propertyAsBoolean( + table.properties(), + TableProperties.WRITE_AUDIT_PUBLISH_ENABLED, + Boolean.getBoolean(TableProperties.WRITE_AUDIT_PUBLISH_ENABLED_DEFAULT)); + } + /** Class representing a table partition. */ public static class SparkPartition implements Serializable { private final Map values; diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkV2Filters.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkV2Filters.java index 6d564bbd623b..cbedc4d56881 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkV2Filters.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkV2Filters.java @@ -28,9 +28,12 @@ import static org.apache.iceberg.expressions.Expressions.lessThan; import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual; import static org.apache.iceberg.expressions.Expressions.not; +import static org.apache.iceberg.expressions.Expressions.notEqual; import static org.apache.iceberg.expressions.Expressions.notIn; +import static org.apache.iceberg.expressions.Expressions.notNaN; import static org.apache.iceberg.expressions.Expressions.notNull; import static org.apache.iceberg.expressions.Expressions.or; +import static org.apache.iceberg.expressions.Expressions.ref; import static org.apache.iceberg.expressions.Expressions.startsWith; import java.util.Arrays; @@ -40,9 +43,12 @@ import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expression.Operation; import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.UnboundPredicate; +import org.apache.iceberg.expressions.UnboundTerm; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.util.NaNUtil; +import org.apache.iceberg.util.Pair; import org.apache.spark.sql.connector.expressions.Literal; import org.apache.spark.sql.connector.expressions.NamedReference; import org.apache.spark.sql.connector.expressions.filter.And; @@ -57,6 +63,7 @@ public class SparkV2Filters { private static final String FALSE = "ALWAYS_FALSE"; private static final String EQ = "="; private static final String EQ_NULL_SAFE = "<=>"; + private static final String NOT_EQ = "<>"; private static final String GT = ">"; private static final String GT_EQ = ">="; private static final String LT = "<"; @@ -75,6 +82,7 @@ public class SparkV2Filters { .put(FALSE, Operation.FALSE) .put(EQ, Operation.EQ) .put(EQ_NULL_SAFE, Operation.EQ) + .put(NOT_EQ, Operation.NOT_EQ) .put(GT, Operation.GT) .put(GT_EQ, Operation.GT_EQ) .put(LT, Operation.LT) @@ -152,31 +160,35 @@ public static Expression convert(Predicate predicate) { } case EQ: // used for both eq and null-safe-eq - Object value; - String columnName; - if (isRef(leftChild(predicate)) && isLiteral(rightChild(predicate))) { - columnName = SparkUtil.toColumnName(leftChild(predicate)); - value = convertLiteral(rightChild(predicate)); - } else if (isRef(rightChild(predicate)) && isLiteral(leftChild(predicate))) { - columnName = SparkUtil.toColumnName(rightChild(predicate)); - value = convertLiteral(leftChild(predicate)); - } else { + Pair, Object> eqChildren = predicateChildren(predicate); + if (eqChildren == null) { return null; } if (predicate.name().equals(EQ)) { // comparison with null in normal equality is always null. this is probably a mistake. Preconditions.checkNotNull( - value, "Expression is always false (eq is not null-safe): %s", predicate); - return handleEqual(columnName, value); - } else { // "<=>" - if (value == null) { - return isNull(columnName); - } else { - return handleEqual(columnName, value); - } + eqChildren.second(), + "Expression is always false (eq is not null-safe): %s", + predicate); + } + + return handleEqual(eqChildren.first(), eqChildren.second()); + + case NOT_EQ: + Pair, Object> notEqChildren = predicateChildren(predicate); + if (notEqChildren == null) { + return null; } + // comparison with null in normal equality is always null. this is probably a mistake. + Preconditions.checkNotNull( + notEqChildren.second(), + "Expression is always false (notEq is not null-safe): %s", + predicate); + + return handleNotEqual(notEqChildren.first(), notEqChildren.second()); + case IN: if (isSupportedInPredicate(predicate)) { return in( @@ -245,6 +257,22 @@ public static Expression convert(Predicate predicate) { return null; } + private static Pair, Object> predicateChildren(Predicate predicate) { + if (isRef(leftChild(predicate)) && isLiteral(rightChild(predicate))) { + UnboundTerm term = ref(SparkUtil.toColumnName(leftChild(predicate))); + Object value = convertLiteral(rightChild(predicate)); + return Pair.of(term, value); + + } else if (isRef(rightChild(predicate)) && isLiteral(leftChild(predicate))) { + UnboundTerm term = ref(SparkUtil.toColumnName(rightChild(predicate))); + Object value = convertLiteral(leftChild(predicate)); + return Pair.of(term, value); + + } else { + return null; + } + } + @SuppressWarnings("unchecked") private static T child(Predicate predicate) { org.apache.spark.sql.connector.expressions.Expression[] children = predicate.children(); @@ -289,11 +317,21 @@ private static Object convertLiteral(Literal literal) { return literal.value(); } - private static Expression handleEqual(String attribute, Object value) { + private static UnboundPredicate handleEqual(UnboundTerm term, Object value) { + if (value == null) { + return isNull(term); + } else if (NaNUtil.isNaN(value)) { + return isNaN(term); + } else { + return equal(term, value); + } + } + + private static UnboundPredicate handleNotEqual(UnboundTerm term, Object value) { if (NaNUtil.isNaN(value)) { - return isNaN(attribute); + return notNaN(term); } else { - return equal(attribute, value); + return notEqual(term, value); } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkValueConverter.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkValueConverter.java index 7de0039fbb3a..53fb7ec44a4d 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkValueConverter.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkValueConverter.java @@ -29,6 +29,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.ByteBuffers; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.util.DateTimeUtils; @@ -129,4 +130,39 @@ private static Record convert(Types.StructType struct, Row row) { } return record; } + + public static Object convertToSpark(Type type, Object object) { + if (object == null) { + return null; + } + + switch (type.typeId()) { + case STRUCT: + case LIST: + case MAP: + return new UnsupportedOperationException("Complex types currently not supported"); + case DATE: + return DateTimeUtils.daysToLocalDate((int) object); + case TIMESTAMP: + Types.TimestampType ts = (Types.TimestampType) type.asPrimitiveType(); + if (ts.shouldAdjustToUTC()) { + return DateTimeUtils.microsToInstant((long) object); + } else { + return DateTimeUtils.microsToLocalDateTime((long) object); + } + case BINARY: + return ByteBuffers.toByteArray((ByteBuffer) object); + case INTEGER: + case BOOLEAN: + case LONG: + case FLOAT: + case DOUBLE: + case DECIMAL: + case STRING: + case FIXED: + return object; + default: + throw new UnsupportedOperationException("Not a supported type: " + type); + } + } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/SparkBinPackPositionDeletesRewriter.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/SparkBinPackPositionDeletesRewriter.java index 51c4cc661f4e..1b36441c3c4d 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/SparkBinPackPositionDeletesRewriter.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/SparkBinPackPositionDeletesRewriter.java @@ -20,6 +20,7 @@ import static org.apache.iceberg.MetadataTableType.POSITION_DELETES; import static org.apache.spark.sql.functions.col; +import static org.apache.spark.sql.functions.lit; import java.util.List; import java.util.Optional; @@ -40,7 +41,9 @@ import org.apache.iceberg.spark.SparkReadOptions; import org.apache.iceberg.spark.SparkTableCache; import org.apache.iceberg.spark.SparkTableUtil; +import org.apache.iceberg.spark.SparkValueConverter; import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.spark.sql.Column; import org.apache.spark.sql.Dataset; @@ -125,10 +128,11 @@ private Dataset dataFiles(Types.StructType partitionType, StructLike partit IntStream.range(0, fields.size()) .mapToObj( i -> { - Class type = fields.get(i).type().typeId().javaClass(); - Object value = partition.get(i, type); - Column col = col("partition." + fields.get(i).name()); - return col.equalTo(value); + Type type = fields.get(i).type(); + Object value = partition.get(i, type.typeId().javaClass()); + Object convertedValue = SparkValueConverter.convertToSpark(type, value); + Column col = col("partition.`" + fields.get(i).name() + "`"); + return col.eqNullSafe(lit(convertedValue)); }) .reduce(Column::and); if (condition.isPresent()) { diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingDataRewriter.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingDataRewriter.java index 1ee469b09072..c9c962526eb3 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingDataRewriter.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingDataRewriter.java @@ -36,6 +36,8 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; +import org.apache.spark.sql.catalyst.plans.logical.OrderAwareCoalesce; +import org.apache.spark.sql.catalyst.plans.logical.OrderAwareCoalescer; import org.apache.spark.sql.connector.distributions.Distribution; import org.apache.spark.sql.connector.distributions.Distributions; import org.apache.spark.sql.connector.distributions.OrderedDistribution; @@ -59,7 +61,24 @@ abstract class SparkShufflingDataRewriter extends SparkSizeBasedDataRewriter { public static final double COMPRESSION_FACTOR_DEFAULT = 1.0; + /** + * The number of shuffle partitions to use for each output file. By default, this file rewriter + * assumes each shuffle partition would become a separate output file. Attempting to generate + * large output files of 512 MB or higher may strain the memory resources of the cluster as such + * rewrites would require lots of Spark memory. This parameter can be used to further divide up + * the data which will end up in a single file. For example, if the target file size is 2 GB, but + * the cluster can only handle shuffles of 512 MB, this parameter could be set to 4. Iceberg will + * use a custom coalesce operation to stitch these sorted partitions back together into a single + * sorted file. + * + *

    Note using this parameter requires enabling Iceberg Spark session extensions. + */ + public static final String SHUFFLE_PARTITIONS_PER_FILE = "shuffle-partitions-per-file"; + + public static final int SHUFFLE_PARTITIONS_PER_FILE_DEFAULT = 1; + private double compressionFactor; + private int numShufflePartitionsPerFile; protected SparkShufflingDataRewriter(SparkSession spark, Table table) { super(spark, table); @@ -75,6 +94,7 @@ public Set validOptions() { return ImmutableSet.builder() .addAll(super.validOptions()) .add(COMPRESSION_FACTOR) + .add(SHUFFLE_PARTITIONS_PER_FILE) .build(); } @@ -82,6 +102,7 @@ public Set validOptions() { public void init(Map options) { super.init(options); this.compressionFactor = compressionFactor(options); + this.numShufflePartitionsPerFile = numShufflePartitionsPerFile(options); } @Override @@ -114,7 +135,16 @@ private Function, Dataset> sortFunction(List gro private LogicalPlan sortPlan(LogicalPlan plan, SortOrder[] ordering, int numShufflePartitions) { SparkFunctionCatalog catalog = SparkFunctionCatalog.get(); OrderedWrite write = new OrderedWrite(ordering, numShufflePartitions); - return DistributionAndOrderingUtils$.MODULE$.prepareQuery(write, plan, Option.apply(catalog)); + LogicalPlan sortPlan = + DistributionAndOrderingUtils$.MODULE$.prepareQuery(write, plan, Option.apply(catalog)); + + if (numShufflePartitionsPerFile == 1) { + return sortPlan; + } else { + OrderAwareCoalescer coalescer = new OrderAwareCoalescer(numShufflePartitionsPerFile); + int numOutputPartitions = numShufflePartitions / numShufflePartitionsPerFile; + return new OrderAwareCoalesce(numOutputPartitions, coalescer, sortPlan); + } } private Dataset transformPlan(Dataset df, Function func) { @@ -134,7 +164,7 @@ private org.apache.iceberg.SortOrder outputSortOrder(List group) { private int numShufflePartitions(List group) { int numOutputFiles = (int) numOutputFiles((long) (inputSize(group) * compressionFactor)); - return Math.max(1, numOutputFiles); + return Math.max(1, numOutputFiles * numShufflePartitionsPerFile); } private double compressionFactor(Map options) { @@ -145,6 +175,19 @@ private double compressionFactor(Map options) { return value; } + private int numShufflePartitionsPerFile(Map options) { + int value = + PropertyUtil.propertyAsInt( + options, SHUFFLE_PARTITIONS_PER_FILE, SHUFFLE_PARTITIONS_PER_FILE_DEFAULT); + Preconditions.checkArgument( + value > 0, "'%s' is set to %s but must be > 0", SHUFFLE_PARTITIONS_PER_FILE, value); + Preconditions.checkArgument( + value == 1 || Spark3Util.extensionsEnabled(spark()), + "Using '%s' requires enabling Iceberg Spark session extensions", + SHUFFLE_PARTITIONS_PER_FILE); + return value; + } + private static class OrderedWrite implements RequiresDistributionAndOrdering { private final OrderedDistribution distribution; private final SortOrder[] ordering; diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java index 85043d2df3d6..259254aa2d51 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java @@ -21,11 +21,14 @@ import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.function.Predicate; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.ChangelogIterator; import org.apache.iceberg.spark.source.SparkChangelogTable; import org.apache.spark.api.java.function.MapPartitionsFunction; @@ -88,10 +91,22 @@ public class CreateChangelogViewProcedure extends BaseProcedure { ProcedureParameter.optional("options", STRING_MAP); private static final ProcedureParameter COMPUTE_UPDATES_PARAM = ProcedureParameter.optional("compute_updates", DataTypes.BooleanType); + + /** + * Enable or disable the remove carry-over rows. + * + * @deprecated since 1.4.0, will be removed in 1.5.0; The procedure will always remove carry-over + * rows. Please query {@link SparkChangelogTable} instead for the use cases doesn't remove + * carry-over rows. + */ + @Deprecated private static final ProcedureParameter REMOVE_CARRYOVERS_PARAM = ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType); + private static final ProcedureParameter IDENTIFIER_COLUMNS_PARAM = ProcedureParameter.optional("identifier_columns", STRING_ARRAY); + private static final ProcedureParameter NET_CHANGES = + ProcedureParameter.optional("net_changes", DataTypes.BooleanType); private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[] { @@ -101,6 +116,7 @@ public class CreateChangelogViewProcedure extends BaseProcedure { COMPUTE_UPDATES_PARAM, REMOVE_CARRYOVERS_PARAM, IDENTIFIER_COLUMNS_PARAM, + NET_CHANGES, }; private static final StructType OUTPUT_TYPE = @@ -142,10 +158,13 @@ public InternalRow[] call(InternalRow args) { Identifier changelogTableIdent = changelogTableIdent(tableIdent); Dataset df = loadRows(changelogTableIdent, options(input)); + boolean netChanges = input.asBoolean(NET_CHANGES, false); + if (shouldComputeUpdateImages(input)) { + Preconditions.checkArgument(!netChanges, "Not support net changes with update images"); df = computeUpdateImages(identifierColumns(input, tableIdent), df); } else if (shouldRemoveCarryoverRows(input)) { - df = removeCarryoverRows(df); + df = removeCarryoverRows(df, netChanges); } String viewName = viewName(input, tableIdent.name()); @@ -164,6 +183,7 @@ private Dataset computeUpdateImages(String[] identifierColumns, Dataset removeCarryoverRows(Dataset df) { + private Dataset removeCarryoverRows(Dataset df, boolean netChanges) { + Predicate columnsToKeep; + if (netChanges) { + Set metadataColumn = + Sets.newHashSet( + MetadataColumns.CHANGE_TYPE.name(), + MetadataColumns.CHANGE_ORDINAL.name(), + MetadataColumns.COMMIT_SNAPSHOT_ID.name()); + + columnsToKeep = column -> !metadataColumn.contains(column); + } else { + columnsToKeep = column -> !column.equals(MetadataColumns.CHANGE_TYPE.name()); + } + Column[] repartitionSpec = - Arrays.stream(df.columns()) - .filter(c -> !c.equals(MetadataColumns.CHANGE_TYPE.name())) - .map(df::col) - .toArray(Column[]::new); - return applyCarryoverRemoveIterator(df, repartitionSpec); + Arrays.stream(df.columns()).filter(columnsToKeep).map(df::col).toArray(Column[]::new); + return applyCarryoverRemoveIterator(df, repartitionSpec, netChanges); } private String[] identifierColumns(ProcedureInput input, Identifier tableIdent) { @@ -214,7 +244,7 @@ private String viewName(ProcedureInput input, String tableName) { } private Dataset applyChangelogIterator(Dataset df, Column[] repartitionSpec) { - Column[] sortSpec = sortSpec(df, repartitionSpec); + Column[] sortSpec = sortSpec(df, repartitionSpec, false); StructType schema = df.schema(); String[] identifierFields = Arrays.stream(repartitionSpec).map(Column::toString).toArray(String[]::new); @@ -228,22 +258,33 @@ private Dataset applyChangelogIterator(Dataset df, Column[] repartitio RowEncoder.apply(schema)); } - private Dataset applyCarryoverRemoveIterator(Dataset df, Column[] repartitionSpec) { - Column[] sortSpec = sortSpec(df, repartitionSpec); + private Dataset applyCarryoverRemoveIterator( + Dataset df, Column[] repartitionSpec, boolean netChanges) { + Column[] sortSpec = sortSpec(df, repartitionSpec, netChanges); StructType schema = df.schema(); return df.repartition(repartitionSpec) .sortWithinPartitions(sortSpec) .mapPartitions( (MapPartitionsFunction) - rowIterator -> ChangelogIterator.removeCarryovers(rowIterator, schema), + rowIterator -> + netChanges + ? ChangelogIterator.removeNetCarryovers(rowIterator, schema) + : ChangelogIterator.removeCarryovers(rowIterator, schema), RowEncoder.apply(schema)); } - private static Column[] sortSpec(Dataset df, Column[] repartitionSpec) { - Column[] sortSpec = new Column[repartitionSpec.length + 1]; + private static Column[] sortSpec(Dataset df, Column[] repartitionSpec, boolean netChanges) { + Column changeType = df.col(MetadataColumns.CHANGE_TYPE.name()); + Column changeOrdinal = df.col(MetadataColumns.CHANGE_ORDINAL.name()); + Column[] extraColumns = + netChanges ? new Column[] {changeOrdinal, changeType} : new Column[] {changeType}; + + Column[] sortSpec = new Column[repartitionSpec.length + extraColumns.length]; + System.arraycopy(repartitionSpec, 0, sortSpec, 0, repartitionSpec.length); - sortSpec[sortSpec.length - 1] = df.col(MetadataColumns.CHANGE_TYPE.name()); + System.arraycopy(extraColumns, 0, sortSpec, repartitionSpec.length, extraColumns.length); + return sortSpec; } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java index dd493fbc5097..036c395f7115 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java @@ -22,6 +22,7 @@ import java.util.Map; import java.util.Objects; import java.util.Set; +import java.util.function.Supplier; import java.util.stream.Collectors; import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionScanTask; @@ -39,6 +40,7 @@ import org.apache.iceberg.expressions.ExpressionUtil; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.expressions.Projections; +import org.apache.iceberg.metrics.ScanReport; 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; @@ -73,9 +75,9 @@ class SparkBatchQueryScan extends SparkPartitioningAwareScan Scan> scan, SparkReadConf readConf, Schema expectedSchema, - List filters) { - - super(spark, table, scan, readConf, expectedSchema, filters); + List filters, + Supplier scanReportSupplier) { + super(spark, table, scan, readConf, expectedSchema, filters, scanReportSupplier); this.snapshotId = readConf.snapshotId(); this.startSnapshotId = readConf.startSnapshotId(); diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java index d978b81e67bd..16eb9c51df5c 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java @@ -22,6 +22,7 @@ import java.util.List; import java.util.Objects; import java.util.Set; +import java.util.function.Supplier; import java.util.stream.Collectors; import org.apache.iceberg.BatchScan; import org.apache.iceberg.FileScanTask; @@ -30,6 +31,7 @@ import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.metrics.ScanReport; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.SparkReadConf; @@ -57,8 +59,9 @@ class SparkCopyOnWriteScan extends SparkPartitioningAwareScan Table table, SparkReadConf readConf, Schema expectedSchema, - List filters) { - this(spark, table, null, null, readConf, expectedSchema, filters); + List filters, + Supplier scanReportSupplier) { + this(spark, table, null, null, readConf, expectedSchema, filters, scanReportSupplier); } SparkCopyOnWriteScan( @@ -68,9 +71,9 @@ class SparkCopyOnWriteScan extends SparkPartitioningAwareScan Snapshot snapshot, SparkReadConf readConf, Schema expectedSchema, - List filters) { - - super(spark, table, scan, readConf, expectedSchema, filters); + List filters, + Supplier scanReportSupplier) { + super(spark, table, scan, readConf, expectedSchema, filters, scanReportSupplier); this.snapshot = snapshot; diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java index cf274f794e52..141dd4dcba0e 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java @@ -24,6 +24,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.IntStream; import org.apache.iceberg.PartitionField; @@ -37,6 +38,7 @@ import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.metrics.ScanReport; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.spark.Spark3Util; @@ -74,9 +76,9 @@ abstract class SparkPartitioningAwareScan extends S Scan> scan, SparkReadConf readConf, Schema expectedSchema, - List filters) { - - super(spark, table, readConf, expectedSchema, filters); + List filters, + Supplier scanReportSupplier) { + super(spark, table, readConf, expectedSchema, filters, scanReportSupplier); this.scan = scan; this.preserveDataGrouping = readConf.preserveDataGrouping(); @@ -198,7 +200,7 @@ protected synchronized List> taskGroups() { CloseableIterable> plannedTaskGroups = TableScanUtil.planTaskGroups( CloseableIterable.withNoopClose(tasks()), - scan.targetSplitSize(), + adjustSplitSize(tasks(), scan.targetSplitSize()), scan.splitLookback(), scan.splitOpenFileCost()); this.taskGroups = Lists.newArrayList(plannedTaskGroups); @@ -212,7 +214,7 @@ protected synchronized List> taskGroups() { List> plannedTaskGroups = TableScanUtil.planTaskGroups( tasks(), - scan.targetSplitSize(), + adjustSplitSize(tasks(), scan.targetSplitSize()), scan.splitLookback(), scan.splitOpenFileCost(), groupingKeyType()); diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java index b47818ec550d..535d43853f3b 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java @@ -21,24 +21,42 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.function.Supplier; import java.util.stream.Collectors; +import org.apache.iceberg.ScanTask; import org.apache.iceberg.ScanTaskGroup; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.Table; import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkReadConf; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.spark.source.metrics.NumDeletes; import org.apache.iceberg.spark.source.metrics.NumSplits; +import org.apache.iceberg.spark.source.metrics.ScannedDataFiles; +import org.apache.iceberg.spark.source.metrics.ScannedDataManifests; +import org.apache.iceberg.spark.source.metrics.SkippedDataFiles; +import org.apache.iceberg.spark.source.metrics.SkippedDataManifests; +import org.apache.iceberg.spark.source.metrics.TaskScannedDataFiles; +import org.apache.iceberg.spark.source.metrics.TaskScannedDataManifests; +import org.apache.iceberg.spark.source.metrics.TaskSkippedDataFiles; +import org.apache.iceberg.spark.source.metrics.TaskSkippedDataManifests; +import org.apache.iceberg.spark.source.metrics.TaskTotalFileSize; +import org.apache.iceberg.spark.source.metrics.TaskTotalPlanningDuration; +import org.apache.iceberg.spark.source.metrics.TotalFileSize; +import org.apache.iceberg.spark.source.metrics.TotalPlanningDuration; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.SnapshotUtil; +import org.apache.iceberg.util.TableScanUtil; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.connector.metric.CustomMetric; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; import org.apache.spark.sql.connector.read.Batch; import org.apache.spark.sql.connector.read.Scan; import org.apache.spark.sql.connector.read.Statistics; @@ -58,6 +76,7 @@ abstract class SparkScan implements Scan, SupportsReportStatistics { private final Schema expectedSchema; private final List filterExpressions; private final String branch; + private final Supplier scanReportSupplier; // lazy variables private StructType readSchema; @@ -67,7 +86,8 @@ abstract class SparkScan implements Scan, SupportsReportStatistics { Table table, SparkReadConf readConf, Schema expectedSchema, - List filters) { + List filters, + Supplier scanReportSupplier) { Schema snapshotSchema = SnapshotUtil.schemaFor(table, readConf.branch()); SparkSchemaUtil.validateMetadataColumnReferences(snapshotSchema, expectedSchema); @@ -78,6 +98,7 @@ abstract class SparkScan implements Scan, SupportsReportStatistics { this.expectedSchema = expectedSchema; this.filterExpressions = filters != null ? filters : Collections.emptyList(); this.branch = readConf.branch(); + this.scanReportSupplier = scanReportSupplier; } protected Table table() { @@ -170,8 +191,46 @@ public String description() { table(), branch(), Spark3Util.describe(filterExpressions), groupingKeyFieldNamesAsString); } + @Override + public CustomTaskMetric[] reportDriverMetrics() { + ScanReport scanReport = scanReportSupplier != null ? scanReportSupplier.get() : null; + + if (scanReport == null) { + return new CustomTaskMetric[0]; + } + + List driverMetrics = Lists.newArrayList(); + driverMetrics.add(TaskTotalFileSize.from(scanReport)); + driverMetrics.add(TaskTotalPlanningDuration.from(scanReport)); + driverMetrics.add(TaskSkippedDataFiles.from(scanReport)); + driverMetrics.add(TaskScannedDataFiles.from(scanReport)); + driverMetrics.add(TaskSkippedDataManifests.from(scanReport)); + driverMetrics.add(TaskScannedDataManifests.from(scanReport)); + + return driverMetrics.toArray(new CustomTaskMetric[0]); + } + @Override public CustomMetric[] supportedCustomMetrics() { - return new CustomMetric[] {new NumSplits(), new NumDeletes()}; + return new CustomMetric[] { + new NumSplits(), + new NumDeletes(), + new TotalFileSize(), + new TotalPlanningDuration(), + new ScannedDataManifests(), + new SkippedDataManifests(), + new ScannedDataFiles(), + new SkippedDataFiles() + }; + } + + protected long adjustSplitSize(List tasks, long splitSize) { + if (readConf.splitSizeOption() == null && readConf.adaptiveSplitSizeEnabled()) { + long scanSize = tasks.stream().mapToLong(ScanTask::sizeBytes).sum(); + int parallelism = sparkContext.defaultParallelism(); + return TableScanUtil.adjustSplitSize(scanSize, parallelism, splitSize); + } else { + return splitSize; + } } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java index ddeec9c4943b..8cbb271382a8 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java @@ -30,6 +30,7 @@ import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.MetricsConfig; import org.apache.iceberg.MetricsModes; +import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; import org.apache.iceberg.StructLike; @@ -43,6 +44,7 @@ import org.apache.iceberg.expressions.ExpressionUtil; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.metrics.InMemoryMetricsReporter; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -91,6 +93,7 @@ public class SparkScanBuilder private final CaseInsensitiveStringMap options; private final SparkReadConf readConf; private final List metaColumns = Lists.newArrayList(); + private final InMemoryMetricsReporter metricsReporter; private Schema schema = null; private boolean caseSensitive; @@ -109,6 +112,7 @@ public class SparkScanBuilder this.options = options; this.readConf = new SparkReadConf(spark, table, branch, options); this.caseSensitive = readConf.caseSensitive(); + this.metricsReporter = new InMemoryMetricsReporter(); } SparkScanBuilder(SparkSession spark, Table table, CaseInsensitiveStringMap options) { @@ -164,7 +168,9 @@ public Filter[] pushFilters(Filter[] filters) { pushableFilters.add(filter); } - if (expr == null || !ExpressionUtil.selectsPartitions(expr, table, caseSensitive)) { + if (expr == null + || unpartitioned() + || !ExpressionUtil.selectsPartitions(expr, table, caseSensitive)) { postScanFilters.add(filter); } else { LOG.info("Evaluating completely on Iceberg side: {}", filter); @@ -182,6 +188,10 @@ public Filter[] pushFilters(Filter[] filters) { return postScanFilters.toArray(new Filter[0]); } + private boolean unpartitioned() { + return table.specs().values().stream().noneMatch(PartitionSpec::isPartitioned); + } + @Override public Filter[] pushedFilters() { return pushedFilters; @@ -430,7 +440,8 @@ private Scan buildBatchScan(Long snapshotId, Long asOfTimestamp, String branch, .newBatchScan() .caseSensitive(caseSensitive) .filter(filterExpression()) - .project(expectedSchema); + .project(expectedSchema) + .metricsReporter(metricsReporter); if (snapshotId != null) { scan = scan.useSnapshot(snapshotId); @@ -450,7 +461,14 @@ private Scan buildBatchScan(Long snapshotId, Long asOfTimestamp, String branch, scan = configureSplitPlanning(scan); - return new SparkBatchQueryScan(spark, table, scan, readConf, expectedSchema, filterExpressions); + return new SparkBatchQueryScan( + spark, + table, + scan, + readConf, + expectedSchema, + filterExpressions, + metricsReporter::scanReport); } private Scan buildIncrementalAppendScan(long startSnapshotId, Long endSnapshotId) { @@ -470,7 +488,14 @@ private Scan buildIncrementalAppendScan(long startSnapshotId, Long endSnapshotId scan = configureSplitPlanning(scan); - return new SparkBatchQueryScan(spark, table, scan, readConf, expectedSchema, filterExpressions); + return new SparkBatchQueryScan( + spark, + table, + scan, + readConf, + expectedSchema, + filterExpressions, + metricsReporter::scanReport); } public Scan buildChangelogScan() { @@ -573,7 +598,13 @@ public Scan buildMergeOnReadScan() { if (snapshot == null) { return new SparkBatchQueryScan( - spark, table, null, readConf, schemaWithMetadataColumns(), filterExpressions); + spark, + table, + null, + readConf, + schemaWithMetadataColumns(), + filterExpressions, + metricsReporter::scanReport); } // remember the current snapshot ID for commit validation @@ -597,7 +628,13 @@ public Scan buildMergeOnReadScan() { scan = configureSplitPlanning(scan); return new SparkBatchQueryScan( - spark, table, scan, adjustedReadConf, expectedSchema, filterExpressions); + spark, + table, + scan, + adjustedReadConf, + expectedSchema, + filterExpressions, + metricsReporter::scanReport); } public Scan buildCopyOnWriteScan() { @@ -605,7 +642,12 @@ public Scan buildCopyOnWriteScan() { if (snapshot == null) { return new SparkCopyOnWriteScan( - spark, table, readConf, schemaWithMetadataColumns(), filterExpressions); + spark, + table, + readConf, + schemaWithMetadataColumns(), + filterExpressions, + metricsReporter::scanReport); } Schema expectedSchema = schemaWithMetadataColumns(); @@ -622,7 +664,14 @@ public Scan buildCopyOnWriteScan() { scan = configureSplitPlanning(scan); return new SparkCopyOnWriteScan( - spark, table, scan, snapshot, readConf, expectedSchema, filterExpressions); + spark, + table, + scan, + snapshot, + readConf, + expectedSchema, + filterExpressions, + metricsReporter::scanReport); } private > T configureSplitPlanning(T scan) { @@ -651,7 +700,7 @@ public Scan buildCopyOnWriteScan() { @Override public Statistics estimateStatistics() { - return ((SparkScan) build()).estimateStatistics(); + return ((SupportsReportStatistics) build()).estimateStatistics(); } @Override diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScan.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScan.java index 89b184c91c51..0290bf7e84ce 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScan.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScan.java @@ -40,7 +40,7 @@ class SparkStagedScan extends SparkScan { private List> taskGroups = null; // lazy cache of tasks SparkStagedScan(SparkSession spark, Table table, SparkReadConf readConf) { - super(spark, table, readConf, table.schema(), ImmutableList.of()); + super(spark, table, readConf, table.schema(), ImmutableList.of(), null); this.taskSetId = readConf.scanTaskSetId(); this.splitSize = readConf.splitSize(); diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java index 240a9df0a88c..0ad33aaadc45 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java @@ -57,6 +57,7 @@ import org.apache.iceberg.spark.SparkFilters; import org.apache.iceberg.spark.SparkReadOptions; import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.SparkTableUtil; import org.apache.iceberg.spark.SparkUtil; import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.SnapshotUtil; @@ -372,6 +373,10 @@ public void deleteWhere(Filter[] filters) { .set("spark.app.id", sparkSession().sparkContext().applicationId()) .deleteFromRowFilter(deleteExpr); + if (SparkTableUtil.wapEnabled(table())) { + branch = SparkTableUtil.determineWriteBranch(sparkSession(), branch); + } + if (branch != null) { deleteFiles.toBranch(branch); } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataFiles.java new file mode 100644 index 000000000000..f453872fdc29 --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataFiles.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source.metrics; + +import org.apache.spark.sql.connector.metric.CustomSumMetric; + +public class ScannedDataFiles extends CustomSumMetric { + + static final String NAME = "scannedDataFiles"; + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "number of scanned data files"; + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java new file mode 100644 index 000000000000..a167904280e6 --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source.metrics; + +import org.apache.spark.sql.connector.metric.CustomSumMetric; + +public class ScannedDataManifests extends CustomSumMetric { + + static final String NAME = "scannedDataManifests"; + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "number of scanned data manifests"; + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataFiles.java new file mode 100644 index 000000000000..7fd17425313d --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataFiles.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source.metrics; + +import org.apache.spark.sql.connector.metric.CustomSumMetric; + +public class SkippedDataFiles extends CustomSumMetric { + + static final String NAME = "skippedDataFiles"; + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "number of skipped data files"; + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataManifests.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataManifests.java new file mode 100644 index 000000000000..b0eaeb5d87f2 --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataManifests.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source.metrics; + +import org.apache.spark.sql.connector.metric.CustomSumMetric; + +public class SkippedDataManifests extends CustomSumMetric { + + static final String NAME = "skippedDataManifests"; + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "number of skipped data manifests"; + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataFiles.java new file mode 100644 index 000000000000..d9a527da08f6 --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataFiles.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source.metrics; + +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskScannedDataFiles implements CustomTaskMetric { + private final long value; + + private TaskScannedDataFiles(long value) { + this.value = value; + } + + @Override + public String name() { + return ScannedDataFiles.NAME; + } + + @Override + public long value() { + return value; + } + + public static TaskScannedDataFiles from(ScanReport scanReport) { + CounterResult counter = scanReport.scanMetrics().resultDataFiles(); + long value = counter != null ? counter.value() : 0L; + return new TaskScannedDataFiles(value); + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataManifests.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataManifests.java new file mode 100644 index 000000000000..09dd0339910c --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataManifests.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source.metrics; + +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskScannedDataManifests implements CustomTaskMetric { + private final long value; + + private TaskScannedDataManifests(long value) { + this.value = value; + } + + @Override + public String name() { + return ScannedDataManifests.NAME; + } + + @Override + public long value() { + return value; + } + + public static TaskScannedDataManifests from(ScanReport scanReport) { + CounterResult counter = scanReport.scanMetrics().scannedDataManifests(); + long value = counter != null ? counter.value() : 0L; + return new TaskScannedDataManifests(value); + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataFiles.java new file mode 100644 index 000000000000..5165f9a3116c --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataFiles.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source.metrics; + +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskSkippedDataFiles implements CustomTaskMetric { + private final long value; + + private TaskSkippedDataFiles(long value) { + this.value = value; + } + + @Override + public String name() { + return SkippedDataFiles.NAME; + } + + @Override + public long value() { + return value; + } + + public static TaskSkippedDataFiles from(ScanReport scanReport) { + CounterResult counter = scanReport.scanMetrics().skippedDataFiles(); + long value = counter != null ? counter.value() : 0L; + return new TaskSkippedDataFiles(value); + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataManifests.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataManifests.java new file mode 100644 index 000000000000..86fef8c4118b --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataManifests.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source.metrics; + +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskSkippedDataManifests implements CustomTaskMetric { + private final long value; + + private TaskSkippedDataManifests(long value) { + this.value = value; + } + + @Override + public String name() { + return SkippedDataManifests.NAME; + } + + @Override + public long value() { + return value; + } + + public static TaskSkippedDataManifests from(ScanReport scanReport) { + CounterResult counter = scanReport.scanMetrics().skippedDataManifests(); + long value = counter != null ? counter.value() : 0L; + return new TaskSkippedDataManifests(value); + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalFileSize.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalFileSize.java new file mode 100644 index 000000000000..c300d835e777 --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalFileSize.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source.metrics; + +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskTotalFileSize implements CustomTaskMetric { + + private final long value; + + private TaskTotalFileSize(long value) { + this.value = value; + } + + @Override + public String name() { + return TotalFileSize.NAME; + } + + @Override + public long value() { + return value; + } + + public static TaskTotalFileSize from(ScanReport scanReport) { + CounterResult counter = scanReport.scanMetrics().totalFileSizeInBytes(); + long value = counter != null ? counter.value() : 0L; + return new TaskTotalFileSize(value); + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalPlanningDuration.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalPlanningDuration.java new file mode 100644 index 000000000000..32ac6fde8bf3 --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalPlanningDuration.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source.metrics; + +import org.apache.iceberg.metrics.ScanReport; +import org.apache.iceberg.metrics.TimerResult; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskTotalPlanningDuration implements CustomTaskMetric { + + private final long value; + + private TaskTotalPlanningDuration(long value) { + this.value = value; + } + + @Override + public String name() { + return TotalPlanningDuration.NAME; + } + + @Override + public long value() { + return value; + } + + public static TaskTotalPlanningDuration from(ScanReport scanReport) { + TimerResult timerResult = scanReport.scanMetrics().totalPlanningDuration(); + long value = timerResult != null ? timerResult.totalDuration().toMillis() : -1; + return new TaskTotalPlanningDuration(value); + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalFileSize.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalFileSize.java new file mode 100644 index 000000000000..994626e54f10 --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalFileSize.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source.metrics; + +import org.apache.spark.sql.connector.metric.CustomSumMetric; + +public class TotalFileSize extends CustomSumMetric { + + static final String NAME = "totalFileSize"; + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "total file size (bytes)"; + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java new file mode 100644 index 000000000000..8b66eeac4046 --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source.metrics; + +import org.apache.spark.sql.connector.metric.CustomSumMetric; + +public class TotalPlanningDuration extends CustomSumMetric { + + static final String NAME = "totalPlanningDuration"; + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "total planning duration (ms)"; + } +} diff --git a/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/OrderAwareCoalesce.scala b/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/OrderAwareCoalesce.scala new file mode 100644 index 000000000000..5acaa6800e68 --- /dev/null +++ b/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/OrderAwareCoalesce.scala @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.catalyst.plans.logical + +import org.apache.spark.rdd.PartitionCoalescer +import org.apache.spark.rdd.PartitionGroup +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.Attribute + +// this node doesn't extend RepartitionOperation on purpose to keep this logic isolated +// and ignore it in optimizer rules such as CollapseRepartition +case class OrderAwareCoalesce( + numPartitions: Int, + coalescer: PartitionCoalescer, + child: LogicalPlan) extends OrderPreservingUnaryNode { + + override def output: Seq[Attribute] = child.output + + override protected def withNewChildInternal(newChild: LogicalPlan): LogicalPlan = { + copy(child = newChild) + } +} + +class OrderAwareCoalescer(val groupSize: Int) extends PartitionCoalescer with Serializable { + + override def coalesce(maxPartitions: Int, parent: RDD[_]): Array[PartitionGroup] = { + val partitionBins = parent.partitions.grouped(groupSize) + partitionBins.map { partitions => + val group = new PartitionGroup() + group.partitions ++= partitions + group + }.toArray + } +} diff --git a/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/execution/OrderAwareCoalesceExec.scala b/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/execution/OrderAwareCoalesceExec.scala new file mode 100644 index 000000000000..2ef99550524a --- /dev/null +++ b/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/execution/OrderAwareCoalesceExec.scala @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.execution + +import org.apache.spark.rdd.PartitionCoalescer +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.expressions.SortOrder +import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.catalyst.plans.physical.SinglePartition +import org.apache.spark.sql.catalyst.plans.physical.UnknownPartitioning + +case class OrderAwareCoalesceExec( + numPartitions: Int, + coalescer: PartitionCoalescer, + child: SparkPlan) extends UnaryExecNode { + + override def output: Seq[Attribute] = child.output + + override def outputOrdering: Seq[SortOrder] = child.outputOrdering + + override def outputPartitioning: Partitioning = { + if (numPartitions == 1) SinglePartition else UnknownPartitioning(numPartitions) + } + + protected override def doExecute(): RDD[InternalRow] = { + val result = child.execute() + if (numPartitions == 1 && result.getNumPartitions < 1) { + // make sure we don't output an RDD with 0 partitions, + // when claiming that we have a `SinglePartition` + // see CoalesceExec in Spark + new CoalesceExec.EmptyRDDWithPartitions(sparkContext, numPartitions) + } else { + result.coalesce(numPartitions, shuffle = false, Some(coalescer)) + } + } + + override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = { + copy(child = newChild) + } +} diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestChangelogIterator.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestChangelogIterator.java index bf98bebb9d50..0539598f147e 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestChangelogIterator.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestChangelogIterator.java @@ -49,7 +49,17 @@ public class TestChangelogIterator extends SparkTestHelperBase { new StructField("name", DataTypes.StringType, false, Metadata.empty()), new StructField("data", DataTypes.StringType, true, Metadata.empty()), new StructField( - MetadataColumns.CHANGE_TYPE.name(), DataTypes.StringType, false, Metadata.empty()) + MetadataColumns.CHANGE_TYPE.name(), DataTypes.StringType, false, Metadata.empty()), + new StructField( + MetadataColumns.CHANGE_ORDINAL.name(), + DataTypes.IntegerType, + false, + Metadata.empty()), + new StructField( + MetadataColumns.COMMIT_SNAPSHOT_ID.name(), + DataTypes.LongType, + false, + Metadata.empty()) }); private static final String[] IDENTIFIER_FIELDS = new String[] {"id", "name"}; @@ -93,18 +103,18 @@ private List toOriginalRows(RowType rowType, int index) { switch (rowType) { case DELETED: return Lists.newArrayList( - new GenericRowWithSchema(new Object[] {index, "b", "data", DELETE}, null)); + new GenericRowWithSchema(new Object[] {index, "b", "data", DELETE, 0, 0}, null)); case INSERTED: return Lists.newArrayList( - new GenericRowWithSchema(new Object[] {index, "c", "data", INSERT}, null)); + new GenericRowWithSchema(new Object[] {index, "c", "data", INSERT, 0, 0}, null)); case CARRY_OVER: return Lists.newArrayList( - new GenericRowWithSchema(new Object[] {index, "d", "data", DELETE}, null), - new GenericRowWithSchema(new Object[] {index, "d", "data", INSERT}, null)); + new GenericRowWithSchema(new Object[] {index, "d", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {index, "d", "data", INSERT, 0, 0}, null)); case UPDATED: return Lists.newArrayList( - new GenericRowWithSchema(new Object[] {index, "a", "data", DELETE}, null), - new GenericRowWithSchema(new Object[] {index, "a", "new_data", INSERT}, null)); + new GenericRowWithSchema(new Object[] {index, "a", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {index, "a", "new_data", INSERT, 0, 0}, null)); default: throw new IllegalArgumentException("Unknown row type: " + rowType); } @@ -114,18 +124,18 @@ private List toExpectedRows(RowType rowType, int order) { switch (rowType) { case DELETED: List rows = Lists.newArrayList(); - rows.add(new Object[] {order, "b", "data", DELETE}); + rows.add(new Object[] {order, "b", "data", DELETE, 0, 0}); return rows; case INSERTED: List insertedRows = Lists.newArrayList(); - insertedRows.add(new Object[] {order, "c", "data", INSERT}); + insertedRows.add(new Object[] {order, "c", "data", INSERT, 0, 0}); return insertedRows; case CARRY_OVER: return Lists.newArrayList(); case UPDATED: return Lists.newArrayList( - new Object[] {order, "a", "data", UPDATE_BEFORE}, - new Object[] {order, "a", "new_data", UPDATE_AFTER}); + new Object[] {order, "a", "data", UPDATE_BEFORE, 0, 0}, + new Object[] {order, "a", "new_data", UPDATE_AFTER, 0, 0}); default: throw new IllegalArgumentException("Unknown row type: " + rowType); } @@ -146,16 +156,16 @@ private void permute(List arr, int start, List pm) { public void testRowsWithNullValue() { final List rowsWithNull = Lists.newArrayList( - new GenericRowWithSchema(new Object[] {2, null, null, DELETE}, null), - new GenericRowWithSchema(new Object[] {3, null, null, INSERT}, null), - new GenericRowWithSchema(new Object[] {4, null, null, DELETE}, null), - new GenericRowWithSchema(new Object[] {4, null, null, INSERT}, null), + new GenericRowWithSchema(new Object[] {2, null, null, DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {3, null, null, INSERT, 0, 0}, null), + new GenericRowWithSchema(new Object[] {4, null, null, DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {4, null, null, INSERT, 0, 0}, null), // mixed null and non-null value in non-identifier columns - new GenericRowWithSchema(new Object[] {5, null, null, DELETE}, null), - new GenericRowWithSchema(new Object[] {5, null, "data", INSERT}, null), + new GenericRowWithSchema(new Object[] {5, null, null, DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {5, null, "data", INSERT, 0, 0}, null), // mixed null and non-null value in identifier columns - new GenericRowWithSchema(new Object[] {6, null, null, DELETE}, null), - new GenericRowWithSchema(new Object[] {6, "name", null, INSERT}, null)); + new GenericRowWithSchema(new Object[] {6, null, null, DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {6, "name", null, INSERT, 0, 0}, null)); Iterator iterator = ChangelogIterator.computeUpdates(rowsWithNull.iterator(), SCHEMA, IDENTIFIER_FIELDS); @@ -164,12 +174,12 @@ public void testRowsWithNullValue() { assertEquals( "Rows should match", Lists.newArrayList( - new Object[] {2, null, null, DELETE}, - new Object[] {3, null, null, INSERT}, - new Object[] {5, null, null, UPDATE_BEFORE}, - new Object[] {5, null, "data", UPDATE_AFTER}, - new Object[] {6, null, null, DELETE}, - new Object[] {6, "name", null, INSERT}), + new Object[] {2, null, null, DELETE, 0, 0}, + new Object[] {3, null, null, INSERT, 0, 0}, + new Object[] {5, null, null, UPDATE_BEFORE, 0, 0}, + new Object[] {5, null, "data", UPDATE_AFTER, 0, 0}, + new Object[] {6, null, null, DELETE, 0, 0}, + new Object[] {6, "name", null, INSERT, 0, 0}), rowsToJava(result)); } @@ -178,10 +188,10 @@ public void testUpdatedRowsWithDuplication() { List rowsWithDuplication = Lists.newArrayList( // two rows with same identifier fields(id, name) - new GenericRowWithSchema(new Object[] {1, "a", "data", DELETE}, null), - new GenericRowWithSchema(new Object[] {1, "a", "data", DELETE}, null), - new GenericRowWithSchema(new Object[] {1, "a", "new_data", INSERT}, null), - new GenericRowWithSchema(new Object[] {1, "a", "new_data", INSERT}, null)); + new GenericRowWithSchema(new Object[] {1, "a", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "a", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "a", "new_data", INSERT, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "a", "new_data", INSERT, 0, 0}, null)); Iterator iterator = ChangelogIterator.computeUpdates(rowsWithDuplication.iterator(), SCHEMA, IDENTIFIER_FIELDS); @@ -194,9 +204,9 @@ public void testUpdatedRowsWithDuplication() { // still allow extra insert rows rowsWithDuplication = Lists.newArrayList( - new GenericRowWithSchema(new Object[] {1, "a", "data", DELETE}, null), - new GenericRowWithSchema(new Object[] {1, "a", "new_data1", INSERT}, null), - new GenericRowWithSchema(new Object[] {1, "a", "new_data2", INSERT}, null)); + new GenericRowWithSchema(new Object[] {1, "a", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "a", "new_data1", INSERT, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "a", "new_data2", INSERT, 0, 0}, null)); Iterator iterator1 = ChangelogIterator.computeUpdates(rowsWithDuplication.iterator(), SCHEMA, IDENTIFIER_FIELDS); @@ -204,9 +214,9 @@ public void testUpdatedRowsWithDuplication() { assertEquals( "Rows should match.", Lists.newArrayList( - new Object[] {1, "a", "data", UPDATE_BEFORE}, - new Object[] {1, "a", "new_data1", UPDATE_AFTER}, - new Object[] {1, "a", "new_data2", INSERT}), + new Object[] {1, "a", "data", UPDATE_BEFORE, 0, 0}, + new Object[] {1, "a", "new_data1", UPDATE_AFTER, 0, 0}, + new Object[] {1, "a", "new_data2", INSERT, 0, 0}), rowsToJava(Lists.newArrayList(iterator1))); } @@ -216,32 +226,28 @@ public void testCarryRowsRemoveWithDuplicates() { List rowsWithDuplication = Lists.newArrayList( // keep all delete rows for id 0 and id 1 since there is no insert row for them - new GenericRowWithSchema(new Object[] {0, "a", "data", DELETE}, null), - new GenericRowWithSchema(new Object[] {0, "a", "data", DELETE}, null), - new GenericRowWithSchema(new Object[] {0, "a", "data", DELETE}, null), - new GenericRowWithSchema(new Object[] {1, "a", "old_data", DELETE}, null), - new GenericRowWithSchema(new Object[] {1, "a", "old_data", DELETE}, null), + new GenericRowWithSchema(new Object[] {0, "a", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {0, "a", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {0, "a", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "a", "old_data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "a", "old_data", DELETE, 0, 0}, null), // the same number of delete and insert rows for id 2 - new GenericRowWithSchema(new Object[] {2, "a", "data", DELETE}, null), - new GenericRowWithSchema(new Object[] {2, "a", "data", DELETE}, null), - new GenericRowWithSchema(new Object[] {2, "a", "data", INSERT}, null), - new GenericRowWithSchema(new Object[] {2, "a", "data", INSERT}, null), - new GenericRowWithSchema(new Object[] {3, "a", "new_data", INSERT}, null)); - - Iterator iterator = - ChangelogIterator.removeCarryovers(rowsWithDuplication.iterator(), SCHEMA); - List result = Lists.newArrayList(iterator); + new GenericRowWithSchema(new Object[] {2, "a", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {2, "a", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {2, "a", "data", INSERT, 0, 0}, null), + new GenericRowWithSchema(new Object[] {2, "a", "data", INSERT, 0, 0}, null), + new GenericRowWithSchema(new Object[] {3, "a", "new_data", INSERT, 0, 0}, null)); - assertEquals( - "Rows should match.", + List expectedRows = Lists.newArrayList( - new Object[] {0, "a", "data", DELETE}, - new Object[] {0, "a", "data", DELETE}, - new Object[] {0, "a", "data", DELETE}, - new Object[] {1, "a", "old_data", DELETE}, - new Object[] {1, "a", "old_data", DELETE}, - new Object[] {3, "a", "new_data", INSERT}), - rowsToJava(result)); + new Object[] {0, "a", "data", DELETE, 0, 0}, + new Object[] {0, "a", "data", DELETE, 0, 0}, + new Object[] {0, "a", "data", DELETE, 0, 0}, + new Object[] {1, "a", "old_data", DELETE, 0, 0}, + new Object[] {1, "a", "old_data", DELETE, 0, 0}, + new Object[] {3, "a", "new_data", INSERT, 0, 0}); + + validateIterators(rowsWithDuplication, expectedRows); } @Test @@ -249,45 +255,39 @@ public void testCarryRowsRemoveLessInsertRows() { // less insert rows than delete rows List rowsWithDuplication = Lists.newArrayList( - new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE}, null), - new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE}, null), - new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT}, null), - new GenericRowWithSchema(new Object[] {2, "d", "data", INSERT}, null)); + new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT, 0, 0}, null), + new GenericRowWithSchema(new Object[] {2, "d", "data", INSERT, 0, 0}, null)); - Iterator iterator = - ChangelogIterator.removeCarryovers(rowsWithDuplication.iterator(), SCHEMA); - List result = Lists.newArrayList(iterator); - - assertEquals( - "Rows should match.", + List expectedRows = Lists.newArrayList( - new Object[] {1, "d", "data", DELETE}, new Object[] {2, "d", "data", INSERT}), - rowsToJava(result)); + new Object[] {1, "d", "data", DELETE, 0, 0}, + new Object[] {2, "d", "data", INSERT, 0, 0}); + + validateIterators(rowsWithDuplication, expectedRows); } @Test public void testCarryRowsRemoveMoreInsertRows() { List rowsWithDuplication = Lists.newArrayList( - new GenericRowWithSchema(new Object[] {0, "d", "data", DELETE}, null), - new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE}, null), - new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE}, null), - new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE}, null), + new GenericRowWithSchema(new Object[] {0, "d", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE, 0, 0}, null), // more insert rows than delete rows, should keep extra insert rows - new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT}, null), - new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT}, null), - new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT}, null), - new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT}, null)); - - Iterator iterator = - ChangelogIterator.removeCarryovers(rowsWithDuplication.iterator(), SCHEMA); - List result = Lists.newArrayList(iterator); + new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT, 0, 0}, null)); - assertEquals( - "Rows should match.", + List expectedRows = Lists.newArrayList( - new Object[] {0, "d", "data", DELETE}, new Object[] {1, "d", "data", INSERT}), - rowsToJava(result)); + new Object[] {0, "d", "data", DELETE, 0, 0}, + new Object[] {1, "d", "data", INSERT, 0, 0}); + + validateIterators(rowsWithDuplication, expectedRows); } @Test @@ -296,17 +296,64 @@ public void testCarryRowsRemoveNoInsertRows() { List rowsWithDuplication = Lists.newArrayList( // next two rows are identical - new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE}, null), - new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE}, null)); + new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE, 0, 0}, null)); + + List expectedRows = + Lists.newArrayList( + new Object[] {1, "d", "data", DELETE, 0, 0}, + new Object[] {1, "d", "data", DELETE, 0, 0}); + validateIterators(rowsWithDuplication, expectedRows); + } + + private void validateIterators(List rowsWithDuplication, List expectedRows) { Iterator iterator = ChangelogIterator.removeCarryovers(rowsWithDuplication.iterator(), SCHEMA); List result = Lists.newArrayList(iterator); - assertEquals( - "Duplicate rows should not be removed", + assertEquals("Rows should match.", expectedRows, rowsToJava(result)); + + iterator = ChangelogIterator.removeNetCarryovers(rowsWithDuplication.iterator(), SCHEMA); + result = Lists.newArrayList(iterator); + + assertEquals("Rows should match.", expectedRows, rowsToJava(result)); + } + + @Test + public void testRemoveNetCarryovers() { + List rowsWithDuplication = Lists.newArrayList( - new Object[] {1, "d", "data", DELETE}, new Object[] {1, "d", "data", DELETE}), - rowsToJava(result)); + // this row are different from other rows, it is a net change, should be kept + new GenericRowWithSchema(new Object[] {0, "d", "data", DELETE, 0, 0}, null), + // a pair of delete and insert rows, should be removed + new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT, 0, 0}, null), + // 2 delete rows and 2 insert rows, should be removed + new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE, 1, 1}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE, 1, 1}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT, 1, 1}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT, 1, 1}, null), + // a pair of insert and delete rows across snapshots, should be removed + new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT, 2, 2}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE, 3, 3}, null), + // extra insert rows, they are net changes, should be kept + new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT, 4, 4}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT, 4, 4}, null), + // different key, net changes, should be kept + new GenericRowWithSchema(new Object[] {2, "d", "data", DELETE, 4, 4}, null)); + + List expectedRows = + Lists.newArrayList( + new Object[] {0, "d", "data", DELETE, 0, 0}, + new Object[] {1, "d", "data", INSERT, 4, 4}, + new Object[] {1, "d", "data", INSERT, 4, 4}, + new Object[] {2, "d", "data", DELETE, 4, 4}); + + Iterator iterator = + ChangelogIterator.removeNetCarryovers(rowsWithDuplication.iterator(), SCHEMA); + List result = Lists.newArrayList(iterator); + + assertEquals("Rows should match.", expectedRows, rowsToJava(result)); } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestSparkV2Filters.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestSparkV2Filters.java index 4c8a32fa41a4..dabd5d991b12 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestSparkV2Filters.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestSparkV2Filters.java @@ -34,11 +34,13 @@ import org.apache.spark.sql.connector.expressions.filter.Predicate; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.unsafe.types.UTF8String; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Test; public class TestSparkV2Filters { + @SuppressWarnings("checkstyle:MethodLength") @Test public void testV2Filters() { Map attrMap = Maps.newHashMap(); @@ -128,6 +130,18 @@ public void testV2Filters() { Expression actualEq2 = SparkV2Filters.convert(eq2); Assert.assertEquals("EqualTo must match", expectedEq2.toString(), actualEq2.toString()); + Predicate notEq1 = new Predicate("<>", attrAndValue); + Expression expectedNotEq1 = Expressions.notEqual(unquoted, 1); + Expression actualNotEq1 = SparkV2Filters.convert(notEq1); + Assert.assertEquals( + "NotEqualTo must match", expectedNotEq1.toString(), actualNotEq1.toString()); + + Predicate notEq2 = new Predicate("<>", valueAndAttr); + Expression expectedNotEq2 = Expressions.notEqual(unquoted, 1); + Expression actualNotEq2 = SparkV2Filters.convert(notEq2); + Assert.assertEquals( + "NotEqualTo must match", expectedNotEq2.toString(), actualNotEq2.toString()); + Predicate eqNullSafe1 = new Predicate("<=>", attrAndValue); Expression expectedEqNullSafe1 = Expressions.equal(unquoted, 1); Expression actualEqNullSafe1 = SparkV2Filters.convert(eqNullSafe1); @@ -189,6 +203,101 @@ public void testV2Filters() { }); } + @Test + public void testEqualToNull() { + String col = "col"; + NamedReference namedReference = FieldReference.apply(col); + LiteralValue value = new LiteralValue(null, DataTypes.IntegerType); + + org.apache.spark.sql.connector.expressions.Expression[] attrAndValue = + new org.apache.spark.sql.connector.expressions.Expression[] {namedReference, value}; + org.apache.spark.sql.connector.expressions.Expression[] valueAndAttr = + new org.apache.spark.sql.connector.expressions.Expression[] {value, namedReference}; + + Predicate eq1 = new Predicate("=", attrAndValue); + Assertions.assertThatThrownBy(() -> SparkV2Filters.convert(eq1)) + .isInstanceOf(NullPointerException.class) + .hasMessageContaining("Expression is always false"); + + Predicate eq2 = new Predicate("=", valueAndAttr); + Assertions.assertThatThrownBy(() -> SparkV2Filters.convert(eq2)) + .isInstanceOf(NullPointerException.class) + .hasMessageContaining("Expression is always false"); + + Predicate eqNullSafe1 = new Predicate("<=>", attrAndValue); + Expression expectedEqNullSafe = Expressions.isNull(col); + Expression actualEqNullSafe1 = SparkV2Filters.convert(eqNullSafe1); + Assertions.assertThat(actualEqNullSafe1.toString()).isEqualTo(expectedEqNullSafe.toString()); + + Predicate eqNullSafe2 = new Predicate("<=>", valueAndAttr); + Expression actualEqNullSafe2 = SparkV2Filters.convert(eqNullSafe2); + Assertions.assertThat(actualEqNullSafe2.toString()).isEqualTo(expectedEqNullSafe.toString()); + } + + @Test + public void testEqualToNaN() { + String col = "col"; + NamedReference namedReference = FieldReference.apply(col); + LiteralValue value = new LiteralValue(Float.NaN, DataTypes.FloatType); + + org.apache.spark.sql.connector.expressions.Expression[] attrAndValue = + new org.apache.spark.sql.connector.expressions.Expression[] {namedReference, value}; + org.apache.spark.sql.connector.expressions.Expression[] valueAndAttr = + new org.apache.spark.sql.connector.expressions.Expression[] {value, namedReference}; + + Predicate eqNaN1 = new Predicate("=", attrAndValue); + Expression expectedEqNaN = Expressions.isNaN(col); + Expression actualEqNaN1 = SparkV2Filters.convert(eqNaN1); + Assertions.assertThat(actualEqNaN1.toString()).isEqualTo(expectedEqNaN.toString()); + + Predicate eqNaN2 = new Predicate("=", valueAndAttr); + Expression actualEqNaN2 = SparkV2Filters.convert(eqNaN2); + Assertions.assertThat(actualEqNaN2.toString()).isEqualTo(expectedEqNaN.toString()); + } + + @Test + public void testNotEqualToNull() { + String col = "col"; + NamedReference namedReference = FieldReference.apply(col); + LiteralValue value = new LiteralValue(null, DataTypes.IntegerType); + + org.apache.spark.sql.connector.expressions.Expression[] attrAndValue = + new org.apache.spark.sql.connector.expressions.Expression[] {namedReference, value}; + org.apache.spark.sql.connector.expressions.Expression[] valueAndAttr = + new org.apache.spark.sql.connector.expressions.Expression[] {value, namedReference}; + + Predicate notEq1 = new Predicate("<>", attrAndValue); + Assertions.assertThatThrownBy(() -> SparkV2Filters.convert(notEq1)) + .isInstanceOf(NullPointerException.class) + .hasMessageContaining("Expression is always false"); + + Predicate notEq2 = new Predicate("<>", valueAndAttr); + Assertions.assertThatThrownBy(() -> SparkV2Filters.convert(notEq2)) + .isInstanceOf(NullPointerException.class) + .hasMessageContaining("Expression is always false"); + } + + @Test + public void testNotEqualToNaN() { + String col = "col"; + NamedReference namedReference = FieldReference.apply(col); + LiteralValue value = new LiteralValue(Float.NaN, DataTypes.FloatType); + + org.apache.spark.sql.connector.expressions.Expression[] attrAndValue = + new org.apache.spark.sql.connector.expressions.Expression[] {namedReference, value}; + org.apache.spark.sql.connector.expressions.Expression[] valueAndAttr = + new org.apache.spark.sql.connector.expressions.Expression[] {value, namedReference}; + + Predicate notEqNaN1 = new Predicate("<>", attrAndValue); + Expression expectedNotEqNaN = Expressions.notNaN(col); + Expression actualNotEqNaN1 = SparkV2Filters.convert(notEqNaN1); + Assertions.assertThat(actualNotEqNaN1.toString()).isEqualTo(expectedNotEqNaN.toString()); + + Predicate notEqNaN2 = new Predicate("<>", valueAndAttr); + Expression actualNotEqNaN2 = SparkV2Filters.convert(notEqNaN2); + Assertions.assertThat(actualNotEqNaN2.toString()).isEqualTo(expectedNotEqNaN.toString()); + } + @Test public void testTimestampFilterConversion() { Instant instant = Instant.parse("2018-10-18T00:00:57.907Z"); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java index bf4bef74c3fe..61cddfb0217b 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java @@ -94,6 +94,7 @@ import org.apache.iceberg.spark.SparkTestBase; import org.apache.iceberg.spark.SparkWriteOptions; import org.apache.iceberg.spark.actions.RewriteDataFilesSparkAction.RewriteExecutionContext; +import org.apache.iceberg.spark.data.TestHelpers; import org.apache.iceberg.spark.source.ThreeColumnRecord; import org.apache.iceberg.types.Comparators; import org.apache.iceberg.types.Conversions; @@ -263,9 +264,7 @@ public void testBinPackWithDeletes() throws Exception { shouldHaveFiles(table, 8); table.refresh(); - CloseableIterable tasks = table.newScan().planFiles(); - List dataFiles = - Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); + List dataFiles = TestHelpers.dataFiles(table); int total = (int) dataFiles.stream().mapToLong(ContentFile::recordCount).sum(); RowDelta rowDelta = table.newRowDelta(); @@ -309,9 +308,7 @@ public void testBinPackWithDeleteAllData() { shouldHaveFiles(table, 1); table.refresh(); - CloseableIterable tasks = table.newScan().planFiles(); - List dataFiles = - Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); + List dataFiles = TestHelpers.dataFiles(table); int total = (int) dataFiles.stream().mapToLong(ContentFile::recordCount).sum(); RowDelta rowDelta = table.newRowDelta(); @@ -894,6 +891,15 @@ public void testInvalidOptions() { () -> basicRewrite(table).option(RewriteDataFiles.REWRITE_JOB_ORDER, "foo").execute()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid rewrite job order name: foo"); + + Assertions.assertThatThrownBy( + () -> + basicRewrite(table) + .sort(SortOrder.builderFor(table.schema()).asc("c2").build()) + .option(SparkShufflingDataRewriter.SHUFFLE_PARTITIONS_PER_FILE, "5") + .execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("requires enabling Iceberg Spark session extensions"); } @Test diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java index 5c0aa2e6aa43..9267fae85f7c 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java @@ -32,7 +32,6 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; -import org.apache.iceberg.FileScanTask; import org.apache.iceberg.Files; import org.apache.iceberg.MetadataTableType; import org.apache.iceberg.MetadataTableUtils; @@ -60,6 +59,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.SparkCatalogConfig; import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.apache.iceberg.spark.data.TestHelpers; import org.apache.iceberg.spark.source.FourColumnRecord; import org.apache.iceberg.spark.source.ThreeColumnRecord; import org.apache.iceberg.types.Types; @@ -136,7 +136,7 @@ public void testEmptyTable() { @Test public void testUnpartitioned() throws Exception { Table table = createTableUnpartitioned(2, SCALE); - List dataFiles = dataFiles(table); + List dataFiles = TestHelpers.dataFiles(table); writePosDeletesForFiles(table, 2, DELETES_SCALE, dataFiles); Assert.assertEquals(2, dataFiles.size()); @@ -170,7 +170,7 @@ public void testUnpartitioned() throws Exception { public void testRewriteAll() throws Exception { Table table = createTablePartitioned(4, 2, SCALE); - List dataFiles = dataFiles(table); + List dataFiles = TestHelpers.dataFiles(table); writePosDeletesForFiles(table, 2, DELETES_SCALE, dataFiles); Assert.assertEquals(4, dataFiles.size()); @@ -206,7 +206,7 @@ public void testRewriteAll() throws Exception { public void testRewriteToSmallerTarget() throws Exception { Table table = createTablePartitioned(4, 2, SCALE); - List dataFiles = dataFiles(table); + List dataFiles = TestHelpers.dataFiles(table); writePosDeletesForFiles(table, 2, DELETES_SCALE, dataFiles); Assert.assertEquals(4, dataFiles.size()); @@ -243,7 +243,7 @@ public void testRewriteToSmallerTarget() throws Exception { public void testRemoveDanglingDeletes() throws Exception { Table table = createTablePartitioned(4, 2, SCALE); - List dataFiles = dataFiles(table); + List dataFiles = TestHelpers.dataFiles(table); writePosDeletesForFiles( table, 2, @@ -288,7 +288,7 @@ public void testRemoveDanglingDeletes() throws Exception { public void testSomePartitionsDanglingDeletes() throws Exception { Table table = createTablePartitioned(4, 2, SCALE); - List dataFiles = dataFiles(table); + List dataFiles = TestHelpers.dataFiles(table); writePosDeletesForFiles(table, 2, DELETES_SCALE, dataFiles); Assert.assertEquals(4, dataFiles.size()); @@ -340,7 +340,7 @@ public void testSomePartitionsDanglingDeletes() throws Exception { @Test public void testPartitionEvolutionAdd() throws Exception { Table table = createTableUnpartitioned(2, SCALE); - List unpartitionedDataFiles = dataFiles(table); + List unpartitionedDataFiles = TestHelpers.dataFiles(table); writePosDeletesForFiles(table, 2, DELETES_SCALE, unpartitionedDataFiles); Assert.assertEquals(2, unpartitionedDataFiles.size()); @@ -354,7 +354,8 @@ public void testPartitionEvolutionAdd() throws Exception { table.updateSpec().addField("c1").commit(); writeRecords(table, 2, SCALE, 2); - List partitionedDataFiles = except(dataFiles(table), unpartitionedDataFiles); + List partitionedDataFiles = + except(TestHelpers.dataFiles(table), unpartitionedDataFiles); writePosDeletesForFiles(table, 2, DELETES_SCALE, partitionedDataFiles); Assert.assertEquals(2, partitionedDataFiles.size()); @@ -391,7 +392,7 @@ public void testPartitionEvolutionAdd() throws Exception { @Test public void testPartitionEvolutionRemove() throws Exception { Table table = createTablePartitioned(2, 2, SCALE); - List dataFilesUnpartitioned = dataFiles(table); + List dataFilesUnpartitioned = TestHelpers.dataFiles(table); writePosDeletesForFiles(table, 2, DELETES_SCALE, dataFilesUnpartitioned); Assert.assertEquals(2, dataFilesUnpartitioned.size()); @@ -401,7 +402,8 @@ public void testPartitionEvolutionRemove() throws Exception { table.updateSpec().removeField("c1").commit(); writeRecords(table, 2, SCALE); - List dataFilesPartitioned = except(dataFiles(table), dataFilesUnpartitioned); + List dataFilesPartitioned = + except(TestHelpers.dataFiles(table), dataFilesUnpartitioned); writePosDeletesForFiles(table, 2, DELETES_SCALE, dataFilesPartitioned); Assert.assertEquals(2, dataFilesPartitioned.size()); @@ -438,7 +440,7 @@ public void testPartitionEvolutionRemove() throws Exception { @Test public void testSchemaEvolution() throws Exception { Table table = createTablePartitioned(2, 2, SCALE); - List dataFiles = dataFiles(table); + List dataFiles = TestHelpers.dataFiles(table); writePosDeletesForFiles(table, 2, DELETES_SCALE, dataFiles); Assert.assertEquals(2, dataFiles.size()); @@ -450,7 +452,7 @@ public void testSchemaEvolution() throws Exception { int newColId = table.schema().findField("c4").fieldId(); List newSchemaDataFiles = - dataFiles(table).stream() + TestHelpers.dataFiles(table).stream() .filter(f -> f.upperBounds().containsKey(newColId)) .collect(Collectors.toList()); writePosDeletesForFiles(table, 2, DELETES_SCALE, newSchemaDataFiles); @@ -679,11 +681,6 @@ private void writePosDeletesForFiles( } } - private List dataFiles(Table table) { - CloseableIterable tasks = table.newScan().includeColumnStats().planFiles(); - return Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); - } - private List deleteFiles(Table table) { Table deletesTable = MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewriter.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewriter.java index 6800ffd404ea..055e5be681c6 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewriter.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewriter.java @@ -261,6 +261,7 @@ public void testSortDataValidOptions() { Assert.assertEquals( "Rewriter must report all supported options", ImmutableSet.of( + SparkSortDataRewriter.SHUFFLE_PARTITIONS_PER_FILE, SparkSortDataRewriter.TARGET_FILE_SIZE_BYTES, SparkSortDataRewriter.MIN_FILE_SIZE_BYTES, SparkSortDataRewriter.MAX_FILE_SIZE_BYTES, @@ -281,6 +282,7 @@ public void testZOrderDataValidOptions() { Assert.assertEquals( "Rewriter must report all supported options", ImmutableSet.of( + SparkZOrderDataRewriter.SHUFFLE_PARTITIONS_PER_FILE, SparkZOrderDataRewriter.TARGET_FILE_SIZE_BYTES, SparkZOrderDataRewriter.MIN_FILE_SIZE_BYTES, SparkZOrderDataRewriter.MAX_FILE_SIZE_BYTES, diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java index 208f7749962e..ca9d14f05e59 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java @@ -51,6 +51,7 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.TableScan; +import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.relocated.com.google.common.collect.Streams; @@ -798,22 +799,18 @@ public static List deleteManifests(Table table) { return table.currentSnapshot().deleteManifests(table.io()); } - public static Set dataFiles(Table table) { + public static List dataFiles(Table table) { return dataFiles(table, null); } - public static Set dataFiles(Table table, String branch) { - Set dataFiles = Sets.newHashSet(); + public static List dataFiles(Table table, String branch) { TableScan scan = table.newScan(); if (branch != null) { scan.useRef(branch); } - for (FileScanTask task : scan.planFiles()) { - dataFiles.add(task.file()); - } - - return dataFiles; + CloseableIterable tasks = scan.includeColumnStats().planFiles(); + return Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); } public static Set deleteFiles(Table table) { diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java index 4082cd6d441f..b23fe729a187 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java @@ -26,7 +26,6 @@ import java.util.Collections; import java.util.Iterator; import java.util.List; -import java.util.Objects; import org.apache.iceberg.Files; import org.apache.iceberg.Schema; import org.apache.iceberg.io.CloseableIterable; @@ -102,12 +101,6 @@ private void writeAndValidateRecords(Schema schema, Iterable expect assertEquals(schema, expectedRows.next(), actualRows.next()); } Assert.assertFalse("Should not have extra rows", actualRows.hasNext()); - } catch (UnsupportedOperationException e) { - // Fixed in https://github.com/apache/spark/pull/41103 - // Can be removed once Spark 3.4.1 is released - if (!Objects.equals(e.getMessage(), "Datatype not supported TimestampNTZType")) { - throw e; - } } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetVectorizedReads.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetVectorizedReads.java index 9250f4ba4295..c763b7b7cc12 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetVectorizedReads.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetVectorizedReads.java @@ -60,15 +60,7 @@ public class TestParquetVectorizedReads extends AvroDataTest { @Override protected void writeAndValidate(Schema schema) throws IOException { - try { - writeAndValidate(schema, getNumRows(), 0L, RandomData.DEFAULT_NULL_PERCENTAGE, true); - } catch (UnsupportedOperationException exc) { - // Fixed in https://github.com/apache/spark/pull/41103 - // Can be removed once Spark 3.4.1 is released - if (!exc.getMessage().equals("Datatype not supported TimestampNTZType")) { - throw exc; - } - } + writeAndValidate(schema, getNumRows(), 0L, RandomData.DEFAULT_NULL_PERCENTAGE, true); } private void writeAndValidate( diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java index 5e819200f56c..83d8953735c5 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java @@ -32,10 +32,12 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.hadoop.HadoopTables; import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.math.LongMath; @@ -425,8 +427,14 @@ public void testExtraSnapshotMetadataWithSQL() throws InterruptedException, IOEx Thread writerThread = new Thread( () -> { - Map properties = Maps.newHashMap(); - properties.put("writer-thread", String.valueOf(Thread.currentThread().getName())); + Map properties = + ImmutableMap.of( + "writer-thread", + String.valueOf(Thread.currentThread().getName()), + SnapshotSummary.EXTRA_METADATA_PREFIX + "extra-key", + "someValue", + SnapshotSummary.EXTRA_METADATA_PREFIX + "another-key", + "anotherValue"); CommitMetadata.withCommitProperties( properties, () -> { @@ -442,8 +450,10 @@ public void testExtraSnapshotMetadataWithSQL() throws InterruptedException, IOEx List snapshots = Lists.newArrayList(table.snapshots()); Assert.assertEquals(2, snapshots.size()); Assert.assertNull(snapshots.get(0).summary().get("writer-thread")); - Assert.assertEquals( - "test-extra-commit-message-writer-thread", snapshots.get(1).summary().get("writer-thread")); + Assertions.assertThat(snapshots.get(1).summary()) + .containsEntry("writer-thread", "test-extra-commit-message-writer-thread") + .containsEntry("extra-key", "someValue") + .containsEntry("another-key", "anotherValue"); } @Test @@ -459,8 +469,14 @@ public void testExtraSnapshotMetadataWithDelete() Thread writerThread = new Thread( () -> { - Map properties = Maps.newHashMap(); - properties.put("writer-thread", String.valueOf(Thread.currentThread().getName())); + Map properties = + ImmutableMap.of( + "writer-thread", + String.valueOf(Thread.currentThread().getName()), + SnapshotSummary.EXTRA_METADATA_PREFIX + "extra-key", + "someValue", + SnapshotSummary.EXTRA_METADATA_PREFIX + "another-key", + "anotherValue"); CommitMetadata.withCommitProperties( properties, () -> { @@ -477,7 +493,9 @@ public void testExtraSnapshotMetadataWithDelete() List snapshots = Lists.newArrayList(table.snapshots()); Assert.assertEquals(2, snapshots.size()); Assert.assertNull(snapshots.get(0).summary().get("writer-thread")); - Assert.assertEquals( - "test-extra-commit-message-delete-thread", snapshots.get(1).summary().get("writer-thread")); + Assertions.assertThat(snapshots.get(1).summary()) + .containsEntry("writer-thread", "test-extra-commit-message-delete-thread") + .containsEntry("extra-key", "someValue") + .containsEntry("another-key", "anotherValue"); } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index a6687a4ce7eb..61bed7464b57 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -28,6 +28,7 @@ import java.io.UncheckedIOException; import java.time.LocalDateTime; import java.time.ZoneOffset; +import java.util.Arrays; import java.util.Comparator; import java.util.List; import java.util.StringJoiner; @@ -47,6 +48,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.actions.DeleteOrphanFiles; +import org.apache.iceberg.actions.RewriteManifests; import org.apache.iceberg.avro.Avro; import org.apache.iceberg.avro.AvroSchemaUtil; import org.apache.iceberg.catalog.TableIdentifier; @@ -204,7 +206,7 @@ public void testEntriesTable() throws Exception { } @Test - public void testEntriesTablePartitionedPrune() throws Exception { + public void testEntriesTablePartitionedPrune() { TableIdentifier tableIdentifier = TableIdentifier.of("db", "entries_test"); Table table = createTable(tableIdentifier, SCHEMA, SPEC); @@ -233,7 +235,7 @@ public void testEntriesTablePartitionedPrune() throws Exception { } @Test - public void testEntriesTableDataFilePrune() throws Exception { + public void testEntriesTableDataFilePrune() { TableIdentifier tableIdentifier = TableIdentifier.of("db", "entries_test"); Table table = createTable(tableIdentifier, SCHEMA, PartitionSpec.unpartitioned()); @@ -266,7 +268,7 @@ public void testEntriesTableDataFilePrune() throws Exception { } @Test - public void testEntriesTableDataFilePruneMulti() throws Exception { + public void testEntriesTableDataFilePruneMulti() { TableIdentifier tableIdentifier = TableIdentifier.of("db", "entries_test"); Table table = createTable(tableIdentifier, SCHEMA, PartitionSpec.unpartitioned()); @@ -304,7 +306,7 @@ public void testEntriesTableDataFilePruneMulti() throws Exception { } @Test - public void testFilesSelectMap() throws Exception { + public void testFilesSelectMap() { TableIdentifier tableIdentifier = TableIdentifier.of("db", "entries_test"); Table table = createTable(tableIdentifier, SCHEMA, PartitionSpec.unpartitioned()); @@ -644,7 +646,7 @@ public void testFilesUnpartitionedTable() throws Exception { } @Test - public void testAllMetadataTablesWithStagedCommits() throws Exception { + public void testAllMetadataTablesWithStagedCommits() { TableIdentifier tableIdentifier = TableIdentifier.of("db", "stage_aggregate_table_test"); Table table = createTable(tableIdentifier, SCHEMA, SPEC); @@ -691,8 +693,7 @@ public void testAllMetadataTablesWithStagedCommits() throws Exception { Assert.assertTrue( "Stage table should have some snapshots", table.snapshots().iterator().hasNext()); - Assert.assertEquals( - "Stage table should have null currentSnapshot", null, table.currentSnapshot()); + Assert.assertNull("Stage table should have null currentSnapshot", table.currentSnapshot()); Assert.assertEquals("Actual results should have two rows", 2, actualAllData.size()); Assert.assertEquals("Actual results should have two rows", 2, actualAllManifests.size()); Assert.assertEquals("Actual results should have two rows", 2, actualAllEntries.size()); @@ -1212,8 +1213,8 @@ public void testAllManifestsTable() { snapshotManifest -> manifestRecord( manifestTable, snapshotManifest.first(), snapshotManifest.second())) + .sorted(Comparator.comparing(o -> o.get("path").toString())) .collect(Collectors.toList()); - expected.sort(Comparator.comparing(o -> o.get("path").toString())); Assert.assertEquals("Manifests table should have 5 manifest rows", 5, actual.size()); for (int i = 0; i < expected.size(); i += 1) { @@ -1225,7 +1226,7 @@ public void testAllManifestsTable() { @Test public void testUnpartitionedPartitionsTable() { TableIdentifier tableIdentifier = TableIdentifier.of("db", "unpartitioned_partitions_test"); - createTable(tableIdentifier, SCHEMA, PartitionSpec.unpartitioned()); + Table table = createTable(tableIdentifier, SCHEMA, PartitionSpec.unpartitioned()); Dataset df = spark.createDataFrame(Lists.newArrayList(new SimpleRecord(1, "a")), SimpleRecord.class); @@ -1240,6 +1241,11 @@ public void testUnpartitionedPartitionsTable() { Types.StructType.of( required(2, "record_count", Types.LongType.get(), "Count of records in data files"), required(3, "file_count", Types.IntegerType.get(), "Count of data files"), + required( + 11, + "total_data_file_size_in_bytes", + Types.LongType.get(), + "Total size in bytes of data files"), required( 5, "position_delete_record_count", @@ -1259,7 +1265,17 @@ public void testUnpartitionedPartitionsTable() { 8, "equality_delete_file_count", Types.IntegerType.get(), - "Count of equality delete files")); + "Count of equality delete files"), + optional( + 9, + "last_updated_at", + Types.TimestampType.withZone(), + "Commit time of snapshot that last updated this partition"), + optional( + 10, + "last_updated_snapshot_id", + Types.LongType.get(), + "Id of snapshot that last updated this partition")); Table partitionsTable = loadTable(tableIdentifier, "partitions"); @@ -1272,8 +1288,13 @@ public void testUnpartitionedPartitionsTable() { new GenericRecordBuilder(AvroSchemaUtil.convert(partitionsTable.schema(), "partitions")); GenericData.Record expectedRow = builder + .set("last_updated_at", table.currentSnapshot().timestampMillis() * 1000) + .set("last_updated_snapshot_id", table.currentSnapshot().snapshotId()) .set("record_count", 1L) .set("file_count", 1) + .set( + "total_data_file_size_in_bytes", + totalSizeInBytes(table.currentSnapshot().addedDataFiles(table.io()))) .set("position_delete_record_count", 0L) .set("position_delete_file_count", 0) .set("equality_delete_record_count", 0L) @@ -1317,6 +1338,9 @@ public void testPartitionsTable() { .mode("append") .save(loadLocation(tableIdentifier)); + table.refresh(); + long secondCommitId = table.currentSnapshot().snapshotId(); + List actual = spark .read() @@ -1337,22 +1361,32 @@ public void testPartitionsTable() { .set("partition", partitionBuilder.set("id", 1).build()) .set("record_count", 1L) .set("file_count", 1) + .set( + "total_data_file_size_in_bytes", + totalSizeInBytes(table.snapshot(firstCommitId).addedDataFiles(table.io()))) .set("position_delete_record_count", 0L) .set("position_delete_file_count", 0) .set("equality_delete_record_count", 0L) .set("equality_delete_file_count", 0) .set("spec_id", 0) + .set("last_updated_at", table.snapshot(firstCommitId).timestampMillis() * 1000) + .set("last_updated_snapshot_id", firstCommitId) .build()); expected.add( builder .set("partition", partitionBuilder.set("id", 2).build()) .set("record_count", 1L) .set("file_count", 1) + .set( + "total_data_file_size_in_bytes", + totalSizeInBytes(table.snapshot(secondCommitId).addedDataFiles(table.io()))) .set("position_delete_record_count", 0L) .set("position_delete_file_count", 0) .set("equality_delete_record_count", 0L) .set("equality_delete_file_count", 0) .set("spec_id", 0) + .set("last_updated_at", table.snapshot(secondCommitId).timestampMillis() * 1000) + .set("last_updated_snapshot_id", secondCommitId) .build()); Assert.assertEquals("Partitions table should have two rows", 2, expected.size()); @@ -1395,13 +1429,160 @@ public void testPartitionsTable() { .load(loadLocation(tableIdentifier, "partitions")) .filter("partition.id < 2 or record_count=1") .collectAsList(); - Assert.assertEquals("Actual results should have one row", 2, nonFiltered.size()); + Assert.assertEquals("Actual results should have two row", 2, nonFiltered.size()); for (int i = 0; i < 2; i += 1) { TestHelpers.assertEqualsSafe( partitionsTable.schema().asStruct(), expected.get(i), actual.get(i)); } } + @Test + public void testPartitionsTableLastUpdatedSnapshot() { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "partitions_test"); + Table table = createTable(tableIdentifier, SCHEMA, SPEC); + Table partitionsTable = loadTable(tableIdentifier, "partitions"); + Dataset df1 = + spark.createDataFrame( + Lists.newArrayList(new SimpleRecord(1, "1"), new SimpleRecord(2, "2")), + SimpleRecord.class); + Dataset df2 = + spark.createDataFrame(Lists.newArrayList(new SimpleRecord(2, "20")), SimpleRecord.class); + + df1.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + table.refresh(); + long firstCommitId = table.currentSnapshot().snapshotId(); + + // add a second file + df2.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + table.refresh(); + long secondCommitId = table.currentSnapshot().snapshotId(); + + // check if rewrite manifest does not override metadata about data file's creating snapshot + RewriteManifests.Result rewriteManifestResult = + SparkActions.get().rewriteManifests(table).execute(); + Assert.assertEquals( + "rewrite replaced 2 manifests", + 2, + Iterables.size(rewriteManifestResult.rewrittenManifests())); + Assert.assertEquals( + "rewrite added 1 manifests", 1, Iterables.size(rewriteManifestResult.addedManifests())); + + List actual = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "partitions")) + .orderBy("partition.id") + .collectAsList(); + + GenericRecordBuilder builder = + new GenericRecordBuilder(AvroSchemaUtil.convert(partitionsTable.schema(), "partitions")); + GenericRecordBuilder partitionBuilder = + new GenericRecordBuilder( + AvroSchemaUtil.convert( + partitionsTable.schema().findType("partition").asStructType(), "partition")); + + List dataFiles = TestHelpers.dataFiles(table); + assertDataFilePartitions(dataFiles, Arrays.asList(1, 2, 2)); + + List expected = Lists.newArrayList(); + expected.add( + builder + .set("partition", partitionBuilder.set("id", 1).build()) + .set("record_count", 1L) + .set("file_count", 1) + .set("total_data_file_size_in_bytes", dataFiles.get(0).fileSizeInBytes()) + .set("position_delete_record_count", 0L) + .set("position_delete_file_count", 0) + .set("equality_delete_record_count", 0L) + .set("equality_delete_file_count", 0) + .set("spec_id", 0) + .set("last_updated_at", table.snapshot(firstCommitId).timestampMillis() * 1000) + .set("last_updated_snapshot_id", firstCommitId) + .build()); + expected.add( + builder + .set("partition", partitionBuilder.set("id", 2).build()) + .set("record_count", 2L) + .set("file_count", 2) + .set( + "total_data_file_size_in_bytes", + dataFiles.get(1).fileSizeInBytes() + dataFiles.get(2).fileSizeInBytes()) + .set("position_delete_record_count", 0L) + .set("position_delete_file_count", 0) + .set("equality_delete_record_count", 0L) + .set("equality_delete_file_count", 0) + .set("spec_id", 0) + .set("last_updated_at", table.snapshot(secondCommitId).timestampMillis() * 1000) + .set("last_updated_snapshot_id", secondCommitId) + .build()); + + Assert.assertEquals("Partitions table should have two rows", 2, expected.size()); + Assert.assertEquals("Actual results should have two rows", 2, actual.size()); + for (int i = 0; i < 2; i += 1) { + TestHelpers.assertEqualsSafe( + partitionsTable.schema().asStruct(), expected.get(i), actual.get(i)); + } + + // check predicate push down + List filtered = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "partitions")) + .filter("partition.id < 2") + .collectAsList(); + Assert.assertEquals("Actual results should have one row", 1, filtered.size()); + TestHelpers.assertEqualsSafe( + partitionsTable.schema().asStruct(), expected.get(0), filtered.get(0)); + + // check for snapshot expiration + // if snapshot with firstCommitId is expired, + // we expect the partition of id=1 will no longer have last updated timestamp and snapshotId + SparkActions.get().expireSnapshots(table).expireSnapshotId(firstCommitId).execute(); + GenericData.Record newPartitionRecord = + builder + .set("partition", partitionBuilder.set("id", 1).build()) + .set("record_count", 1L) + .set("file_count", 1) + .set("total_data_file_size_in_bytes", dataFiles.get(0).fileSizeInBytes()) + .set("position_delete_record_count", 0L) + .set("position_delete_file_count", 0) + .set("equality_delete_record_count", 0L) + .set("equality_delete_file_count", 0) + .set("spec_id", 0) + .set("last_updated_at", null) + .set("last_updated_snapshot_id", null) + .build(); + expected.remove(0); + expected.add(0, newPartitionRecord); + + List actualAfterSnapshotExpiration = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "partitions")) + .collectAsList(); + Assert.assertEquals( + "Actual results should have two row", 2, actualAfterSnapshotExpiration.size()); + for (int i = 0; i < 2; i += 1) { + TestHelpers.assertEqualsSafe( + partitionsTable.schema().asStruct(), + expected.get(i), + actualAfterSnapshotExpiration.get(i)); + } + } + @Test public void testPartitionsTableDeleteStats() { TableIdentifier tableIdentifier = TableIdentifier.of("db", "partitions_test"); @@ -1419,6 +1600,7 @@ public void testPartitionsTableDeleteStats() { .save(loadLocation(tableIdentifier)); table.refresh(); + long firstCommitId = table.currentSnapshot().snapshotId(); // add a second file df2.select("id", "data") @@ -1431,6 +1613,8 @@ public void testPartitionsTableDeleteStats() { table.updateProperties().set(TableProperties.FORMAT_VERSION, "2").commit(); DeleteFile deleteFile = writePosDeleteFile(table); table.newRowDelta().addDeletes(deleteFile).commit(); + table.refresh(); + long posDeleteCommitId = table.currentSnapshot().snapshotId(); List actual = spark @@ -1453,23 +1637,34 @@ public void testPartitionsTableDeleteStats() { .set("partition", partitionBuilder.set("id", 1).build()) .set("record_count", 1L) .set("file_count", 1) + .set( + "total_data_file_size_in_bytes", + totalSizeInBytes(table.snapshot(firstCommitId).addedDataFiles(table.io()))) .set("position_delete_record_count", 0L) .set("position_delete_file_count", 0) .set("equality_delete_record_count", 0L) .set("equality_delete_file_count", 0) .set("spec_id", 0) + .set("last_updated_at", table.snapshot(firstCommitId).timestampMillis() * 1000) + .set("last_updated_snapshot_id", firstCommitId) .build()); expected.add( builder .set("partition", partitionBuilder.set("id", 2).build()) .set("record_count", 1L) .set("file_count", 1) + .set( + "total_data_file_size_in_bytes", + totalSizeInBytes(table.snapshot(firstCommitId).addedDataFiles(table.io()))) .set("position_delete_record_count", 1L) // should be incremented now .set("position_delete_file_count", 1) // should be incremented now .set("equality_delete_record_count", 0L) .set("equality_delete_file_count", 0) .set("spec_id", 0) + .set("last_updated_at", table.snapshot(posDeleteCommitId).timestampMillis() * 1000) + .set("last_updated_snapshot_id", posDeleteCommitId) .build()); + for (int i = 0; i < 2; i += 1) { TestHelpers.assertEqualsSafe( partitionsTable.schema().asStruct(), expected.get(i), actual.get(i)); @@ -1478,6 +1673,8 @@ public void testPartitionsTableDeleteStats() { // test equality delete DeleteFile eqDeleteFile = writeEqDeleteFile(table); table.newRowDelta().addDeletes(eqDeleteFile).commit(); + table.refresh(); + long eqDeleteCommitId = table.currentSnapshot().snapshotId(); actual = spark .read() @@ -1497,7 +1694,8 @@ public void testPartitionsTableDeleteStats() { .set("position_delete_file_count", 0) .set("equality_delete_record_count", 1L) // should be incremented now .set("equality_delete_file_count", 1) // should be incremented now - .set("spec_id", 0) + .set("last_updated_at", table.snapshot(eqDeleteCommitId).timestampMillis() * 1000) + .set("last_updated_snapshot_id", eqDeleteCommitId) .build()); for (int i = 0; i < 2; i += 1) { TestHelpers.assertEqualsSafe( @@ -1771,7 +1969,7 @@ public void testRemoveOrphanFilesActionSupport() throws InterruptedException { } @Test - public void testFilesTablePartitionId() throws Exception { + public void testFilesTablePartitionId() { TableIdentifier tableIdentifier = TableIdentifier.of("db", "files_test"); Table table = createTable( @@ -1811,7 +2009,7 @@ public void testFilesTablePartitionId() throws Exception { } @Test - public void testAllManifestTableSnapshotFiltering() throws Exception { + public void testAllManifestTableSnapshotFiltering() { TableIdentifier tableIdentifier = TableIdentifier.of("db", "all_manifest_snapshot_filtering"); Table table = createTable(tableIdentifier, SCHEMA, SPEC); Table manifestTable = loadTable(tableIdentifier, "all_manifests"); @@ -1880,8 +2078,8 @@ public void testAllManifestTableSnapshotFiltering() throws Exception { snapshotManifest -> manifestRecord( manifestTable, snapshotManifest.first(), snapshotManifest.second())) + .sorted(Comparator.comparing(o -> o.get("path").toString())) .collect(Collectors.toList()); - expected.sort(Comparator.comparing(o -> o.get("path").toString())); Assert.assertEquals("Manifests table should have 3 manifest rows", 3, actual.size()); for (int i = 0; i < expected.size(); i += 1) { @@ -2055,4 +2253,23 @@ private DeleteFile writeEqDeleteFile(Table table) { throw new RuntimeException(e); } } + + private long totalSizeInBytes(Iterable dataFiles) { + return Lists.newArrayList(dataFiles).stream().mapToLong(DataFile::fileSizeInBytes).sum(); + } + + private void assertDataFilePartitions( + List dataFiles, List expectedPartitionIds) { + Assert.assertEquals( + "Table should have " + expectedPartitionIds.size() + " data files", + expectedPartitionIds.size(), + dataFiles.size()); + + for (int i = 0; i < dataFiles.size(); ++i) { + Assert.assertEquals( + "Data file should have partition of id " + expectedPartitionIds.get(i), + expectedPartitionIds.get(i).intValue(), + dataFiles.get(i).partition().get(0, Integer.class).intValue()); + } + } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java new file mode 100644 index 000000000000..7b943372d167 --- /dev/null +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source; + +import static scala.collection.JavaConverters.seqAsJavaListConverter; + +import java.util.List; +import java.util.Map; +import org.apache.iceberg.spark.SparkTestBaseWithCatalog; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.execution.SparkPlan; +import org.apache.spark.sql.execution.metric.SQLMetric; +import org.assertj.core.api.Assertions; +import org.junit.After; +import org.junit.Test; +import scala.collection.JavaConverters; + +public class TestSparkReadMetrics extends SparkTestBaseWithCatalog { + + @After + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @Test + public void testReadMetricsForV1Table() throws NoSuchTableException { + sql( + "CREATE TABLE %s (id BIGINT) USING iceberg TBLPROPERTIES ('format-version'='2')", + tableName); + + spark.range(10000).coalesce(1).writeTo(tableName).append(); + spark.range(10001, 20000).coalesce(1).writeTo(tableName).append(); + + Dataset df = spark.sql(String.format("select * from %s where id < 10000", tableName)); + df.collect(); + + List sparkPlans = + seqAsJavaListConverter(df.queryExecution().executedPlan().collectLeaves()).asJava(); + Map metricsMap = + JavaConverters.mapAsJavaMapConverter(sparkPlans.get(0).metrics()).asJava(); + Assertions.assertThat(metricsMap.get("skippedDataFiles").value()).isEqualTo(1); + Assertions.assertThat(metricsMap.get("scannedDataManifests").value()).isEqualTo(2); + Assertions.assertThat(metricsMap.get("scannedDataFiles").value()).isEqualTo(1); + Assertions.assertThat(metricsMap.get("skippedDataManifests").value()).isEqualTo(0); + Assertions.assertThat(metricsMap.get("totalFileSize").value()).isNotEqualTo(0); + Assertions.assertThat(metricsMap.get("totalPlanningDuration").value()).isNotEqualTo(0); + } + + @Test + public void testReadMetricsForV2Table() throws NoSuchTableException { + sql( + "CREATE TABLE %s (id BIGINT) USING iceberg TBLPROPERTIES ('format-version'='2')", + tableName); + + spark.range(10000).coalesce(1).writeTo(tableName).append(); + spark.range(10001, 20000).coalesce(1).writeTo(tableName).append(); + + Dataset df = spark.sql(String.format("select * from %s where id < 10000", tableName)); + df.collect(); + + List sparkPlans = + seqAsJavaListConverter(df.queryExecution().executedPlan().collectLeaves()).asJava(); + Map metricsMap = + JavaConverters.mapAsJavaMapConverter(sparkPlans.get(0).metrics()).asJava(); + Assertions.assertThat(metricsMap.get("skippedDataFiles").value()).isEqualTo(1); + Assertions.assertThat(metricsMap.get("scannedDataManifests").value()).isEqualTo(2); + Assertions.assertThat(metricsMap.get("scannedDataFiles").value()).isEqualTo(1); + Assertions.assertThat(metricsMap.get("skippedDataManifests").value()).isEqualTo(0); + Assertions.assertThat(metricsMap.get("totalFileSize").value()).isNotEqualTo(0); + Assertions.assertThat(metricsMap.get("totalPlanningDuration").value()).isNotEqualTo(0); + } +} diff --git a/versions.props b/versions.props index a91a4ab8e257..03c9502b31f1 100644 --- a/versions.props +++ b/versions.props @@ -4,16 +4,16 @@ org.apache.calcite:* = 1.10.0 org.apache.hadoop:* = 2.7.3 org.apache.hive:* = 2.3.9 org.apache.httpcomponents.client5:* = 5.2.1 -org.apache.orc:* = 1.8.3 +org.apache.orc:* = 1.9.0 org.apache.parquet:* = 1.13.1 org.apache.pig:pig = 0.14.0 com.fasterxml.jackson.*:* = 2.14.1 com.google.code.findbugs:jsr305 = 3.0.2 com.google.errorprone:error_prone_annotations = 2.3.3 -com.google.guava:* = 31.1-jre +com.google.guava:* = 32.1.1-jre com.github.ben-manes.caffeine:caffeine = 2.9.3 -org.apache.arrow:arrow-vector = 12.0.0 -org.apache.arrow:arrow-memory-netty = 12.0.0 +org.apache.arrow:arrow-vector = 12.0.1 +org.apache.arrow:arrow-memory-netty = 12.0.1 org.roaringbitmap:RoaringBitmap = 0.9.44 io.airlift:aircompressor = 0.24 io.netty:netty-buffer = 4.1.68.Final @@ -22,10 +22,10 @@ javax.xml.bind:jaxb-api = 2.3.1 javax.activation:activation = 1.1.1 org.glassfish.jaxb:jaxb-runtime = 2.3.3 software.amazon.awssdk:* = 2.20.18 -org.projectnessie.nessie:* = 0.60.1 -com.google.cloud:libraries-bom = 24.1.0 -org.scala-lang.modules:scala-collection-compat_2.12 = 2.6.0 -org.scala-lang.modules:scala-collection-compat_2.13 = 2.6.0 +org.projectnessie.nessie:* = 0.65.0 +com.google.cloud:libraries-bom = 26.18.0 +org.scala-lang.modules:scala-collection-compat_2.12 = 2.11.0 +org.scala-lang.modules:scala-collection-compat_2.13 = 2.11.0 com.emc.ecs:object-client-bundle = 3.3.2 org.immutables:value = 2.9.2 net.snowflake:snowflake-jdbc = 3.13.30 @@ -36,7 +36,7 @@ org.junit.vintage:junit-vintage-engine = 5.9.2 org.junit.jupiter:* = 5.9.2 org.mockito:* = 4.11.0 org.apache.tez:* = 0.8.4 -com.adobe.testing:s3mock-junit4 = 2.11.0 +com.adobe.testing:s3mock-junit5 = 2.11.0 org.assertj:assertj-core = 3.24.2 org.xerial:sqlite-jdbc = 3.41.0.0 com.fasterxml.jackson.dataformat:jackson-dataformat-xml = 2.9.9