From 77496d33661c8e412773901a0de4bc4ba0bae138 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Marko=20Ili=C4=87?= Date: Mon, 20 Jan 2025 22:26:59 +0100 Subject: [PATCH 01/28] Fixed 'collated-string-type.md' JSON example. (#3750) #### Which Delta project/connector is this regarding? - [x] Spark - [ ] Standalone - [ ] Flink - [x] Kernel - [ ] Other (fill in here) ## Description Fixed example for storing collation information in metadata in `collated-string-type.md`. JSON representation had incorrect column name and path to StringType. ## How was this patch tested? ## Does this PR introduce _any_ user-facing changes? No. --- protocol_rfcs/collated-string-type.md | 99 ++++++++++++--------------- 1 file changed, 45 insertions(+), 54 deletions(-) diff --git a/protocol_rfcs/collated-string-type.md b/protocol_rfcs/collated-string-type.md index 01ba942b668..99bde61c458 100644 --- a/protocol_rfcs/collated-string-type.md +++ b/protocol_rfcs/collated-string-type.md @@ -42,68 +42,59 @@ This example provides an overview of how collations are stored in the schema. No Example schema ``` -|-- col2: string -|-- col2: map -| |-- keyType: string -| |-- valueType: array -| |-- elementType: map -| |-- keyType: string -| |-- valueType: map -| |-- keyType: string -| |-- valueType: struct -| |-- f1: string +|-- col1: string +|-- col2: array +| |-- elementType: map +| |-- keyType: string +| |-- valueType: struct +| |-- f1: string ``` Schema with collation information ``` { - "type" : "struct", - "fields" : [ - { - "name" : "col1", - "type" : "string", - "metadata": { - "__COLLATIONS": { "col1": "ICU.de_DE" } - } - }, - { - "name" : "col2", - "type" : "map", - "keyType": "string" - "valueType": { - "type": "array" - "elementType": { - "type": "map" - "keyType: { - "type": "array" - "elementType": "string" - }, - "valueType": { - "type": "map", - "keyType": "string", - "valueType": { - "type": "struct", - "fields": [ { - "name": "f1" - "type": "string" - } ], - }, - "metadata": { - "__COLLATIONS": { "f1": "ICU.de_DE" } + "type":"struct", + "fields":[ + { + "name":"col1", + "type":"string", + "metadata":{ + "__COLLATIONS":{ + "col1":"ICU.de_DE" } - } - } - } - "metadata": { - "__COLLATIONS": { - "col1.key": "ICU.en_US", - "col1.value.element.key": "ICU.en_US", - "col1.value.element.value.key": "ICU.en_US" - } + } + }, + { + "name":"col2", + "type":{ + "type":"array", + "elementType":{ + "type":"map", + "keyType":"string", + "valueType":{ + "type":"struct", + "fields":[ + { + "name":"f1", + "type":"string", + "metadata":{ + "__COLLATIONS":{ + "f1":"ICU.de_DE" + } + } + } + ] + } + } + }, + "metadata":{ + "__COLLATIONS":{ + "col2.element.key":"ICU.en_US" + } + } } - } - ] + ] } ``` From 74ea2e2dbcb6c05a1ce08e8bb7352f69bde83ef5 Mon Sep 17 00:00:00 2001 From: Stefan Kandic <154237371+stefankandic@users.noreply.github.com> Date: Tue, 21 Jan 2025 06:12:53 +0100 Subject: [PATCH 02/28] [Spark] Refactor data normalization (#4046) #### Which Delta project/connector is this regarding? - [X] Spark - [ ] Standalone - [ ] Flink - [ ] Kernel - [ ] Other (fill in here) ## Description Refactors the `normalizeData` method to enable support for multiple data normalization methods. The original normalizeData method has been renamed to `normalizeSchema`, while the new `normalizeData` serves as a wrapper that orchestrates various normalization processes. This ensures that by calling `normalizeData`, all necessary normalization steps are executed, providing a single point of entry for all data normalization. ## How was this patch tested? Since this is only a refactor, the existing tests should be sufficient. ## Does this PR introduce _any_ user-facing changes? No. --- .../sql/delta/files/TransactionalWrite.scala | 32 +++++++++++++------ 1 file changed, 23 insertions(+), 9 deletions(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/files/TransactionalWrite.scala b/spark/src/main/scala/org/apache/spark/sql/delta/files/TransactionalWrite.scala index 463ff44083c..ff4942c2b90 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/files/TransactionalWrite.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/files/TransactionalWrite.scala @@ -90,20 +90,34 @@ trait TransactionalWrite extends DeltaLogging { self: OptimisticTransactionImpl } /** - * Normalize the schema of the query, and return the QueryExecution to execute. If the table has + * Used to perform all required normalizations before writing out the data. + * Returns the QueryExecution to execute. + */ + protected def normalizeData( + deltaLog: DeltaLog, + options: Option[DeltaOptions], + data: DataFrame): (QueryExecution, Seq[Attribute], Seq[Constraint], Set[String]) = { + val (normalizedSchema, output, constraints, trackHighWaterMarks) = normalizeSchema( + deltaLog, options, data) + + (normalizedSchema.queryExecution, output, constraints, trackHighWaterMarks) + } + + /** + * Normalize the schema of the query, and returns the updated DataFrame. If the table has * generated columns and users provide these columns in the output, we will also return * constraints that should be respected. If any constraints are returned, the caller should apply * these constraints when writing data. * - * Note: The output attributes of the QueryExecution may not match the attributes we return as the + * Note: The schema of the DataFrame may not match the attributes we return as the * output schema. This is because streaming queries create `IncrementalExecution`, which cannot be * further modified. We can however have the Parquet writer use the physical plan from * `IncrementalExecution` and the output schema provided through the attributes. */ - protected def normalizeData( + protected def normalizeSchema( deltaLog: DeltaLog, options: Option[DeltaOptions], - data: Dataset[_]): (QueryExecution, Seq[Attribute], Seq[Constraint], Set[String]) = { + data: DataFrame): (DataFrame, Seq[Attribute], Seq[Constraint], Set[String]) = { val normalizedData = SchemaUtils.normalizeColumnNames( deltaLog, metadata.schema, data ) @@ -131,12 +145,12 @@ trait TransactionalWrite extends DeltaLogging { self: OptimisticTransactionImpl (normalizedData, Nil, Set[String]()) } val cleanedData = SchemaUtils.dropNullTypeColumns(dataWithDefaultExprs) - val queryExecution = if (cleanedData.schema != dataWithDefaultExprs.schema) { + val finalData = if (cleanedData.schema != dataWithDefaultExprs.schema) { // This must be batch execution as DeltaSink doesn't accept NullType in micro batch DataFrame. // For batch executions, we need to use the latest DataFrame query execution - cleanedData.queryExecution + cleanedData } else if (enforcesDefaultExprs) { - dataWithDefaultExprs.queryExecution + dataWithDefaultExprs } else { assert( normalizedData == dataWithDefaultExprs, @@ -144,7 +158,7 @@ trait TransactionalWrite extends DeltaLogging { self: OptimisticTransactionImpl // Ideally, we should use `normalizedData`. But it may use `QueryExecution` rather than // `IncrementalExecution`. So we use the input `data` and leverage the `nullableOutput` // below to fix the column names. - data.queryExecution + data } val nullableOutput = makeOutputNullable(cleanedData.queryExecution.analyzed.output) val columnMapping = metadata.columnMappingMode @@ -156,7 +170,7 @@ trait TransactionalWrite extends DeltaLogging { self: OptimisticTransactionImpl val mappedOutput = if (columnMapping == NoMapping) nullableOutput else { mapColumnAttributes(nullableOutput, columnMapping) } - (queryExecution, mappedOutput, generatedColumnConstraints, trackHighWaterMarks) + (finalData, mappedOutput, generatedColumnConstraints, trackHighWaterMarks) } protected def checkPartitionColumns( From 7600e8aa86ab7ac03f369286aafa6d28217bf569 Mon Sep 17 00:00:00 2001 From: Ze'ev Maor Date: Tue, 21 Jan 2025 18:59:54 +0200 Subject: [PATCH 03/28] [PROTOCOL] add restriction on path appearing at more once in a snapshot list of `add` files (#4058) add restriction on path appearing at more once in a snapshot list of `add` files per discussion at [4021](https://github.com/delta-io/delta/discussions/4021) --------- Co-authored-by: Lars Kroll --- PROTOCOL.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/PROTOCOL.md b/PROTOCOL.md index a1702d77b26..97bf5442959 100644 --- a/PROTOCOL.md +++ b/PROTOCOL.md @@ -488,6 +488,8 @@ That means specifically that for any commit… - it is **legal** for the same `path` to occur in an `add` action and a `remove` action, but with two different `dvId`s. - it is **legal** for the same `path` to be added and/or removed and also occur in a `cdc` action. - it is **illegal** for the same `path` to be occur twice with different `dvId`s within each set of `add` or `remove` actions. + - it is **illegal** for a `path` to occur in an `add` action that already occurs with a different `dvId` in the list of `add` actions from the snapshot of the version immediately preceeding the commit, unless the commit also contains a remove for the later combination. + - it is **legal** to commit an existing `path` and `dvId` combination again (this allows metadata updates). The `dataChange` flag on either an `add` or a `remove` can be set to `false` to indicate that an action when combined with other actions in the same atomic version only rearranges existing data or adds new statistics. For example, streaming queries that are tailing the transaction log can use this flag to skip actions that would not affect the final results. @@ -825,7 +827,7 @@ A given snapshot of the table can be computed by replaying the events committed - A single `metaData` action - A collection of `txn` actions with unique `appId`s - A collection of `domainMetadata` actions with unique `domain`s. - - A collection of `add` actions with unique `(path, deletionVector.uniqueId)` keys. + - A collection of `add` actions with unique path keys, corresponding to the newest (path, deletionVector.uniqueId) pair encountered for each path. - A collection of `remove` actions with unique `(path, deletionVector.uniqueId)` keys. The intersection of the primary keys in the `add` collection and `remove` collection must be empty. That means a logical file cannot exist in both the `remove` and `add` collections at the same time; however, the same *data file* can exist with *different* DVs in the `remove` collection, as logically they represent different content. The `remove` actions act as _tombstones_, and only exist for the benefit of the VACUUM command. Snapshot reads only return `add` actions on the read path. To achieve the requirements above, related actions from different delta files need to be reconciled with each other: From 8ba774cbcd95f0c753bfa5390211263a589bd00e Mon Sep 17 00:00:00 2001 From: Qiyuan Dong Date: Tue, 21 Jan 2025 18:04:52 +0100 Subject: [PATCH 04/28] [Kernel] Implement row tracking conflict resolution (#4053) #### Which Delta project/connector is this regarding? - [ ] Spark - [ ] Standalone - [ ] Flink - [x] Kernel - [ ] Other (fill in here) ## Description This PR implements transaction conflict resolution for the row tracking feature. Key changes include: 1. Add logic for resolving conflicts for the row tracking metadata domain 2. Reassign `baseRowId` and `defaultRowCommitVersion` for `AddFile` actions during conflict resolution 3. Update the row tracking domain metadata action during conflict resolution ## How was this patch tested? Tests were added in `RowTrackingSuite.scala`. ## Does this PR introduce _any_ user-facing changes? No. --- .../kernel/internal/TransactionImpl.java | 41 ++- .../internal/replay/ConflictChecker.java | 104 +++++++- .../internal/rowtracking/RowTracking.java | 158 ++++++++---- .../kernel/defaults/RowTrackingSuite.scala | 240 +++++++++++++++++- 4 files changed, 463 insertions(+), 80 deletions(-) diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionImpl.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionImpl.java index 1c9638e1c2b..97f07f544dc 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionImpl.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionImpl.java @@ -66,7 +66,7 @@ public class TransactionImpl implements Transaction { private final Optional setTxnOpt; private final boolean shouldUpdateProtocol; private final Clock clock; - private final List domainMetadatas = new ArrayList<>(); + private List domainMetadatas = new ArrayList<>(); private Metadata metadata; private boolean shouldUpdateMetadata; private int maxRetries; @@ -134,6 +134,10 @@ public List getDomainMetadatas() { return domainMetadatas; } + public Protocol getProtocol() { + return protocol; + } + @Override public TransactionCommitResult commit(Engine engine, CloseableIterable dataActions) throws ConcurrentWriteException { @@ -172,11 +176,21 @@ private TransactionCommitResult commitWithRetry( // AddFile actions that do not yet have them. If the row ID high watermark changes, emit a // DomainMetadata action to update it. if (TableFeatures.isRowTrackingSupported(protocol)) { - RowTracking.createNewHighWaterMarkIfNeeded(readSnapshot, dataActions) - .ifPresent(domainMetadatas::add); + domainMetadatas = + RowTracking.updateRowIdHighWatermarkIfNeeded( + readSnapshot, + protocol, + Optional.empty() /* winningTxnRowIdHighWatermark */, + dataActions, + domainMetadatas); dataActions = RowTracking.assignBaseRowIdAndDefaultRowCommitVersion( - readSnapshot, commitAsVersion, dataActions); + readSnapshot, + protocol, + Optional.empty() /* winningTxnRowIdHighWatermark */, + Optional.empty() /* prevCommitVersion */, + commitAsVersion, + dataActions); } int numTries = 0; @@ -191,8 +205,11 @@ private TransactionCommitResult commitWithRetry( "Concurrent write detected when committing as version = {}.", commitAsVersion); if (numTries < maxRetries) { // only try and resolve conflicts if we're going to retry - commitAsVersion = - resolveConflicts(engine, commitAsVersion, attemptCommitInfo, numTries); + TransactionRebaseState rebaseState = + resolveConflicts(engine, commitAsVersion, attemptCommitInfo, numTries, dataActions); + commitAsVersion = rebaseState.getLatestVersion() + 1; + dataActions = rebaseState.getUpdatedDataActions(); + domainMetadatas = rebaseState.getUpdatedDomainMetadatas(); } } numTries++; @@ -206,15 +223,19 @@ private TransactionCommitResult commitWithRetry( throw new ConcurrentWriteException(); } - private long resolveConflicts( - Engine engine, long commitAsVersion, CommitInfo attemptCommitInfo, int numTries) { + private TransactionRebaseState resolveConflicts( + Engine engine, + long commitAsVersion, + CommitInfo attemptCommitInfo, + int numTries, + CloseableIterable dataActions) { logger.info( "Table {}, trying to resolve conflicts and retry commit. (tries/maxRetries: {}/{})", dataPath, numTries, maxRetries); TransactionRebaseState rebaseState = - ConflictChecker.resolveConflicts(engine, readSnapshot, commitAsVersion, this); + ConflictChecker.resolveConflicts(engine, readSnapshot, commitAsVersion, this, dataActions); long newCommitAsVersion = rebaseState.getLatestVersion() + 1; checkArgument( commitAsVersion < newCommitAsVersion, @@ -227,7 +248,7 @@ private long resolveConflicts( updateMetadataWithICTIfRequired( engine, updatedInCommitTimestamp, rebaseState.getLatestVersion()); attemptCommitInfo.setInCommitTimestamp(updatedInCommitTimestamp); - return newCommitAsVersion; + return rebaseState; } private void updateMetadata(Metadata metadata) { diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/ConflictChecker.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/ConflictChecker.java index 33808db8be4..828da78d77b 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/ConflictChecker.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/ConflictChecker.java @@ -26,14 +26,18 @@ import io.delta.kernel.data.ColumnVector; import io.delta.kernel.data.ColumnarBatch; +import io.delta.kernel.data.Row; import io.delta.kernel.engine.Engine; import io.delta.kernel.exceptions.ConcurrentWriteException; import io.delta.kernel.internal.*; import io.delta.kernel.internal.actions.CommitInfo; import io.delta.kernel.internal.actions.DomainMetadata; import io.delta.kernel.internal.actions.SetTransaction; +import io.delta.kernel.internal.rowtracking.RowTracking; +import io.delta.kernel.internal.rowtracking.RowTrackingMetadataDomain; import io.delta.kernel.internal.util.DomainMetadataUtils; import io.delta.kernel.internal.util.FileNames; +import io.delta.kernel.utils.CloseableIterable; import io.delta.kernel.utils.CloseableIterator; import io.delta.kernel.utils.FileStatus; import java.io.*; @@ -61,11 +65,20 @@ public class ConflictChecker { // Losing transaction private final TransactionImpl transaction; private final long attemptVersion; + private final CloseableIterable attemptDataActions; - private ConflictChecker(SnapshotImpl snapshot, TransactionImpl transaction, long attemptVersion) { + // Helper states during conflict resolution + private Optional lastWinningRowIdHighWatermark = Optional.empty(); + + private ConflictChecker( + SnapshotImpl snapshot, + TransactionImpl transaction, + long attemptVersion, + CloseableIterable dataActions) { this.snapshot = snapshot; this.transaction = transaction; this.attemptVersion = attemptVersion; + this.attemptDataActions = dataActions; } /** @@ -76,15 +89,22 @@ private ConflictChecker(SnapshotImpl snapshot, TransactionImpl transaction, long * @param snapshot {@link SnapshotImpl} of the table when the losing transaction has started * @param transaction {@link TransactionImpl} that encountered the conflict (a.k.a the losing * transaction) + * @param dataActions {@link CloseableIterable} of data actions that the losing transaction is + * trying to commit * @return {@link TransactionRebaseState} that the losing transaction needs to rebase against * @throws ConcurrentWriteException if there are logical conflicts between the losing transaction * and the winning transactions that cannot be resolved. */ public static TransactionRebaseState resolveConflicts( - Engine engine, SnapshotImpl snapshot, long attemptVersion, TransactionImpl transaction) + Engine engine, + SnapshotImpl snapshot, + long attemptVersion, + TransactionImpl transaction, + CloseableIterable dataActions) throws ConcurrentWriteException { checkArgument(transaction.isBlindAppend(), "Current support is for blind appends only."); - return new ConflictChecker(snapshot, transaction, attemptVersion).resolveConflicts(engine); + return new ConflictChecker(snapshot, transaction, attemptVersion, dataActions) + .resolveConflicts(engine); } public TransactionRebaseState resolveConflicts(Engine engine) throws ConcurrentWriteException { @@ -120,30 +140,63 @@ public TransactionRebaseState resolveConflicts(Engine engine) throws ConcurrentW throw new UncheckedIOException("Error reading actions from winning commits.", ioe); } + // Initialize updated actions for the next commit attempt with the current attempt's actions + CloseableIterable updatedDataActions = attemptDataActions; + List updatedDomainMetadatas = transaction.getDomainMetadatas(); + + if (TableFeatures.isRowTrackingSupported(transaction.getProtocol())) { + updatedDomainMetadatas = + RowTracking.updateRowIdHighWatermarkIfNeeded( + snapshot, + transaction.getProtocol(), + lastWinningRowIdHighWatermark, + attemptDataActions, + transaction.getDomainMetadatas()); + updatedDataActions = + RowTracking.assignBaseRowIdAndDefaultRowCommitVersion( + snapshot, + transaction.getProtocol(), + lastWinningRowIdHighWatermark, + Optional.of(attemptVersion), + lastWinningVersion + 1, + attemptDataActions); + } + // if we get here, we have successfully rebased (i.e no logical conflicts) // against the winning transactions return new TransactionRebaseState( lastWinningVersion, getLastCommitTimestamp( - engine, lastWinningVersion, lastWinningTxn, winningCommitInfoOpt.get())); + engine, lastWinningVersion, lastWinningTxn, winningCommitInfoOpt.get()), + updatedDataActions, + updatedDomainMetadatas); } /** * Class containing the rebase state from winning transactions that the current transaction needs * to rebase against before attempting the commit. * - *

Currently, the rebase state is just the latest winning version of the table. In future once - * we start supporting read-after-write, domain metadata, row tracking, etc., we will have more - * state to add. For example read-after-write will need to know the files deleted in the winning - * transactions to make sure the same files are not deleted by the current (losing) transaction. + *

Currently, the rebase state is just the latest winning version of the table plus the updated + * data actions and domainMetadata actions to commit. In future once we start supporting + * read-after-write, row tracking, etc., we will have more state to add. For example + * read-after-write will need to know the files deleted in the winning transactions to make sure + * the same files are not deleted by the current (losing) transaction. */ public static class TransactionRebaseState { private final long latestVersion; private final long latestCommitTimestamp; + private final CloseableIterable updatedDataActions; + private final List updatedDomainMetadatas; - public TransactionRebaseState(long latestVersion, long latestCommitTimestamp) { + public TransactionRebaseState( + long latestVersion, + long latestCommitTimestamp, + CloseableIterable updatedDataActions, + List updatedDomainMetadatas) { this.latestVersion = latestVersion; this.latestCommitTimestamp = latestCommitTimestamp; + this.updatedDataActions = updatedDataActions; + this.updatedDomainMetadatas = updatedDomainMetadatas; } /** @@ -165,6 +218,14 @@ public long getLatestVersion() { public long getLatestCommitTimestamp() { return latestCommitTimestamp; } + + public CloseableIterable getUpdatedDataActions() { + return updatedDataActions; + } + + public List getUpdatedDomainMetadatas() { + return updatedDomainMetadatas; + } } /** @@ -209,8 +270,9 @@ private void handleMetadata(ColumnVector metadataVector) { * * * @param domainMetadataVector domainMetadata rows from the winning transactions + * @return a map of domain name to {@link DomainMetadata} from the winning transaction */ - private void handleDomainMetadata(ColumnVector domainMetadataVector) { + private Map handleDomainMetadata(ColumnVector domainMetadataVector) { // Build a domain metadata map from the winning transaction. Map winningTxnDomainMetadataMap = new HashMap<>(); DomainMetadataUtils.populateDomainMetadataMap( @@ -223,11 +285,27 @@ private void handleDomainMetadata(ColumnVector domainMetadataVector) { DomainMetadata winningTxnDM = winningTxnDomainMetadataMap.get(domainName); if (winningTxnDM != null) { // Conflict - check if the conflict can be resolved. - // Currently, we don't have any domain-specific way of resolving the conflict. - // Domain-specific ways of resolving the conflict can be added here (e.g. for Row Tracking). - throw concurrentDomainMetadataAction(currentTxnDM, winningTxnDM); + // Domain-specific ways of resolving the conflict can be added here. + switch (domainName) { + case RowTrackingMetadataDomain.DOMAIN_NAME: + // We keep updating the new row ID high watermark we have seen from all winning txns. + // The latest one will be used to reassign row IDs later + final long winningRowIdHighWatermark = + RowTrackingMetadataDomain.fromJsonConfiguration(winningTxnDM.getConfiguration()) + .getRowIdHighWaterMark(); + checkState( + !lastWinningRowIdHighWatermark.isPresent() + || lastWinningRowIdHighWatermark.get() <= winningRowIdHighWatermark, + "row ID high watermark should be monotonically increasing"); + this.lastWinningRowIdHighWatermark = Optional.of(winningRowIdHighWatermark); + break; + default: + throw concurrentDomainMetadataAction(currentTxnDM, winningTxnDM); + } } } + + return winningTxnDomainMetadataMap; } /** diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/rowtracking/RowTracking.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/rowtracking/RowTracking.java index d6b1cd53ba5..9e54b6e1fa2 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/rowtracking/RowTracking.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/rowtracking/RowTracking.java @@ -25,8 +25,10 @@ import io.delta.kernel.utils.CloseableIterable; import io.delta.kernel.utils.CloseableIterator; import java.io.IOException; +import java.util.List; import java.util.Optional; import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; /** A collection of helper methods for working with row tracking. */ public class RowTracking { @@ -35,43 +37,68 @@ private RowTracking() { } /** - * Assigns base row IDs and default row commit versions to {@link AddFile} actions in the provided - * {@code dataActions}. This method should be called when processing data actions during commit - * preparation and before the actual commit. + * Assigns or reassigns baseRowIds and defaultRowCommitVersions to {@link AddFile} actions in the + * provided {@code dataActions}. This method should be invoked only when the 'rowTracking' feature + * is supported and is used in two scenarios: * - *

This method should be called exactly once per transaction during commit preparation, i.e., - * not for each commit attempt. And it should only be called when the 'rowTracking' feature is - * supported. + *

    + *
  1. Initial Assignment: Assigns row tracking fields to AddFile actions during commit + * preparation before they are committed. + *
  2. Conflict Resolution: Updates row tracking fields when a transaction conflict occurs. + * Since the losing transaction gets a new commit version and winning transactions may have + * increased the row ID high watermark, this method reassigns the fields for the losing + * transaction using the latest state from winning transactions before retrying the commit. + *
* - *

For {@link AddFile} actions missing a base row ID, assigns the current row ID high watermark - * plus 1. The high watermark is then incremented by the number of records in the file. For - * actions missing a default row commit version, assigns the specified commit version. - * - * @param snapshot the snapshot of the table that this transaction is reading at - * @param commitVersion the version of the commit for default row commit version assignment - * @param dataActions the {@link CloseableIterable} of data actions to process - * @return an {@link CloseableIterable} of data actions with base row IDs and default row commit - * versions assigned + * @param txnReadSnapshot the snapshot of the table that this transaction is reading from + * @param txnProtocol the (updated, if any) protocol that will result from this txn + * @param winningTxnRowIdHighWatermark the latest row ID high watermark from the winning + * transactions. Should be empty for initial assignment and present for conflict resolution. + * @param prevCommitVersion the commit version used by this transaction in the previous commit + * attempt. Should be empty for initial assignment and present for conflict resolution. + * @param currCommitVersion the transaction's (latest) commit version + * @param txnDataActions a {@link CloseableIterable} of data actions this txn is trying to commit + * @return a {@link CloseableIterable} of data actions with baseRowIds and + * defaultRowCommitVersions assigned or reassigned */ public static CloseableIterable assignBaseRowIdAndDefaultRowCommitVersion( - SnapshotImpl snapshot, long commitVersion, CloseableIterable dataActions) { + SnapshotImpl txnReadSnapshot, + Protocol txnProtocol, + Optional winningTxnRowIdHighWatermark, + Optional prevCommitVersion, + long currCommitVersion, + CloseableIterable txnDataActions) { checkArgument( - TableFeatures.isRowTrackingSupported(snapshot.getProtocol()), + TableFeatures.isRowTrackingSupported(txnProtocol), "Base row ID and default row commit version are assigned " + "only when feature 'rowTracking' is supported."); return new CloseableIterable() { @Override public void close() throws IOException { - dataActions.close(); + txnDataActions.close(); } @Override public CloseableIterator iterator() { - // Used to keep track of the current high watermark as we iterate through the data actions. - // Use an AtomicLong to allow for updating the high watermark in the lambda. - final AtomicLong currRowIdHighWatermark = new AtomicLong(readRowIdHighWaterMark(snapshot)); - return dataActions + // The row ID high watermark from the snapshot of the table that this transaction is reading + // at. Any baseRowIds higher than this watermark are assigned by this transaction. + final long prevRowIdHighWatermark = readRowIdHighWaterMark(txnReadSnapshot); + + // Used to track the current high watermark as we iterate through the data actions and + // assign baseRowIds. Use an AtomicLong to allow for updating in the lambda. + final AtomicLong currRowIdHighWatermark = + new AtomicLong(winningTxnRowIdHighWatermark.orElse(prevRowIdHighWatermark)); + + // The row ID high watermark must increase monotonically, so the winning transaction's high + // watermark must be greater than or equal to the high watermark from the current + // transaction's read snapshot. + checkArgument( + currRowIdHighWatermark.get() >= prevRowIdHighWatermark, + "The current row ID high watermark must be greater than or equal to " + + "the high watermark from the transaction's read snapshot"); + + return txnDataActions .iterator() .map( row -> { @@ -82,16 +109,20 @@ public CloseableIterator iterator() { AddFile addFile = new AddFile(row.getStruct(SingleAction.ADD_FILE_ORDINAL)); - // Assign base row ID if missing - if (!addFile.getBaseRowId().isPresent()) { - final long numRecords = getNumRecordsOrThrow(addFile); + // Assign a baseRowId if not present, or update it if previously assigned + // by this transaction + if (!addFile.getBaseRowId().isPresent() + || addFile.getBaseRowId().get() > prevRowIdHighWatermark) { addFile = addFile.withNewBaseRowId(currRowIdHighWatermark.get() + 1L); - currRowIdHighWatermark.addAndGet(numRecords); + currRowIdHighWatermark.addAndGet(getNumRecordsOrThrow(addFile)); } - // Assign default row commit version if missing - if (!addFile.getDefaultRowCommitVersion().isPresent()) { - addFile = addFile.withNewDefaultRowCommitVersion(commitVersion); + // Assign a defaultRowCommitVersion if not present, or update it if previously + // assigned by this transaction + if (!addFile.getDefaultRowCommitVersion().isPresent() + || addFile.getDefaultRowCommitVersion().get() + == prevCommitVersion.orElse(-1L)) { + addFile = addFile.withNewDefaultRowCommitVersion(currCommitVersion); } return SingleAction.createAddFileSingleAction(addFile.toRow()); @@ -101,38 +132,71 @@ public CloseableIterator iterator() { } /** - * Returns a {@link DomainMetadata} action if the row ID high watermark has changed due to newly - * processed {@link AddFile} actions. + * Inserts or updates the {@link DomainMetadata} action reflecting the new row ID high watermark + * when this transaction adds rows and pushed it higher. * - *

This method should be called during commit preparation to prepare the domain metadata - * actions for commit. It should be called only when the 'rowTracking' feature is supported. + *

This method should only be called when the 'rowTracking' feature is supported. Similar to + * {@link #assignBaseRowIdAndDefaultRowCommitVersion}, it should be called during the initial row + * ID assignment or conflict resolution to reflect the change to the row ID high watermark. * - * @param snapshot the snapshot of the table that this transaction is reading at - * @param dataActions the iterable of data actions that may update the high watermark + * @param txnReadSnapshot the snapshot of the table that this transaction is reading at + * @param txnProtocol the (updated, if any) protocol that will result from this txn + * @param winningTxnRowIdHighWatermark the latest row ID high watermark from the winning + * transaction. Should be empty for initial assignment and present for conflict resolution. + * @param txnDataActions a {@link CloseableIterable} of data actions this txn is trying to commit + * @param txnDomainMetadatas a list of domain metadata actions this txn is trying to commit + * @return Updated list of domain metadata actions for commit */ - public static Optional createNewHighWaterMarkIfNeeded( - SnapshotImpl snapshot, CloseableIterable dataActions) { + public static List updateRowIdHighWatermarkIfNeeded( + SnapshotImpl txnReadSnapshot, + Protocol txnProtocol, + Optional winningTxnRowIdHighWatermark, + CloseableIterable txnDataActions, + List txnDomainMetadatas) { checkArgument( - TableFeatures.isRowTrackingSupported(snapshot.getProtocol()), + TableFeatures.isRowTrackingSupported(txnProtocol), "Row ID high watermark is updated only when feature 'rowTracking' is supported."); - final long prevRowIdHighWatermark = readRowIdHighWaterMark(snapshot); - // Use an AtomicLong to allow for updating the high watermark in the lambda - final AtomicLong newRowIdHighWatermark = new AtomicLong(prevRowIdHighWatermark); + // Filter out existing row tracking domainMetadata action, if any + List nonRowTrackingDomainMetadatas = + txnDomainMetadatas.stream() + .filter(dm -> !dm.getDomain().equals(RowTrackingMetadataDomain.DOMAIN_NAME)) + .collect(Collectors.toList()); + + // The row ID high watermark from the snapshot of the table that this transaction is reading at. + // Any baseRowIds higher than this watermark are assigned by this transaction. + final long prevRowIdHighWatermark = readRowIdHighWaterMark(txnReadSnapshot); + + // Tracks the new row ID high watermark as we iterate through data actions and counting new rows + // added in this transaction. + final AtomicLong currRowIdHighWatermark = + new AtomicLong(winningTxnRowIdHighWatermark.orElse(prevRowIdHighWatermark)); - dataActions.forEach( + // The row ID high watermark must increase monotonically, so the winning transaction's high + // watermark (if present) must be greater than or equal to the high watermark from the + // current transaction's read snapshot. + checkArgument( + currRowIdHighWatermark.get() >= prevRowIdHighWatermark, + "The current row ID high watermark must be greater than or equal to " + + "the high watermark from the transaction's read snapshot"); + + txnDataActions.forEach( row -> { if (!row.isNullAt(SingleAction.ADD_FILE_ORDINAL)) { AddFile addFile = new AddFile(row.getStruct(SingleAction.ADD_FILE_ORDINAL)); - if (!addFile.getBaseRowId().isPresent()) { - newRowIdHighWatermark.addAndGet(getNumRecordsOrThrow(addFile)); + if (!addFile.getBaseRowId().isPresent() + || addFile.getBaseRowId().get() > prevRowIdHighWatermark) { + currRowIdHighWatermark.addAndGet(getNumRecordsOrThrow(addFile)); } } }); - return (newRowIdHighWatermark.get() != prevRowIdHighWatermark) - ? Optional.of(new RowTrackingMetadataDomain(newRowIdHighWatermark.get()).toDomainMetadata()) - : Optional.empty(); + if (currRowIdHighWatermark.get() != prevRowIdHighWatermark) { + nonRowTrackingDomainMetadatas.add( + new RowTrackingMetadataDomain(currRowIdHighWatermark.get()).toDomainMetadata()); + } + + return nonRowTrackingDomainMetadatas; } /** diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/RowTrackingSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/RowTrackingSuite.scala index f40ad58c0e8..f5406ce8869 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/RowTrackingSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/RowTrackingSuite.scala @@ -15,20 +15,23 @@ */ package io.delta.kernel.defaults -import io.delta.kernel.data.Row +import io.delta.kernel.data.{FilteredColumnarBatch, Row} import io.delta.kernel.defaults.internal.parquet.ParquetSuiteBase import io.delta.kernel.engine.Engine -import io.delta.kernel.exceptions.KernelException +import io.delta.kernel.exceptions.{KernelException, ProtocolChangedException} import io.delta.kernel.expressions.Literal import io.delta.kernel.internal.{InternalScanFileUtils, SnapshotImpl, TableImpl} import io.delta.kernel.internal.actions.{AddFile, Protocol, SingleAction} import io.delta.kernel.internal.util.Utils.toCloseableIterator import io.delta.kernel.internal.rowtracking.RowTrackingMetadataDomain import io.delta.kernel.internal.util.VectorUtils +import io.delta.kernel.internal.util.VectorUtils.stringStringMapValue import io.delta.kernel.types.StructType import io.delta.kernel.types.LongType.LONG import io.delta.kernel.utils.CloseableIterable import io.delta.kernel.utils.CloseableIterable.{emptyIterable, inMemoryIterable} +import org.apache.hadoop.fs.Path +import org.apache.spark.sql.delta.DeltaLog import java.util import java.util.{Collections, Optional} @@ -110,6 +113,11 @@ class RowTrackingSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase { assert(rowTrackingMetadataDomain.get().getRowIdHighWaterMark === expectedValue) } + private def prepareDataForCommit(data: Seq[FilteredColumnarBatch]*) + : Seq[(Map[String, Literal], Seq[FilteredColumnarBatch])] = { + data.map(Map.empty[String, Literal] -> _).toIndexedSeq + } + test("Base row IDs/default row commit versions are assigned to AddFile actions") { withTempDirAndEngine { (tablePath, engine) => createTableWithRowTrackingSupported(engine, tablePath) @@ -122,7 +130,7 @@ class RowTrackingSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase { val commitVersion = appendData( engine, tablePath, - data = Seq(dataBatch1, dataBatch2, dataBatch3).map(Map.empty[String, Literal] -> _) + data = prepareDataForCommit(dataBatch1, dataBatch2, dataBatch3) ).getVersion verifyBaseRowIDs(engine, tablePath, Seq(0, 100, 300)) @@ -150,7 +158,7 @@ class RowTrackingSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase { val commitVersion2 = appendData( engine, tablePath, - data = Seq(dataBatch2).map(Map.empty[String, Literal] -> _) + data = prepareDataForCommit(dataBatch2) ).getVersion verifyBaseRowIDs(engine, tablePath, Seq(0, 100)) @@ -170,13 +178,13 @@ class RowTrackingSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase { val commitVersion1 = appendData( engine, tablePath, - data = Seq(dataBatch1).map(Map.empty[String, Literal] -> _) + data = prepareDataForCommit(dataBatch1) ).getVersion val commitVersion2 = appendData( engine, tablePath, - data = Seq(dataBatch2).map(Map.empty[String, Literal] -> _) + data = prepareDataForCommit(dataBatch2) ).getVersion // Checkpoint the table @@ -187,7 +195,7 @@ class RowTrackingSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase { val commitVersion3 = appendData( engine, tablePath, - data = Seq(dataBatch3).map(Map.empty[String, Literal] -> _) + data = prepareDataForCommit(dataBatch3) ).getVersion verifyBaseRowIDs(engine, tablePath, Seq(0, 100, 300)) @@ -246,7 +254,7 @@ class RowTrackingSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase { appendData( engine, tablePath, - data = Seq(dataBatch1).map(Map.empty[String, Literal] -> _) + data = prepareDataForCommit(dataBatch1) ).getVersion } @@ -275,7 +283,7 @@ class RowTrackingSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase { engine, tablePath, schema = schema, - data = Seq(dataBatch1, dataBatch2, dataBatch3).map(Map.empty[String, Literal] -> _) + data = prepareDataForCommit(dataBatch1, dataBatch2, dataBatch3) ).getVersion // version 2 // Verify the table state @@ -327,7 +335,7 @@ class RowTrackingSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase { engine, tablePath, schema = schema, - data = Seq(dataBatch1, dataBatch2, dataBatch3).map(Map.empty[String, Literal] -> _) + data = prepareDataForCommit(dataBatch1, dataBatch2, dataBatch3) ) // version 3 // Verify the table state @@ -337,4 +345,216 @@ class RowTrackingSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase { } }) } + + private def validateConflictResolution( + engine: Engine, + tablePath: String, + dataSizeTxn1: Int, + dataSizeTxn2: Int, + useSparkTxn2: Boolean = false, + dataSizeTxn3: Int, + useSparkTxn3: Boolean = false): Unit = { + + /** + * Txn1: the current transaction that commits later than winning transactions. + * Txn2: the winning transaction that was committed first. + * Txn3: the winning transaction that was committed second. + * + * Note tx is the timestamp. + * + * t1 ------------------------ Txn1 starts. + * t2 ------- Txn2 starts. + * t3 ------- Txn2 commits. + * t4 ------- Txn3 starts. + * t5 ------- Txn3 commits. + * t6 ------------------------ Txn1 commits. + */ + val schema = new StructType().add("id", LONG) + + // Create a row-tracking-supported table and bump the row ID high watermark to the initial value + createTableWithRowTrackingSupported(engine, tablePath, schema) + val initDataSize = 100L + val dataBatch = generateData(schema, Seq.empty, Map.empty, initDataSize.toInt, 1) + val v0 = appendData(engine, tablePath, data = prepareDataForCommit(dataBatch)).getVersion + + var expectedBaseRowIDs = Seq(0L) + var expectedDefaultRowCommitVersion = Seq(v0) + var expectedHighWatermark = initDataSize - 1 + + def verifyRowTrackingStates(): Unit = { + verifyBaseRowIDs(engine, tablePath, expectedBaseRowIDs) + verifyDefaultRowCommitVersion(engine, tablePath, expectedDefaultRowCommitVersion) + verifyHighWatermark(engine, tablePath, expectedHighWatermark) + } + verifyRowTrackingStates() + + // Create txn1 but don't commit it yet + val txn1 = createTxn(engine, tablePath) + + // Create and commit txn2 + if (dataSizeTxn2 > 0) { + val v = if (useSparkTxn2) { + spark.range(0, dataSizeTxn2).write.format("delta").mode("append").save(tablePath) + DeltaLog.forTable(spark, new Path(tablePath)).snapshot.version + } else { + val dataBatchTxn2 = generateData(schema, Seq.empty, Map.empty, dataSizeTxn2, 1) + appendData(engine, tablePath, data = prepareDataForCommit(dataBatchTxn2)).getVersion + } + expectedBaseRowIDs = expectedBaseRowIDs ++ Seq(initDataSize) + expectedDefaultRowCommitVersion = expectedDefaultRowCommitVersion ++ Seq(v) + expectedHighWatermark = initDataSize + dataSizeTxn2 - 1 + } else { + createTxn(engine, tablePath).commit(engine, emptyIterable()) + } + verifyRowTrackingStates() + + // Create and commit txn3 + if (dataSizeTxn3 > 0) { + val v = if (useSparkTxn3) { + spark.range(0, dataSizeTxn3).write.format("delta").mode("append").save(tablePath) + DeltaLog.forTable(spark, new Path(tablePath)).snapshot.version + } else { + val dataBatchTxn3 = generateData(schema, Seq.empty, Map.empty, dataSizeTxn3, 1) + appendData(engine, tablePath, data = prepareDataForCommit(dataBatchTxn3)).getVersion + } + expectedBaseRowIDs = expectedBaseRowIDs ++ Seq(initDataSize + dataSizeTxn2) + expectedDefaultRowCommitVersion = expectedDefaultRowCommitVersion ++ Seq(v) + expectedHighWatermark = initDataSize + dataSizeTxn2 + dataSizeTxn3 - 1 + } else { + createTxn(engine, tablePath).commit(engine, emptyIterable()) + } + verifyRowTrackingStates() + + // Commit txn1 + if (dataSizeTxn1 > 0) { + val dataBatchTxn1 = generateData(schema, Seq.empty, Map.empty, dataSizeTxn1, 1) + val v = commitAppendData(engine, txn1, prepareDataForCommit(dataBatchTxn1)).getVersion + expectedBaseRowIDs = expectedBaseRowIDs ++ Seq(initDataSize + dataSizeTxn2 + dataSizeTxn3) + expectedDefaultRowCommitVersion = expectedDefaultRowCommitVersion ++ Seq(v) + expectedHighWatermark = initDataSize + dataSizeTxn2 + dataSizeTxn3 + dataSizeTxn1 - 1 + } else { + txn1.commit(engine, emptyIterable()) + } + verifyRowTrackingStates() + } + + test("Conflict resolution - two concurrent txns both added new files") { + withTempDirAndEngine((tablePath, engine) => { + validateConflictResolution( + engine, + tablePath, + dataSizeTxn1 = 200, + dataSizeTxn2 = 300, + dataSizeTxn3 = 400 + ) + }) + } + + test("Conflict resolution - only one of the two concurrent txns added new files") { + withTempDirAndEngine((tablePath, engine) => { + validateConflictResolution( + engine, + tablePath, + dataSizeTxn1 = 200, + dataSizeTxn2 = 300, + dataSizeTxn3 = 0 + ) + }) + withTempDirAndEngine((tablePath, engine) => { + validateConflictResolution( + engine, + tablePath, + dataSizeTxn1 = 200, + dataSizeTxn2 = 0, + dataSizeTxn3 = 300 + ) + }) + } + + test("Conflict resolution - none of the two concurrent txns added new files") { + withTempDirAndEngine((tablePath, engine) => { + validateConflictResolution( + engine, + tablePath, + dataSizeTxn1 = 200, + dataSizeTxn2 = 0, + dataSizeTxn3 = 0 + ) + }) + } + + test("Conflict resolution - the current txn didn't add new files") { + withTempDirAndEngine((tablePath, engine) => { + validateConflictResolution( + engine, + tablePath, + dataSizeTxn1 = 0, + dataSizeTxn2 = 200, + dataSizeTxn3 = 300 + ) + }) + } + + test( + "Conflict resolution - two concurrent txns were commited by delta-spark " + + "and both added new files" + ) { + withTempDirAndEngine((tablePath, engine) => { + validateConflictResolution( + engine, + tablePath, + dataSizeTxn1 = 200, + dataSizeTxn2 = 300, + useSparkTxn2 = true, + dataSizeTxn3 = 400, + useSparkTxn3 = true + ) + }) + } + + test("Conflict resolution - Row tracking is made supported by a concurrent txn") { + withTempDirAndEngine((tablePath, engine) => { + // Create a table without row tracking + createTxn(engine, tablePath, isNewTable = true, testSchema, Seq.empty) + .commit(engine, emptyIterable()) + + // Create a txn but don't commit it yet + val dataBatch1 = generateData(testSchema, Seq.empty, Map.empty, 100, 1) + val txn1 = createTxn(engine, tablePath, isNewTable = false, testSchema, Seq.empty) + + // A concurrent txn makes row tracking supported + setWriterFeatureSupported(engine, tablePath, testSchema, Seq("rowTracking", "domainMetadata")) + + // Commit txn1 and expect failure + val e = intercept[ProtocolChangedException] { + commitAppendData( + engine, + txn1, + prepareDataForCommit(dataBatch1) + ) + } + }) + } + + test("Conflict resolution - Row tracking is made unsupported by a concurrent txn") { + withTempDirAndEngine((tablePath, engine) => { + createTableWithRowTrackingSupported(engine, tablePath) + + // Create a txn but don't commit it yet + val dataBatch1 = generateData(testSchema, Seq.empty, Map.empty, 100, 1) + val txn1 = createTxn(engine, tablePath, isNewTable = false, testSchema, Seq.empty) + + // A concurrent txn makes row tracking unsupported + setWriterFeatureSupported(engine, tablePath, testSchema, Seq()) + + // Commit txn1 and expect failure + val e = intercept[ProtocolChangedException] { + commitAppendData( + engine, + txn1, + prepareDataForCommit(dataBatch1) + ) + } + }) + } } From cac6e2786a290b4af41944eaf58e5ab1a7bf7108 Mon Sep 17 00:00:00 2001 From: Venki Korukanti Date: Tue, 21 Jan 2025 15:21:22 -0800 Subject: [PATCH 05/28] [DynamoDBLogStore] Update setup requirements in docs (#4079) Just a docs update to clarify the setup. --- docs/source/delta-storage.md | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/docs/source/delta-storage.md b/docs/source/delta-storage.md index ae41dc987cf..e18996daa73 100644 --- a/docs/source/delta-storage.md +++ b/docs/source/delta-storage.md @@ -133,6 +133,17 @@ This mode supports concurrent writes to S3 from multiple clusters and has to be #### Requirements (S3 multi-cluster) - All of the requirements listed in [_](#requirements-s3-single-cluster) section - In additon to S3 credentials, you also need DynamoDB operating permissions +- To ensure proper coordination across clusters, it's crucial to maintain consistency in how tables are referenced. Always use the same scheme (e.g., all cluster refer the table path with either s3a:// or s3:// but not a combination of the two) and maintain case sensitivity when referring to a table from different clusters. For example, use s3a://mybucket/mytable consistently across all clusters. This consistency is vital because DynamoDB relies on the table path as a key to achieve put-if-absent semantics, and inconsistent references can lead to coordination issues. If the table is registered in a catalog, verify (use `DESCRIBE FORMATTED` or other equivalent commands) that the registered path matches the path used for writes from other clusters. By adhering to these guidelines, you can minimize the risk of coordination problems and ensure smooth operation across multiple clusters. + - In case the table should be referred using `s3` scheme Delta-Spark connector, following configs are needed: + + + ``` + "spark.delta.logStore.s3.impl" = "io.delta.storage.S3DynamoDBLogStore" + "spark.io.delta.storage.S3DynamoDBLogStore.ddb.region" = "" + "spark.io.delta.storage.S3DynamoDBLogStore.ddb.tableName" = "" + "spark.hadoop.fs.s3.impl"="org.apache.hadoop.fs.s3a.S3AFileSystem" + # and any other config key name that has `s3a` in it should be changed to `s3` + ``` #### Quickstart (S3 multi-cluster) @@ -554,4 +565,4 @@ spark.read.format("delta").load("cos://.service/ replace:: Delta Lake -.. replace:: Apache Spark \ No newline at end of file +.. replace:: Apache Spark From b5edef29b988ecfa08d369b93212c0760cf5269a Mon Sep 17 00:00:00 2001 From: Hao Jiang Date: Wed, 22 Jan 2025 07:06:10 -0800 Subject: [PATCH 06/28] [UniForm] Allow Delta to convert Iceberg stats during UniForm conversion (#4078) #### Which Delta project/connector is this regarding? - [x] Spark - [ ] Standalone - [ ] Flink - [ ] Kernel - [ ] Other (fill in here) ## Description We allow Delta to read and convert Iceberg stats instead of collecting stats from Parquet during UniForm conversion. ## How was this patch tested? UT & E2E ## Does this PR introduce _any_ user-facing changes? No --- .../spark/sql/delta/IcebergFileManifest.scala | 9 +- .../spark/sql/delta/IcebergStatsUtils.scala | 192 ++++++++++++++ .../apache/spark/sql/delta/IcebergTable.scala | 19 +- .../spark/sql/delta/CloneIcebergSuite.scala | 14 + .../convert/IcebergStatsUtilsSuite.scala | 242 ++++++++++++++++++ .../commands/ConvertToDeltaCommand.scala | 2 +- .../delta/commands/convert/ConvertUtils.scala | 22 +- .../delta/commands/convert/interfaces.scala | 4 +- .../sql/delta/sources/DeltaSQLConf.scala | 7 + 9 files changed, 498 insertions(+), 13 deletions(-) create mode 100644 iceberg/src/main/scala/org/apache/spark/sql/delta/IcebergStatsUtils.scala create mode 100644 iceberg/src/test/scala/org/apache/spark/sql/delta/commands/convert/IcebergStatsUtilsSuite.scala diff --git a/iceberg/src/main/scala/org/apache/spark/sql/delta/IcebergFileManifest.scala b/iceberg/src/main/scala/org/apache/spark/sql/delta/IcebergFileManifest.scala index ccf95f30a3e..63c0e0cbe1d 100644 --- a/iceberg/src/main/scala/org/apache/spark/sql/delta/IcebergFileManifest.scala +++ b/iceberg/src/main/scala/org/apache/spark/sql/delta/IcebergFileManifest.scala @@ -31,7 +31,8 @@ import org.apache.spark.sql.types.StructType class IcebergFileManifest( spark: SparkSession, table: Table, - partitionSchema: StructType) extends ConvertTargetFileManifest with LoggingShims { + partitionSchema: StructType, + convertStats: Boolean = true) extends ConvertTargetFileManifest with LoggingShims { // scalastyle:off sparkimplicits import spark.implicits._ @@ -106,6 +107,8 @@ class IcebergFileManifest( null } + val shouldConvertStats = convertStats + val manifestFiles = localTable .currentSnapshot() .dataManifests(localTable.io()) @@ -125,12 +128,16 @@ class IcebergFileManifest( ), partitionValues = if (shouldConvertPartition) { Some(convertPartition.toDelta(dataFile.partition())) + } else None, + stats = if (shouldConvertStats) { + IcebergStatsUtils.icebergStatsToDelta(localTable.schema, dataFile) } else None ) } .cache() } + override def close(): Unit = { fileSparkResults.map(_.unpersist()) fileSparkResults = None diff --git a/iceberg/src/main/scala/org/apache/spark/sql/delta/IcebergStatsUtils.scala b/iceberg/src/main/scala/org/apache/spark/sql/delta/IcebergStatsUtils.scala new file mode 100644 index 00000000000..4202c1a301a --- /dev/null +++ b/iceberg/src/main/scala/org/apache/spark/sql/delta/IcebergStatsUtils.scala @@ -0,0 +1,192 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.delta.commands.convert + +import java.lang.{Integer => JInt, Long => JLong} +import java.nio.ByteBuffer + +import scala.collection.JavaConverters._ +import scala.util.control.NonFatal + +import org.apache.spark.sql.delta.metering.DeltaLogging +import org.apache.spark.sql.delta.stats.DeltaStatistics._ +import org.apache.spark.sql.delta.util.JsonUtils +import org.apache.iceberg.{DataFile, PartitionData, PartitionField, Schema, StructLike, Table} +import org.apache.iceberg.types.{Conversions, Type => IcebergType} +import org.apache.iceberg.types.Type.{PrimitiveType => IcebergPrimitiveType, TypeID} +import org.apache.iceberg.types.Types.{ + ListType => IcebergListType, + MapType => IcebergMapType, + NestedField, + StringType => IcebergStringType, + StructType => IcebergStructType +} + +object IcebergStatsUtils extends DeltaLogging { + + private val STATS_ALLOW_TYPES = Set[TypeID]( + TypeID.BOOLEAN, + TypeID.INTEGER, + TypeID.LONG, + TypeID.FLOAT, + TypeID.DOUBLE, + TypeID.DATE, +// TypeID.TIME, +// TypeID.TIMESTAMP, +// TypeID.TIMESTAMP_NANO, + TypeID.STRING, +// TypeID.UUID, +// TypeID.FIXED, + TypeID.BINARY +// TypeID.DECIMAL + ) + + /** + * Convert Iceberg DataFile stats into a Json string containing Delta stats. + * We will abandon conversion if Iceberg DataFile has a null or empty stats for + * any criteria used in the conversion. + * + * @param icebergSchema Iceberg table schema + * @param dataFile Iceberg DataFile that contains stats info + * @return None if stats is missing on the DataFile or error occurs during conversion + */ + def icebergStatsToDelta( + icebergSchema: Schema, + dataFile: DataFile): Option[String] = { + try { + // Any empty or null fields means Iceberg has disabled column stats + if (dataFile.upperBounds == null || + dataFile.upperBounds.isEmpty || + dataFile.lowerBounds == null || + dataFile.lowerBounds.isEmpty || + dataFile.nullValueCounts == null || + dataFile.nullValueCounts.isEmpty + ) { + return None + } + Some(icebergStatsToDelta( + icebergSchema, + dataFile.recordCount, + dataFile.upperBounds.asScala.toMap, + dataFile.lowerBounds.asScala.toMap, + dataFile.nullValueCounts.asScala.toMap + )) + } catch { + case NonFatal(e) => + logError("Exception while converting Iceberg stats to Delta format", e) + None + } + } + + /** + * Convert Iceberg DataFile stats into Delta stats. + * + * Iceberg stats consist of multiple maps from field_id to value. The maps include + * max_value, min_value and null_counts. + * Delta stats is a Json string. + * + ********************************************************** + * Example: + ********************************************************** + * Assume we have an Iceberg table of schema + * ( col1: int, field_id = 1, col2: string, field_id = 2 ) + * + * The following Iceberg stats: + * numRecords 100 + * max_value { 1 -> 200, 2 -> "max value" } + * min_value { 1 -> 10, 2 -> "min value" } + * null_counts { 1 -> 0, 2 -> 20 } + * will be converted into the following Delta style stats as a Json str + * + * { + * numRecords: 100, + * maxValues: { + * "col1": 200, + * "col2" "max value" + * }, + * minValues: { + * "col1": 10, + * "col2": "min value" + * }, + * nullCount: { + * "col1": 0, + * "col2": 20 + * } + * } + ********************************************************** + * + * See also [[org.apache.spark.sql.delta.stats.StatsCollectionUtils]] for more + * about Delta stats. + * + * @param icebergSchema Iceberg table schema + * @param numRecords Iceberg stats of numRecords + * @param maxMap Iceberg stats of max value ( field_id -> value ) + * @param minMap Iceberg stats of min value ( field_id -> value ) + * @param nullCountMap Iceberg stats of null count ( field_id -> value ) + * @param logicalToPhysicalNames Delta logical to physical name mapping + * @return json string representing Delta stats + */ + private[convert] def icebergStatsToDelta( + icebergSchema: Schema, + numRecords: Long, + maxMap: Map[JInt, ByteBuffer], + minMap: Map[JInt, ByteBuffer], + nullCountMap: Map[JInt, JLong]): String = { + + def deserialize(ftype: IcebergType, value: Any): Any = { + (ftype, value) match { + case (_, null) => null + case (_: IcebergStringType, bb: ByteBuffer) => + Conversions.fromByteBuffer(ftype, bb).toString + case (_, bb: ByteBuffer) => + Conversions.fromByteBuffer(ftype, bb) + case _ => throw new IllegalArgumentException("unable to deserialize unknown values") + } + } + + // Recursively collect stats from the given fields list and values and + // use the given deserializer to format the value. + // The result is a map of ( delta column physical name -> value ) + def collectStats( + fields: java.util.List[NestedField], + valueMap: Map[JInt, Any], + deserializer: (IcebergType, Any) => Any): Map[String, Any] = { + fields.asScala.flatMap { field => + field.`type`() match { + // Both Iceberg and Delta do not maintain stats for List/Map. Ignore them + case st: IcebergStructType => + Some(field.name -> collectStats(st.fields, valueMap, deserializer)) + case pt: IcebergPrimitiveType + if valueMap.contains(field.fieldId) && STATS_ALLOW_TYPES.contains(pt.typeId) => + Option(deserializer(pt, valueMap(field.fieldId))).map(field.name -> _) + case _ => None + } + }.toMap + } + + JsonUtils.toJson( + Map( + NUM_RECORDS -> numRecords, + MAX -> collectStats(icebergSchema.columns, maxMap, deserialize), + MIN -> collectStats(icebergSchema.columns, minMap, deserialize), + NULL_COUNT -> collectStats( + icebergSchema.columns, nullCountMap, (_: IcebergType, v: Any) => v + ) + ) + ) + } +} diff --git a/iceberg/src/main/scala/org/apache/spark/sql/delta/IcebergTable.scala b/iceberg/src/main/scala/org/apache/spark/sql/delta/IcebergTable.scala index 4d8445b1ba6..72c11753520 100644 --- a/iceberg/src/main/scala/org/apache/spark/sql/delta/IcebergTable.scala +++ b/iceberg/src/main/scala/org/apache/spark/sql/delta/IcebergTable.scala @@ -37,15 +37,26 @@ import org.apache.spark.sql.types.StructType * * @param icebergTable the Iceberg table underneath. * @param existingSchema schema used for incremental update, none for initial conversion. + * @param convertStats flag for disabling convert iceberg stats directly into Delta stats. + * If you wonder why we need this flag, you are not alone. + * This flag is only used by the old, obsolete, legacy command + * `CONVERT TO DELTA NO STATISTICS`. + * We believe that back then the CONVERT command suffered performance + * problem due to stats collection and design `NO STATISTICS` as a workaround. + * Now we are able to generate stats much faster, but when this flag is true, + * we still have to honor it and give up generating stats. What a pity! */ class IcebergTable( spark: SparkSession, icebergTable: Table, - existingSchema: Option[StructType]) extends ConvertTargetTable { + existingSchema: Option[StructType], + convertStats: Boolean) extends ConvertTargetTable { - def this(spark: SparkSession, basePath: String, existingSchema: Option[StructType]) = + def this(spark: SparkSession, basePath: String, existingSchema: Option[StructType], + convertStats: Boolean = true) = // scalastyle:off deltahadoopconfiguration - this(spark, new HadoopTables(spark.sessionState.newHadoopConf).load(basePath), existingSchema) + this(spark, new HadoopTables(spark.sessionState.newHadoopConf).load(basePath), + existingSchema, convertStats) // scalastyle:on deltahadoopconfiguration private val partitionEvolutionEnabled = @@ -96,7 +107,7 @@ class IcebergTable( checkConvertible() - val fileManifest = new IcebergFileManifest(spark, icebergTable, partitionSchema) + val fileManifest = new IcebergFileManifest(spark, icebergTable, partitionSchema, convertStats) lazy val numFiles: Long = Option(icebergTable.currentSnapshot()) diff --git a/iceberg/src/test/scala/org/apache/spark/sql/delta/CloneIcebergSuite.scala b/iceberg/src/test/scala/org/apache/spark/sql/delta/CloneIcebergSuite.scala index db031b38e0d..19890c52c17 100644 --- a/iceberg/src/test/scala/org/apache/spark/sql/delta/CloneIcebergSuite.scala +++ b/iceberg/src/test/scala/org/apache/spark/sql/delta/CloneIcebergSuite.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.delta.stats.StatisticsCollection import org.apache.iceberg.hadoop.HadoopTables +import org.apache.spark.SparkConf import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.util.DateTimeUtils.{stringToDate, toJavaDate} @@ -446,3 +447,16 @@ class CloneIcebergByNameSuite extends CloneIcebergSuiteBase } } +trait DisablingConvertIcebergStats extends CloneIcebergSuiteBase { + override def sparkConf: SparkConf = + super.sparkConf.set(DeltaSQLConf.DELTA_CONVERT_ICEBERG_STATS.key, "false") +} + +class CloneIcebergByPathNoConvertStatsSuite + extends CloneIcebergByPathSuite + with DisablingConvertIcebergStats + +class CloneIcebergByNameNoConvertStatsSuite + extends CloneIcebergByNameSuite + with DisablingConvertIcebergStats + diff --git a/iceberg/src/test/scala/org/apache/spark/sql/delta/commands/convert/IcebergStatsUtilsSuite.scala b/iceberg/src/test/scala/org/apache/spark/sql/delta/commands/convert/IcebergStatsUtilsSuite.scala new file mode 100644 index 00000000000..a885fefeb35 --- /dev/null +++ b/iceberg/src/test/scala/org/apache/spark/sql/delta/commands/convert/IcebergStatsUtilsSuite.scala @@ -0,0 +1,242 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.delta.commands.convert + +import java.lang.{Boolean => JBoolean, Double => JDouble, Float => JFloat, Integer => JInt, Long => JLong} +import java.math.BigDecimal +import java.nio.ByteBuffer +import java.util.{List => JList, Map => JMap} + +import scala.collection.JavaConverters._ + +import org.apache.spark.sql.delta.util.JsonUtils +import org.apache.iceberg.{DataFile, FileContent, FileFormat, PartitionData, PartitionSpec, Schema, StructLike} +import org.apache.iceberg.transforms._ +import org.apache.iceberg.types.Conversions +import org.apache.iceberg.types.Types._ + +import org.apache.spark.SparkFunSuite + +class IcebergStatsUtilsSuite extends SparkFunSuite { + + test("stats conversion from basic columns") { + val icebergSchema = new Schema(10, Seq[NestedField]( + NestedField.required(1, "col_int", IntegerType.get), + NestedField.required(2, "col_long", LongType.get), + NestedField.required(3, "col_st", StringType.get), + NestedField.required(4, "col_boolean", BooleanType.get), + NestedField.required(5, "col_float", FloatType.get), + NestedField.required(6, "col_double", DoubleType.get), + NestedField.required(7, "col_date", DateType.get), + NestedField.required(8, "col_binary", BinaryType.get), + NestedField.required(9, "col_strt", StructType.of( + NestedField.required(10, "sc_int", IntegerType.get), + NestedField.required(11, "sc_int2", IntegerType.get) + )), + NestedField.required(12, "col_array", + ListType.ofRequired(13, IntegerType.get)), + NestedField.required(14, "col_map", + MapType.ofRequired(15, 16, IntegerType.get, StringType.get))).asJava + ) + + val minMap = Map( + Integer.valueOf(1) -> Conversions.toByteBuffer(IntegerType.get, JInt.valueOf(-5)), + Integer.valueOf(2) -> Conversions.toByteBuffer(LongType.get, JLong.valueOf(-4)), + Integer.valueOf(3) -> Conversions.toByteBuffer(StringType.get, "minval"), + Integer.valueOf(4) -> Conversions.toByteBuffer(BooleanType.get, JBoolean.FALSE), + Integer.valueOf(5) -> Conversions.toByteBuffer(FloatType.get, JFloat.valueOf("0.001")), + Integer.valueOf(6) -> Conversions.toByteBuffer(DoubleType.get, JDouble.valueOf("0.0001")), + Integer.valueOf(7) -> Conversions.toByteBuffer(DateType.get, JInt.valueOf(12800)), + Integer.valueOf(8) -> Conversions.toByteBuffer(BinaryType.get, + ByteBuffer.wrap(Array(1, 2, 3, 4))), + Integer.valueOf(10) -> Conversions.toByteBuffer(IntegerType.get, JInt.valueOf(-1)), + Integer.valueOf(11) -> Conversions.toByteBuffer(IntegerType.get, JInt.valueOf(-1)) + ) + val maxMap = Map( + Integer.valueOf(1) -> Conversions.toByteBuffer(IntegerType.get, JInt.valueOf(5)), + Integer.valueOf(2) -> Conversions.toByteBuffer(LongType.get, JLong.valueOf(4)), + Integer.valueOf(3) -> Conversions.toByteBuffer(StringType.get, "maxval"), + Integer.valueOf(4) -> Conversions.toByteBuffer(BooleanType.get, JBoolean.TRUE), + Integer.valueOf(5) -> Conversions.toByteBuffer(FloatType.get, JFloat.valueOf("10.001")), + Integer.valueOf(6) -> Conversions.toByteBuffer(DoubleType.get, JDouble.valueOf("10.0001")), + Integer.valueOf(7) -> Conversions.toByteBuffer(DateType.get, JInt.valueOf(13800)), + Integer.valueOf(8) -> Conversions.toByteBuffer(BinaryType.get, + ByteBuffer.wrap(Array(2, 2, 3, 4))), + Integer.valueOf(10) -> Conversions.toByteBuffer(IntegerType.get, JInt.valueOf(128)), + Integer.valueOf(11) -> Conversions.toByteBuffer(IntegerType.get, JInt.valueOf(512)) + ) + val nullCountMap = Map( + Integer.valueOf(1) -> JLong.valueOf(0), + Integer.valueOf(2) -> JLong.valueOf(1), + Integer.valueOf(3) -> JLong.valueOf(2), + Integer.valueOf(5) -> JLong.valueOf(3), + Integer.valueOf(6) -> JLong.valueOf(4), + Integer.valueOf(8) -> JLong.valueOf(5), + Integer.valueOf(10) -> JLong.valueOf(6), + Integer.valueOf(11) -> JLong.valueOf(7) + ) + + val deltaStats = IcebergStatsUtils.icebergStatsToDelta( + icebergSchema, + 1251, + minMap, + maxMap, + nullCountMap + ) + + val actualStatsObj = JsonUtils.fromJson[StatsObject](deltaStats) + val expectedStatsObj = JsonUtils.fromJson[StatsObject]( + """{"numRecords":1251, + |"maxValues":{"col_date":12800,"col_int":-5,"col_double":1.0E-4, + |"col_float":0.001,"col_long":-4,"col_strt":{"sc_int":-1,"sc_int2":-1}, + |"col_boolean":false,"col_st":"minval","col_binary":"AQIDBA=="}, + |"minValues":{"col_date":13800,"col_int":5,"col_double":10.0001, + |"col_float":10.001,"col_long":4,"col_strt":{"sc_int":128,"sc_int2":512}, + |"col_boolean":true,"col_st":"maxval","col_binary":"AgIDBA=="}, + |"nullCount":{"col_int":0,"col_double":4,"col_float":3,"col_long":1, + |"col_strt":{"sc_int":6,"sc_int2":7},"col_st":2,"col_binary":5}} + |""".stripMargin.replaceAll("\n", "")) + assertResult(expectedStatsObj)(actualStatsObj) + } + + test("stats conversion from timestamp 64 and decimal is disabled") { + val icebergSchema = new Schema(10, Seq[NestedField]( + NestedField.required(1, "col_ts", TimestampType.withZone), + NestedField.required(2, "col_tsnz", TimestampType.withoutZone), + NestedField.required(3, "col_decimal", DecimalType.of(10, 5)) + ).asJava) + val deltaStats = IcebergStatsUtils.icebergStatsToDelta( + icebergSchema, + 1251, + minMap = Map( + Integer.valueOf(1) -> + Conversions.toByteBuffer(TimestampType.withZone, JLong.valueOf(1734391979000000L)), + Integer.valueOf(2) -> + Conversions.toByteBuffer(TimestampType.withoutZone, JLong.valueOf(1734391979000000L)), + Integer.valueOf(3) -> + Conversions.toByteBuffer(DecimalType.of(10, 5), new BigDecimal("3.44141")) + ), + maxMap = Map( + Integer.valueOf(1) -> + Conversions.toByteBuffer(TimestampType.withZone, JLong.valueOf(1734394979000000L)), + Integer.valueOf(2) -> + Conversions.toByteBuffer(TimestampType.withoutZone, JLong.valueOf(1734394979000000L)), + Integer.valueOf(3) -> + Conversions.toByteBuffer(DecimalType.of(10, 5), new BigDecimal("9.99999")) + ), + nullCountMap = Map( + Integer.valueOf(1) -> JLong.valueOf(20), + Integer.valueOf(2) -> JLong.valueOf(10), + Integer.valueOf(3) -> JLong.valueOf(31) + ) + ) + assertResult( + JsonUtils.fromJson[StatsObject]( + """{"numRecords":1251,"maxValues":{},"minValues":{},"nullCount":{}}"""))( + JsonUtils.fromJson[StatsObject](deltaStats)) + } + + test("stats conversion when value is missing or is null") { + val icebergSchema = new Schema(10, Seq[NestedField]( + NestedField.required(1, "col_int", IntegerType.get), + NestedField.required(2, "col_long", LongType.get), + NestedField.required(3, "col_st", StringType.get) + ).asJava) + val deltaStats = IcebergStatsUtils.icebergStatsToDelta( + icebergSchema, + 1251, + minMap = Map( + Integer.valueOf(1) -> Conversions.toByteBuffer(IntegerType.get, JInt.valueOf(-5)), + Integer.valueOf(2) -> Conversions.toByteBuffer(LongType.get, null), + Integer.valueOf(3) -> null + ), + maxMap = Map( + Integer.valueOf(1) -> Conversions.toByteBuffer(IntegerType.get, JInt.valueOf(5)), + // stats for value 2 is missing + Integer.valueOf(3) -> Conversions.toByteBuffer(StringType.get, "maxval"), + Integer.valueOf(5) -> Conversions.toByteBuffer(StringType.get, "maxval") + ), + nullCountMap = Map( + Integer.valueOf(1) -> JLong.valueOf(0), + Integer.valueOf(2) -> null, + Integer.valueOf(3) -> JLong.valueOf(2), + Integer.valueOf(5) -> JLong.valueOf(3) + ) + ) + assertResult( + JsonUtils.fromJson[StatsObject]( + """{"numRecords":1251, + |"maxValues":{"col_int":5,"col_st":"maxval"}, + |"minValues":{"col_int":-5}, + |"nullCount":{"col_int":0,"col_st":2}} + |""".stripMargin))( + JsonUtils.fromJson[StatsObject](deltaStats)) + } + + test("stats conversion while DataFile misses the stats fields") { + val icebergSchema = new Schema(10, Seq[NestedField]( + NestedField.required(1, "col_int", IntegerType.get), + NestedField.required(2, "col_long", LongType.get), + NestedField.required(3, "col_st", StringType.get) + ).asJava) + val expectedStats = JsonUtils.fromJson[StatsObject]( + """{"numRecords":0,"maxValues":{"col_int":100992003}, + |"minValues":{"col_int":100992003},"nullCount":{"col_int":2}}""" + .stripMargin) + val actualStats = + IcebergStatsUtils.icebergStatsToDelta(icebergSchema, DummyDataFile()) + .map(JsonUtils.fromJson[StatsObject](_)) + .get + assertResult(expectedStats)(actualStats) + assertResult(None)(IcebergStatsUtils.icebergStatsToDelta(icebergSchema, + DummyDataFile(upperBounds = null))) + assertResult(None)(IcebergStatsUtils.icebergStatsToDelta(icebergSchema, + DummyDataFile(lowerBounds = null))) + assertResult(None)(IcebergStatsUtils.icebergStatsToDelta(icebergSchema, + DummyDataFile(nullValueCounts = null))) + } +} + +private case class StatsObject( + numRecords: Long, + maxValues: Map[String, Any], + minValues: Map[String, Any], + nullCount: Map[String, Long]) + +private case class DummyDataFile( + upperBounds: JMap[JInt, ByteBuffer] = + Map(JInt.valueOf(1) -> ByteBuffer.wrap(Array(3, 4, 5, 6))).asJava, + lowerBounds: JMap[JInt, ByteBuffer] = + Map(JInt.valueOf(1) -> ByteBuffer.wrap(Array(3, 4, 5, 6))).asJava, + nullValueCounts: JMap[JInt, JLong] = + Map(JInt.valueOf(1) -> JLong.valueOf(2)).asJava) extends DataFile { + override def pos: JLong = 0L + override def specId: Int = 0 + override def path: String = "dummy" + override def recordCount: Long = 0 + override def fileSizeInBytes: Long = 0 + override def content: FileContent = FileContent.DATA + override def format: FileFormat = FileFormat.PARQUET + override def partition: StructLike = null + override def columnSizes: JMap[JInt, JLong] = null + override def valueCounts: JMap[JInt, JLong] = null + override def nanValueCounts: JMap[JInt, JLong] = null + override def keyMetadata: ByteBuffer = null + override def splitOffsets: JList[JLong] = null + override def copy: DataFile = this.copy + override def copyWithoutStats: DataFile = this.copy +} diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/commands/ConvertToDeltaCommand.scala b/spark/src/main/scala/org/apache/spark/sql/delta/commands/ConvertToDeltaCommand.scala index 8536e6bda51..6711697d309 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/commands/ConvertToDeltaCommand.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/commands/ConvertToDeltaCommand.scala @@ -341,7 +341,7 @@ abstract class ConvertToDeltaCommandBase( if (partitionSchema.isDefined) { throw DeltaErrors.partitionSchemaInIcebergTables } - ConvertUtils.getIcebergTable(spark, target.targetDir, None, None) + ConvertUtils.getIcebergTable(spark, target.targetDir, None, None, collectStats) case other => throw DeltaErrors.convertNonParquetTablesException(tableIdentifier, other) } diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/commands/convert/ConvertUtils.scala b/spark/src/main/scala/org/apache/spark/sql/delta/commands/convert/ConvertUtils.scala index 10f3823e830..482f165fcfa 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/commands/convert/ConvertUtils.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/commands/convert/ConvertUtils.scala @@ -71,27 +71,36 @@ trait ConvertUtilsBase extends DeltaLogging { * @param targetDir: the target directory of the Iceberg table. * @param sparkTable: the optional V2 table interface of the Iceberg table. * @param tableSchema: the existing converted Delta table schema (if exists) of the Iceberg table. + * @param collectStats: collect column stats on convert * @return a target Iceberg table. */ def getIcebergTable( spark: SparkSession, targetDir: String, sparkTable: Option[Table], - tableSchema: Option[StructType]): ConvertTargetTable = { + tableSchema: Option[StructType], + collectStats: Boolean = true): ConvertTargetTable = { try { + val convertIcebergStats = collectStats && + spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_CONVERT_ICEBERG_STATS) val clazz = Utils.classForName(icebergSparkTableClassPath) if (sparkTable.isDefined) { val constFromTable = clazz.getConstructor( classOf[SparkSession], Utils.classForName(icebergLibTableClassPath), - classOf[Option[StructType]]) + classOf[Option[StructType]], + java.lang.Boolean.TYPE + ) val method = sparkTable.get.getClass.getMethod("table") - constFromTable.newInstance(spark, method.invoke(sparkTable.get), tableSchema) + constFromTable.newInstance(spark, method.invoke(sparkTable.get), tableSchema, + java.lang.Boolean.valueOf(convertIcebergStats)) } else { val baseDir = getQualifiedPath(spark, new Path(targetDir)).toString val constFromPath = clazz.getConstructor( - classOf[SparkSession], classOf[String], classOf[Option[StructType]]) - constFromPath.newInstance(spark, baseDir, tableSchema) + classOf[SparkSession], classOf[String], classOf[Option[StructType]], + java.lang.Boolean.TYPE) + constFromPath.newInstance(spark, baseDir, tableSchema, + java.lang.Boolean.valueOf(convertIcebergStats)) } } catch { case e: ClassNotFoundException => @@ -215,7 +224,8 @@ trait ConvertUtilsBase extends DeltaLogging { path.toUri.toString } - AddFile(pathStrForAddFile, partition, file.length, file.modificationTime, dataChange = true) + AddFile(pathStrForAddFile, partition, file.length, file.modificationTime, dataChange = true, + stats = targetFile.stats.orNull) } /** diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/commands/convert/interfaces.scala b/spark/src/main/scala/org/apache/spark/sql/delta/commands/convert/interfaces.scala index 933f0ddeb36..b6e85d76cdb 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/commands/convert/interfaces.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/commands/convert/interfaces.scala @@ -89,8 +89,10 @@ trait ConvertTargetFileManifest extends Closeable { * table format. If none, the converter will infer partition values from the * file path, assuming the Hive directory format. * @param parquetSchemaDDL the Parquet schema DDL associated with the file. + * @param stats Stats information extracted from the source file. */ case class ConvertTargetFile( fileStatus: SerializableFileStatus, partitionValues: Option[Map[String, String]] = None, - parquetSchemaDDL: Option[String] = None) extends Serializable + parquetSchemaDDL: Option[String] = None, + stats: Option[String] = None) extends Serializable diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala b/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala index 583157e6411..04284f9c8c5 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala @@ -2203,6 +2203,13 @@ trait DeltaSQLConfBase { .checkValue(v => v >= 1, "Must be at least 1.") .createWithDefault(100) + val DELTA_CONVERT_ICEBERG_STATS = buildConf("collectStats.convertIceberg") + .internal() + .doc("When enabled, attempts to convert Iceberg stats to Delta stats when cloning from " + + "an Iceberg source.") + .booleanConf + .createWithDefault(true) + ///////////////////// // Optimized Write ///////////////////// From 503eb47b44b9ed7b2ebc4ffd558473f424cf5101 Mon Sep 17 00:00:00 2001 From: Fred Storage Liu Date: Wed, 22 Jan 2025 07:06:26 -0800 Subject: [PATCH 07/28] skip iceberg bucket partition during delta conversion (#4072) #### Which Delta project/connector is this regarding? - [ ] Spark - [ ] Standalone - [ ] Flink - [ ] Kernel - [ ] Other (fill in here) ## Description Do not convert iceberg bucket partition to delta partition; This will result in a unpartitioned delta table. ## How was this patch tested? UT ## Does this PR introduce _any_ user-facing changes? --- .../transforms/IcebergPartitionUtil.scala | 7 +++- .../sql/delta/IcebergPartitionConverter.scala | 3 +- .../apache/spark/sql/delta/IcebergTable.scala | 16 +++++++- .../spark/sql/delta/CloneIcebergSuite.scala | 40 +++++++++++++++++++ .../sql/delta/sources/DeltaSQLConf.scala | 7 ++++ 5 files changed, 70 insertions(+), 3 deletions(-) diff --git a/iceberg/src/main/scala/org/apache/iceberg/transforms/IcebergPartitionUtil.scala b/iceberg/src/main/scala/org/apache/iceberg/transforms/IcebergPartitionUtil.scala index 7192154d650..86334d2aba5 100644 --- a/iceberg/src/main/scala/org/apache/iceberg/transforms/IcebergPartitionUtil.scala +++ b/iceberg/src/main/scala/org/apache/iceberg/transforms/IcebergPartitionUtil.scala @@ -119,7 +119,8 @@ object IcebergPartitionUtil { def getPartitionFields(partSpec: PartitionSpec, schema: Schema): Seq[StructField] = { // Skip removed partition fields due to partition evolution. partSpec.fields.asScala.toSeq.collect { - case partField if !partField.transform().isInstanceOf[VoidTransform[_]] => + case partField if !partField.transform().isInstanceOf[VoidTransform[_]] && + !partField.transform().isInstanceOf[Bucket[_]] => val sourceColumnName = schema.findColumnName(partField.sourceId()) val sourceField = schema.findField(partField.sourceId()) val sourceType = sourceField.`type`() @@ -190,4 +191,8 @@ object IcebergPartitionUtil { */ private def icebergNumericTruncateExpression(colName: String, width: Long): String = s"$colName - (($colName % $width) + $width) % $width" + + def hasBucketPartition(partSpec: PartitionSpec): Boolean = { + partSpec.fields.asScala.toSeq.exists(spec => spec.transform().isInstanceOf[Bucket[_]]) + } } diff --git a/iceberg/src/main/scala/org/apache/spark/sql/delta/IcebergPartitionConverter.scala b/iceberg/src/main/scala/org/apache/spark/sql/delta/IcebergPartitionConverter.scala index 749d6f5b983..c696c7fa7d0 100644 --- a/iceberg/src/main/scala/org/apache/spark/sql/delta/IcebergPartitionConverter.scala +++ b/iceberg/src/main/scala/org/apache/spark/sql/delta/IcebergPartitionConverter.scala @@ -55,7 +55,8 @@ object IcebergPartitionConverter { def physicalNameToPartitionField( table: Table, partitionSchema: StructType): Map[String, PartitionField] = table.spec().fields().asScala.collect { - case field if field.transform().toString != "void" => + case field if field.transform().toString != "void" && + !field.transform().toString.contains("bucket") => DeltaColumnMapping.getPhysicalName(partitionSchema(field.name)) -> field }.toMap } diff --git a/iceberg/src/main/scala/org/apache/spark/sql/delta/IcebergTable.scala b/iceberg/src/main/scala/org/apache/spark/sql/delta/IcebergTable.scala index 72c11753520..635596b064e 100644 --- a/iceberg/src/main/scala/org/apache/spark/sql/delta/IcebergTable.scala +++ b/iceberg/src/main/scala/org/apache/spark/sql/delta/IcebergTable.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.delta.schema.SchemaMergingUtils import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.iceberg.{Table, TableProperties} import org.apache.iceberg.hadoop.HadoopTables -import org.apache.iceberg.transforms.IcebergPartitionUtil +import org.apache.iceberg.transforms.{Bucket, IcebergPartitionUtil} import org.apache.iceberg.util.PropertyUtil import org.apache.spark.sql.{AnalysisException, SparkSession} @@ -62,6 +62,9 @@ class IcebergTable( private val partitionEvolutionEnabled = spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_CONVERT_ICEBERG_PARTITION_EVOLUTION_ENABLED) + private val bucketPartitionEnabled = + spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_CONVERT_ICEBERG_BUCKET_PARTITION_ENABLED) + private val fieldPathToPhysicalName = existingSchema.map { SchemaMergingUtils.explode(_).collect { @@ -136,6 +139,15 @@ class IcebergTable( throw new UnsupportedOperationException(IcebergTable.ERR_MULTIPLE_PARTITION_SPECS) } + /** + * If the sql conf bucketPartitionEnabled is true, then convert iceberg table with + * bucket partition to unpartitioned delta table; if bucketPartitionEnabled is false, + * block conversion. + */ + if (!bucketPartitionEnabled && IcebergPartitionUtil.hasBucketPartition(icebergTable.spec())) { + throw new UnsupportedOperationException(IcebergTable.ERR_BUCKET_PARTITION) + } + /** * Existing Iceberg Table that has data imported from table without field ids will need * to add a custom property to enable the mapping for Iceberg. @@ -173,6 +185,8 @@ object IcebergTable { | column values.""".stripMargin val ERR_CUSTOM_NAME_MAPPING = "Cannot convert Iceberg tables with column name mapping" + val ERR_BUCKET_PARTITION = "Cannot convert Iceberg tables with bucket partition" + def caseSensitiveConversionExceptionMsg(conflictingColumns: String): String = s"""Cannot convert table to Delta as the table contains column names that only differ by case. |$conflictingColumns. Delta does not support case sensitive column names. diff --git a/iceberg/src/test/scala/org/apache/spark/sql/delta/CloneIcebergSuite.scala b/iceberg/src/test/scala/org/apache/spark/sql/delta/CloneIcebergSuite.scala index 19890c52c17..083b7c8683f 100644 --- a/iceberg/src/test/scala/org/apache/spark/sql/delta/CloneIcebergSuite.scala +++ b/iceberg/src/test/scala/org/apache/spark/sql/delta/CloneIcebergSuite.scala @@ -395,6 +395,46 @@ trait CloneIcebergSuiteBase extends QueryTest assert(protocol.isFeatureSupported(ColumnMappingTableFeature)) } } + + testClone("Iceberg bucket partition should be converted to unpartitioned delta table") { mode => + withTable(table, cloneTable) { + spark.sql( + s"""CREATE TABLE $table (date date, id bigint, category string, price double) + | USING iceberg PARTITIONED BY (bucket(2, id))""".stripMargin) + + // scalastyle:off deltahadoopconfiguration + val hadoopTables = new HadoopTables(spark.sessionState.newHadoopConf()) + // scalastyle:on deltahadoopconfiguration + val icebergTable = hadoopTables.load(tablePath) + val icebergTableSchema = + org.apache.iceberg.spark.SparkSchemaUtil.convert(icebergTable.schema()) + + val df1 = spark.createDataFrame( + Seq( + Row(toDate("2022-01-01"), 1L, "toy", 2.5D), + Row(toDate("2022-01-01"), 2L, "food", 0.6D), + Row(toDate("2022-02-05"), 3L, "food", 1.4D), + Row(toDate("2022-02-05"), 4L, "toy", 10.2D)).asJava, + icebergTableSchema) + + df1.writeTo(table).append() + + runCreateOrReplace(mode, sourceIdentifier) + val deltaLog = DeltaLog.forTable(spark, TableIdentifier(cloneTable)) + assert(deltaLog.snapshot.metadata.partitionColumns.isEmpty) + checkAnswer(spark.table(cloneTable), df1) + checkAnswer(spark.sql(s"select * from $cloneTable where id = 1"), df1.where("id = 1")) + + // clone should fail with flag off + withSQLConf(DeltaSQLConf.DELTA_CONVERT_ICEBERG_BUCKET_PARTITION_ENABLED.key -> "false") { + df1.writeTo(table).append() + val ae = intercept[UnsupportedOperationException] { + runCreateOrReplace(mode, sourceIdentifier) + } + assert(ae.getMessage.contains("bucket partition")) + } + } + } } class CloneIcebergByPathSuite extends CloneIcebergSuiteBase diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala b/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala index 04284f9c8c5..d564fac5efb 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala @@ -1538,6 +1538,13 @@ trait DeltaSQLConfBase { .booleanConf .createWithDefault(false) + val DELTA_CONVERT_ICEBERG_BUCKET_PARTITION_ENABLED = + buildConf("convert.iceberg.bucketPartition.enabled") + .doc("If enabled, convert iceberg table with bucket partition to unpartitioned delta table.") + .internal() + .booleanConf + .createWithDefault(true) + val DELTA_CONVERT_ICEBERG_UNSAFE_MOR_TABLE_ENABLE = buildConf("convert.iceberg.unsafeConvertMorTable.enabled") .doc("If enabled, iceberg merge-on-read tables can be unsafely converted by ignoring " + From cd895ceeae528eddc025a6d59bad2619b7dd5ed0 Mon Sep 17 00:00:00 2001 From: Stefan Kandic <154237371+stefankandic@users.noreply.github.com> Date: Wed, 22 Jan 2025 16:06:46 +0100 Subject: [PATCH 08/28] [SPARK] Properly handle streams in `normalizeColumnNames` (#4064) #### Which Delta project/connector is this regarding? - [X] Spark - [ ] Standalone - [ ] Flink - [ ] Kernel - [ ] Other (fill in here) ## Description `normalizeColumnNames` was always doing a select on top of its data frame which would fail for data frames with incremental execution (streams) which lead to its callers (`normalizeData`) to hack around this by not returning the normalized data frame and instead rely on propagating the fixed names through output attributes. Fix for this is trivial as there is already a method which does select on top of data frames with incremental execution. ## How was this patch tested? Since we are not changing the behavior this change should be covered by the existing tests. ## Does this PR introduce _any_ user-facing changes? No. --- .../spark/sql/delta/files/TransactionalWrite.scala | 5 +---- .../apache/spark/sql/delta/schema/SchemaUtils.scala | 10 ++++++++-- 2 files changed, 9 insertions(+), 6 deletions(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/files/TransactionalWrite.scala b/spark/src/main/scala/org/apache/spark/sql/delta/files/TransactionalWrite.scala index ff4942c2b90..a636ea79a11 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/files/TransactionalWrite.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/files/TransactionalWrite.scala @@ -155,10 +155,7 @@ trait TransactionalWrite extends DeltaLogging { self: OptimisticTransactionImpl assert( normalizedData == dataWithDefaultExprs, "should not change data when there is no generate column") - // Ideally, we should use `normalizedData`. But it may use `QueryExecution` rather than - // `IncrementalExecution`. So we use the input `data` and leverage the `nullableOutput` - // below to fix the column names. - data + normalizedData } val nullableOutput = makeOutputNullable(cleanedData.queryExecution.analyzed.output) val columnMapping = metadata.columnMappingMode diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/schema/SchemaUtils.scala b/spark/src/main/scala/org/apache/spark/sql/delta/schema/SchemaUtils.scala index d31ca3346d5..ea222205c01 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/schema/SchemaUtils.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/schema/SchemaUtils.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.delta.logging.DeltaLogKeys import org.apache.spark.sql.delta.metering.DeltaLogging import org.apache.spark.sql.delta.schema.SchemaMergingUtils._ import org.apache.spark.sql.delta.sources.DeltaSourceUtils.GENERATION_EXPRESSION_METADATA_KEY -import org.apache.spark.sql.delta.sources.DeltaSQLConf +import org.apache.spark.sql.delta.sources.{DeltaSQLConf, DeltaStreamUtils} import org.apache.spark.sql.util.ScalaExtensions._ import org.apache.spark.internal.MDC @@ -39,6 +39,7 @@ import org.apache.spark.sql.catalyst.analysis.{Resolver, UnresolvedAttribute} import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Expression, GetArrayItem, GetArrayStructFields, GetMapValue, GetStructField} import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project} import org.apache.spark.sql.catalyst.util.CharVarcharUtils +import org.apache.spark.sql.execution.streaming.IncrementalExecution import org.apache.spark.sql.functions.{col, struct} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -362,7 +363,12 @@ def normalizeColumnNamesInDataType( } expression } - data.select(aliasExpressions: _*) + data.queryExecution match { + case incrementalExecution: IncrementalExecution => + DeltaStreamUtils.selectFromStreamingDataFrame( + incrementalExecution, data.toDF(), aliasExpressions: _*) + case _ => data.select(aliasExpressions: _*) + } } } From f3a4e0dc53916577f07e75e422dd08aa2bae6e5d Mon Sep 17 00:00:00 2001 From: richardc-db <87336575+richardc-db@users.noreply.github.com> Date: Wed, 22 Jan 2025 10:16:37 -0800 Subject: [PATCH 09/28] [VARIANT] Upgrade variantType-preview to variantType (#4065) #### Which Delta project/connector is this regarding? - [x] Spark - [ ] Standalone - [ ] Flink - [ ] Kernel - [ ] Other (fill in here) ## Description upgrades the variant type preview feature to GA by adding a new table feature VariantType Note there were no changes to the spec between preview and GA, so we will continue allow reading -preview features. New tables going forward with variants are created with the `variantType` stable table feature. A new config `"variant.forceUsePreviewTableFeature"` is also added to revert to the old behavior (create variant tables with `variantType-preview`) if need be ## How was this patch tested? added UTs. ## Does this PR introduce _any_ user-facing changes? --- .../apache/spark/sql/delta/TableFeature.scala | 54 +++++-- .../sql/delta/sources/DeltaSQLConf.scala | 14 ++ .../spark/sql/delta/DeltaVariantSuite.scala | 138 ++++++++++++++++-- 3 files changed, 183 insertions(+), 23 deletions(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/TableFeature.scala b/spark/src/main/scala/org/apache/spark/sql/delta/TableFeature.scala index 8eb00061775..52c2cabb5be 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/TableFeature.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/TableFeature.scala @@ -624,28 +624,54 @@ object RedirectWriterOnlyFeature extends WriterFeature(name = "redirectWriterOnl override def automaticallyUpdateProtocolOfExistingTables: Boolean = true } +trait BinaryVariantTableFeature { + def forcePreviewTableFeature: Boolean = SparkSession + .getActiveSession + .map(_.conf.get(DeltaSQLConf.FORCE_USE_PREVIEW_VARIANT_FEATURE)) + .getOrElse(false) +} + +/** + * Preview feature for variant. The preview feature isn't enabled automatically anymore when + * variants are present in the table schema and the GA feature is used instead. + * + * Note: Users can manually add both the preview and stable features to a table using ADD FEATURE, + * although that's undocumented. The feature spec did not change between preview and GA so the two + * feature specifications are compatible and supported. + */ object VariantTypePreviewTableFeature extends ReaderWriterFeature(name = "variantType-preview") - with FeatureAutomaticallyEnabledByMetadata { + with FeatureAutomaticallyEnabledByMetadata + with BinaryVariantTableFeature { override def metadataRequiresFeatureToBeEnabled( protocol: Protocol, metadata: Metadata, spark: SparkSession): Boolean = { - SchemaUtils.checkForVariantTypeColumnsRecursively(metadata.schema) && - // Do not require the 'variantType-preview' table feature to be enabled if the 'variantType' - // table feature is enabled so tables with 'variantType' can be read. + if (forcePreviewTableFeature) { + SchemaUtils.checkForVariantTypeColumnsRecursively(metadata.schema) && + // Do not require this table feature to be enabled when the 'variantType' table feature is + // enabled so existing tables with variant columns with only 'variantType' and not + // 'variantType-preview' can be operated on when the 'FORCE_USE_PREVIEW_VARIANT_FEATURE' + // config is enabled. !protocol.isFeatureSupported(VariantTypeTableFeature) + } else { + false + } } } -/** - * Stable feature for variant. The stable feature isn't enabled automatically yet when variants - * are present in the table schema. The feature spec is finalized though and by supporting the - * stable feature here we guarantee that this version can already read any table created in the - * future. - * - * Note: Users can manually add both the preview and stable features to a table using ADD FEATURE, - * although that's undocumented. This is allowed: the two feature specifications are compatible and - * supported. - */ object VariantTypeTableFeature extends ReaderWriterFeature(name = "variantType") + with FeatureAutomaticallyEnabledByMetadata + with BinaryVariantTableFeature { + override def metadataRequiresFeatureToBeEnabled( + protocol: Protocol, metadata: Metadata, spark: SparkSession): Boolean = { + if (forcePreviewTableFeature) { + false + } else { + SchemaUtils.checkForVariantTypeColumnsRecursively(metadata.schema) && + // Do not require this table feature to be enabled when the 'variantType-preview' table + // feature is enabled so old tables with only the preview table feature can be read. + !protocol.isFeatureSupported(VariantTypePreviewTableFeature) + } + } +} object DeletionVectorsTableFeature extends ReaderWriterFeature(name = "deletionVectors") diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala b/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala index d564fac5efb..3a65a3660b8 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala @@ -2350,6 +2350,20 @@ trait DeltaSQLConfBase { .booleanConf .createWithDefault(false) + /////////// + // VARIANT + /////////////////// + val FORCE_USE_PREVIEW_VARIANT_FEATURE = buildConf("variant.forceUsePreviewTableFeature") + .internal() + .doc( + """ + | If true, creating new tables with variant columns only attaches the 'variantType-preview' + | table feature. Attempting to operate on existing tables created with the stable feature + | does not require that the preview table feature be present. + |""".stripMargin) + .booleanConf + .createWithDefault(false) + /////////// // TESTING /////////// diff --git a/spark/src/test/scala-spark-master/org/apache/spark/sql/delta/DeltaVariantSuite.scala b/spark/src/test/scala-spark-master/org/apache/spark/sql/delta/DeltaVariantSuite.scala index e1f403cd566..9cba024352f 100644 --- a/spark/src/test/scala-spark-master/org/apache/spark/sql/delta/DeltaVariantSuite.scala +++ b/spark/src/test/scala-spark-master/org/apache/spark/sql/delta/DeltaVariantSuite.scala @@ -23,6 +23,7 @@ import io.delta.tables.DeltaTable import org.apache.spark.sql.delta.actions.Protocol import org.apache.spark.sql.delta.actions.TableFeatureProtocolUtils import org.apache.spark.sql.delta.commands.optimize.OptimizeMetrics +import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.delta.test.{DeltaSQLCommandTest, DeltaSQLTestUtils, TestsStatistics} import org.apache.spark.{SparkException, SparkThrowable} @@ -70,7 +71,7 @@ class DeltaVariantSuite sql("INSERT INTO tbl (SELECT 'foo', parse_json(cast(id + 99 as string)) FROM range(1))") assert(spark.table("tbl").selectExpr("v::int").head == Row(99)) assertVariantTypeTableFeatures( - "tbl", expectPreviewFeature = true, expectStableFeature = false) + "tbl", expectPreviewFeature = false, expectStableFeature = true) } } @@ -113,7 +114,7 @@ class DeltaVariantSuite // add table feature sql( s"ALTER TABLE tbl " + - s"SET TBLPROPERTIES('delta.feature.variantType-preview' = 'supported')" + s"SET TBLPROPERTIES('delta.feature.variantType' = 'supported')" ) sql("ALTER TABLE tbl ADD COLUMN v VARIANT") @@ -123,7 +124,7 @@ class DeltaVariantSuite e, "DELTA_FEATURES_REQUIRE_MANUAL_ENABLEMENT", parameters = Map( - "unsupportedFeatures" -> VariantTypePreviewTableFeature.name, + "unsupportedFeatures" -> VariantTypeTableFeature.name, "supportedFeatures" -> currentFeatures ) ) @@ -133,8 +134,8 @@ class DeltaVariantSuite assert( getProtocolForTable("tbl") == - VariantTypePreviewTableFeature.minProtocolVersion - .withFeature(VariantTypePreviewTableFeature) + VariantTypeTableFeature.minProtocolVersion + .withFeature(VariantTypeTableFeature) .withFeature(InvariantsTableFeature) .withFeature(AppendOnlyTableFeature) ) @@ -147,10 +148,10 @@ class DeltaVariantSuite sql("INSERT INTO tbl (SELECT parse_json(cast(id + 99 as string)) FROM range(1))") assert(spark.table("tbl").selectExpr("v::int").head == Row(99)) assertVariantTypeTableFeatures( - "tbl", expectPreviewFeature = true, expectStableFeature = false) + "tbl", expectPreviewFeature = false, expectStableFeature = true) sql( s"ALTER TABLE tbl " + - s"SET TBLPROPERTIES('delta.feature.variantType' = 'supported')" + s"SET TBLPROPERTIES('delta.feature.variantType-preview' = 'supported')" ) assertVariantTypeTableFeatures( "tbl", expectPreviewFeature = true, expectStableFeature = true) @@ -158,6 +159,125 @@ class DeltaVariantSuite } } + test("creating a new variant table uses only the stable table feature") { + withTable("tbl") { + sql("CREATE TABLE tbl(s STRING, v VARIANT) USING DELTA") + sql("INSERT INTO tbl (SELECT 'foo', parse_json(cast(id + 99 as string)) FROM range(1))") + assert(spark.table("tbl").selectExpr("v::int").head == Row(99)) + assertVariantTypeTableFeatures( + "tbl", expectPreviewFeature = false, expectStableFeature = true) + } + } + + test("manually adding preview table feature does not require adding stable table feature") { + withTable("tbl") { + sql("CREATE TABLE tbl(s STRING) USING delta") + sql( + s"ALTER TABLE tbl " + + s"SET TBLPROPERTIES('delta.feature.variantType-preview' = 'supported')" + ) + + sql("ALTER TABLE tbl ADD COLUMN v VARIANT") + + sql("INSERT INTO tbl (SELECT 'foo', parse_json(cast(id + 99 as string)) FROM range(1))") + assert(spark.table("tbl").selectExpr("v::int").head == Row(99)) + + assertVariantTypeTableFeatures( + "tbl", + expectPreviewFeature = true, + expectStableFeature = false + ) + } + } + + test("creating table with preview feature does not add stable feature") { + withTable("tbl") { + sql(s"""CREATE TABLE tbl(v VARIANT) + USING delta + TBLPROPERTIES('delta.feature.variantType-preview' = 'supported')""" + ) + sql("INSERT INTO tbl (SELECT parse_json(cast(id + 99 as string)) FROM range(1))") + assertVariantTypeTableFeatures( + "tbl", + expectPreviewFeature = true, + expectStableFeature = false + ) + } + } + + test("enabling 'FORCE_USE_PREVIEW_VARIANT_FEATURE' adds preview table feature for new table") { + withSQLConf(DeltaSQLConf.FORCE_USE_PREVIEW_VARIANT_FEATURE.key -> "true") { + withTable("tbl") { + sql("CREATE TABLE tbl(s STRING, v VARIANT) USING DELTA") + sql("INSERT INTO tbl (SELECT 'foo', parse_json(cast(id + 99 as string)) FROM range(1))") + assert(spark.table("tbl").selectExpr("v::int").head == Row(99)) + assertVariantTypeTableFeatures( + "tbl", expectPreviewFeature = true, expectStableFeature = false) + } + } + } + + test("enabling 'FORCE_USE_PREVIEW_VARIANT_FEATURE' and adding a variant column hints to add " + + " the preview table feature") { + withSQLConf(DeltaSQLConf.FORCE_USE_PREVIEW_VARIANT_FEATURE.key -> "true") { + withTable("tbl") { + sql("CREATE TABLE tbl(s STRING) USING delta") + + val e = intercept[SparkThrowable] { + sql("ALTER TABLE tbl ADD COLUMN v VARIANT") + } + + checkError( + e, + "DELTA_FEATURES_REQUIRE_MANUAL_ENABLEMENT", + parameters = Map( + "unsupportedFeatures" -> VariantTypePreviewTableFeature.name, + "supportedFeatures" -> DeltaLog.forTable(spark, TableIdentifier("tbl")) + .unsafeVolatileSnapshot + .protocol + .implicitlyAndExplicitlySupportedFeatures + .map(_.name) + .toSeq + .sorted + .mkString(", ") + ) + ) + + sql( + s"ALTER TABLE tbl " + + s"SET TBLPROPERTIES('delta.feature.variantType-preview' = 'supported')" + ) + sql("ALTER TABLE tbl ADD COLUMN v VARIANT") + sql("INSERT INTO tbl (SELECT 'foo', parse_json(cast(id + 99 as string)) FROM range(1))") + assert(spark.table("tbl").selectExpr("v::int").head == Row(99)) + + assertVariantTypeTableFeatures( + "tbl", + expectPreviewFeature = true, + expectStableFeature = false + ) + } + } + } + + test("enabling 'FORCE_USE_PREVIEW_VARIANT_FEATURE' on table with stable feature does not " + + "require adding preview feature") { + withTable("tbl") { + sql("CREATE TABLE tbl(s STRING, v VARIANT) USING DELTA") + sql("INSERT INTO tbl (SELECT 'foo', parse_json(cast(id + 99 as string)) FROM range(1))") + assert(spark.table("tbl").selectExpr("v::int").head == Row(99)) + assertVariantTypeTableFeatures( + "tbl", expectPreviewFeature = false, expectStableFeature = true) + + withSQLConf(DeltaSQLConf.FORCE_USE_PREVIEW_VARIANT_FEATURE.key -> "true") { + sql("INSERT INTO tbl (SELECT 'foo', parse_json(cast(id + 99 as string)) FROM range(1))") + assert(spark.table("tbl").selectExpr("v::int").count == 2) + assertVariantTypeTableFeatures( + "tbl", expectPreviewFeature = false, expectStableFeature = true) + } + } + } + test("VariantType may not be used as a partition column") { withTable("delta_test") { checkError( @@ -305,8 +425,8 @@ class DeltaVariantSuite .selectExpr("tableFeatures") .collect()(0) .getAs[MutableSeq[String]](0) - assert(tableFeatures.find(f => f == VariantTypePreviewTableFeature.name).nonEmpty) - assert(tableFeatures.find(f => f == VariantTypeTableFeature.name).isEmpty) + assert(tableFeatures.find(f => f == VariantTypePreviewTableFeature.name).isEmpty) + assert(tableFeatures.find(f => f == VariantTypeTableFeature.name).nonEmpty) } } From 4676bf46beae6ffb3cf2a56e936c6412177e1e3f Mon Sep 17 00:00:00 2001 From: Venki Korukanti Date: Wed, 22 Jan 2025 19:27:19 -0800 Subject: [PATCH 10/28] [Build] Changes to avoid making too many requests to central maven repo (#4082) ## Description * Google maven repo URL is changed from `https://maven-central.storage-download.googleapis.com/repos/central/data/` to `https://maven-central.storage-download.googleapis.com/maven2/`. This caused more load on the central maven repo as the google maven repo is wrong. * Currently we hard coded the URL to central maven repo for downloading the `sbt-launch-x.x.x.jar` as part of the build start. Change it to google maven repo which seems have better rate limits than the central maven repo. Also add the central maven repo as a backup so that we have two options. Example retrying: ``` (base) delta2 % ./build/sbt clean package test:package Attempting to fetch sbt from https://eemaven-central.storage-download.googleapis.com/maven2/org/scala-sbt/sbt-launch/1.9.9/sbt-launch-1.9.9.jar Download from https://eemaven-central.storage-download.googleapis.com/maven2/org/scala-sbt/sbt-launch/1.9.9/sbt-launch-1.9.9.jar failed. Retrying from https://repo1.maven.org/maven2/org/scala-sbt/sbt-launch/1.9.9/sbt-launch-1.9.9.jar Launching sbt from build/sbt-launch-1.9.9.jar ``` * Add Google repo to iceberg build so that it is attempted first before the central maven repo and also as additional backup. * Testing * Without any repo changes: ``` (base) iceberg_src % rm -rf $HOME/.gradle/caches/ (base) iceberg_src % ./gradlew clean --refresh-dependencies :iceberg-core:build --info | grep Downloading | grep "mockito-core-4.0.0.jar" Downloading https://repo.maven.apache.org/maven2/org/mockito/mockito-core/4.0.0/mockito-core-4.0.0.jar to /Users/venkateshwar.korukanti/.gradle/.tmp/gradle_download6315884371983156241bin ``` * By adding google repo as a first in the list ``` (base) iceberg_src % git diff diff --git a/build.gradle b/build.gradle index 12ed701..c80bbd3 100644 --- a/build.gradle +++ b/build.gradle @@ -97,6 +97,9 @@ allprojects { group = "org.apache.iceberg" version = projectVersion repositories { + maven { + url "https://maven-central.storage-download.googleapis.com/maven2" + } mavenCentral() mavenLocal() } (base) iceberg_src % rm -rf $HOME/.gradle/caches/ (base) iceberg_src % ./gradlew clean --refresh-dependencies :iceberg-core:build --info | grep Downloading | grep "mockito-core-4.0.0.jar" Downloading https://maven-central.storage-download.googleapis.com/maven2/org/mockito/mockito-core/4.0.0/mockito-core-4.0.0.jar to /Users/venkateshwar.korukanti/.gradle/.tmp/gradle_download7065461132604582238bin ``` --- build/.sbt-launch-lib.bash.swo | Bin 0 -> 20480 bytes build/sbt-config/repositories | 2 +- build/sbt-launch-lib.bash | 50 +++++++++++++++---------- icebergShaded/generate_iceberg_jars.py | 25 +++++++++++++ 4 files changed, 56 insertions(+), 21 deletions(-) create mode 100644 build/.sbt-launch-lib.bash.swo diff --git a/build/.sbt-launch-lib.bash.swo b/build/.sbt-launch-lib.bash.swo new file mode 100644 index 0000000000000000000000000000000000000000..5f16778328e16b9430ac66514fa92a4027cfd257 GIT binary patch literal 20480 zcmeI4Ym6jS6@Uxh$U^~F&=|ej)64_5x~B(LVU}JPc4uXm#|$&GE4aI9Yr1N>Yp1)a zsj8mYVRnGv6TuJ@AE-g&4^)hai5lY{e1=Cb_yRO0AfQAeLBT(S#0P%oR#iV{hnFE5 zlWO?ru3NXxJ@?#Ox6V0rhxyTcyDwGe7Df#|Z#ImJZv15JcgmZOF_s)}(Tbd~ zzH9}BML%dST3+Nz#s1IbqtoxX;@n;P-g`xV`TC=4HWpa)!?0*OjmR1+&bQr$T@2?V zvthNpYTazO^M!dUtgn}q9cLEEEbuZc(2f?&b51bM8Y}5reIU14opSQtmmwglnpq&T zKxTo=0+|Ie3uG3^ERb0sv%vqm1)|ogjce(|tCEgvO}>v=_x+mWcq;iWtULeefuffG0nHdJ5| zE`$j<8wOxAoC;4KPy6sF{0M#k--mC&t#BjMp$3!iT6l~;+yOVkBG_;WydI=~C&6>{ z^$GYadm87qxufd33mZyOM^r6vTFSgi4XL4R za)Wrf=>B4}TJnU54asx2=^;A&} z*K@{mQfCzz1F-_qgh*~TxfMc1pXFwjSLCTnko?ay4M(R=23df7Z<>lo<$cki{a-AR4ifO|P0e0N1 zOhz+SDTfIYoA;}hwq22SsC?nXj*ItBSN6_aqHko~pL7-^9Ut16>Z|*FdZ-q3R|;yz ziGmd|NOIzUHzYCD|*S}{?0?G@1&g*(fNR+*V1 zmMm4a^3SEaXJ)2mDp)+8i@J6^U`f5y;yG0=t6ZtDwXjtNPZj|tzv;=$dwzM{`$6&P zZGVH7U!dL5Lg}nR>6{H)UN5P)%>gy*M6ymbEgtAR?zo8>jd`oOD9VH~g?gq{IX`SI zFDi4F%9V3!NT$Mp0?TCR`i3bTz?QPcn5Fcv#Si%xzl-nqzq%kU?!AKF$;?S*$k}w2 zyV7s7Kxr?g&SO$l+d)IIWH00Rq>GeeZ&7Xi)axu~mT$8fQa4!*J44NLbxDRsyqk(V zZ{wB0Z%3_ml-$o+wrO6~c4;G)qRgrFR}BsxFk}wN=&1Z~`YbS)v~D9RUGStgFQrx8 zGt-pCOqr%Qdy(sV{0!03>nO5)V?B=6LgaO|FjD`q({b@Ul5Q7TRuIW89J`otX*oaE zDM1uonUUU@dbQThrhYKVLv#Iu(g^|-#RJezn64|1z>7U!JC`kq@7B3Ydk>}UgJZEw zIh2~Q3d!uTood4h9HqUNYSS*u52G{iN1e{Gs_vVpTr_p*WO3g0ic&pZ&tqTZGCy** z@DO)H+-07GIMF!5LEzf9-83Eo8ha3 zy@jdY9M?9qnLAAcDy#->E2`LTz)aFsEYNB$wfOVZ9OeSg!e+UABB5nP_1s900j}np z2+XBj9#C^rJExSXX6t^tLHHt}NKnI}!o_OhI>};{C=|ywdf13R0+zS)32c>&=D3=Z z(F|iDjAuK<17aj2-*1FCW{Zxhxsel`Q^X~UwNpIBB{-Vkw3a4IOz1XS-1jR8&<}gG z5C!5L^Z0D~f$yVSpz!Awd8UO0lrRQ17u#W=ht%a>mPzU;OlTZ+zQvjLD&vw!X|hQTRjf84y>w~GIq)7KBEpzTQ(Lqftdo3(=^a#YR_ zP=v!vh$6k)u-_y#$!CSyg`^VkK%5`brP_VNOU3)vsp(oM{x;4vq7W<9#oH1mFBPsM z(C9S5frvu3!G)lIjo6(ZUw}nTH3m=j5R-HjWPti1t#P#=l zqKe>r)vlX9$sS)P~Q+a7TMdJknJIC>i zbT95=YQ5%4YQWXzin!_BX_XpAS11ol+M8fgS1J#0I4V-b4J!-$b*BWF6{du?|N6#s zHj}!YY^@VoRgQZ&!dXg>`o!Qroum)i$T8-JgMe;re@G>3fK>Fg33{ z#)#FQSNhJr+Pneitr;uQ4cJDQQ1MV$Q9D6^_x;L*A#b8;g|-SIHzEwYo0 z2Z?5N9)JA1RP@@+^j@P$qpVF!O;ZQ=>)Msq8GSZmB6Qx^Y-AKIP9*BJsn`DG{T+GF z(^@XXMsf+K45a_K~#9y65H6QFbwx zj>rJZdx8;_{MewHXjngD)}2OcgkX_Ca9f-yE2VcwRg&$x?Xx%+!wvUH+sS&f+V+f`>QjSX({?gG=Ba*JOL#EWR2Crf&BgzCI}w9<06 zcNYGfk|YdzO1g48Ra zoPv^ODd9z6??maUA1iBK(F}Qg<}}xuirn6Fkj zuFCORQ@@^y+v}8)>Ks=2mJ>8pQH`k)SuHyy=c%(t&wV$AvXo1mvK;r;*sjL5_Eq4p z(60{QoQ7HxEh-OZ-2TI4}Q_*dG1?e}YfLryziL!M}*%Z-Z-K8GcIa zeizKZ=@E?b( z;UXANgL$*cSjLf8#YVCNr0_Q@=eSs=5(D`SE5nI+!J z3+>E#srAS<|0g?mUWWKraw}2V>?1z`zZ}Ex(;f#`zhP~B*D-g*(KE7dBKg-+3vtSla$u0bsc0YaK`8Gd!MYXCoDyjU~+K?&~$mYB+vUuzlv=i0h7z literal 0 HcmV?d00001 diff --git a/build/sbt-config/repositories b/build/sbt-config/repositories index b12fece5fd5..f5fcaa71859 100644 --- a/build/sbt-config/repositories +++ b/build/sbt-config/repositories @@ -2,7 +2,7 @@ local local-preloaded-ivy: file:///${sbt.preloaded-${sbt.global.base-${user.home}/.sbt}/preloaded/}, [organization]/[module]/[revision]/[type]s/[artifact](-[classifier]).[ext] local-preloaded: file:///${sbt.preloaded-${sbt.global.base-${user.home}/.sbt}/preloaded/} - gcs-maven-central-mirror: https://maven-central.storage-download.googleapis.com/repos/central/data/ + gcs-maven-central-mirror: https://maven-central.storage-download.googleapis.com/maven2/ maven-central typesafe-ivy-releases: https://repo.typesafe.com/typesafe/ivy-releases/, [organization]/[module]/[revision]/[type]s/[artifact](-[classifier]).[ext], bootOnly sbt-ivy-snapshots: https://repo.scala-sbt.org/scalasbt/ivy-snapshots/, [organization]/[module]/[revision]/[type]s/[artifact](-[classifier]).[ext], bootOnly diff --git a/build/sbt-launch-lib.bash b/build/sbt-launch-lib.bash index 0d58bb7269c..f2f501e0a7d 100755 --- a/build/sbt-launch-lib.bash +++ b/build/sbt-launch-lib.bash @@ -36,42 +36,52 @@ dlog () { [[ $debug ]] && echoerr "$@" } +download_sbt () { + local url=$1 + local output=$2 + local temp_file="${output}.part" + + if [ $(command -v curl) ]; then + curl --fail --location --silent ${url} > "${temp_file}" &&\ + mv "${temp_file}" "${output}" + elif [ $(command -v wget) ]; then + wget --quiet ${url} -O "${temp_file}" &&\ + mv "${temp_file}" "${output}" + else + printf "You do not have curl or wget installed, unable to downlaod ${url}\n" + exit -1 + fi +} + + acquire_sbt_jar () { SBT_VERSION=`awk -F "=" '/sbt\.version/ {print $2}' ./project/build.properties` - # Download sbt from mirror URL if the environment variable is provided + # Set primary and fallback URLs if [[ "${SBT_VERSION}" == "0.13.18" ]] && [[ -n "${SBT_MIRROR_JAR_URL}" ]]; then URL1="${SBT_MIRROR_JAR_URL}" elif [[ "${SBT_VERSION}" == "1.5.5" ]] && [[ -n "${SBT_1_5_5_MIRROR_JAR_URL}" ]]; then URL1="${SBT_1_5_5_MIRROR_JAR_URL}" else - URL1=${DEFAULT_ARTIFACT_REPOSITORY:-https://repo1.maven.org/maven2/}org/scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch-${SBT_VERSION}.jar + URL1=${DEFAULT_ARTIFACT_REPOSITORY:-https://maven-central.storage-download.googleapis.com/maven2/}org/scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch-${SBT_VERSION}.jar fi + BACKUP_URL="https://repo1.maven.org/maven2/org/scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch-${SBT_VERSION}.jar" JAR=build/sbt-launch-${SBT_VERSION}.jar sbt_jar=$JAR if [[ ! -f "$sbt_jar" ]]; then - # Download sbt launch jar if it hasn't been downloaded yet - if [ ! -f "${JAR}" ]; then - # Download printf 'Attempting to fetch sbt from %s\n' "${URL1}" - JAR_DL="${JAR}.part" - if [ $(command -v curl) ]; then - curl --fail --location --silent ${URL1} > "${JAR_DL}" &&\ - mv "${JAR_DL}" "${JAR}" - elif [ $(command -v wget) ]; then - wget --quiet ${URL1} -O "${JAR_DL}" &&\ - mv "${JAR_DL}" "${JAR}" - else - printf "You do not have curl or wget installed, please install sbt manually from https://www.scala-sbt.org/\n" - exit -1 - fi + download_sbt "${URL1}" "${JAR}" + + if [[ ! -f "${JAR}" ]]; then + printf 'Download from %s failed. Retrying from %s\n' "${URL1}" "${BACKUP_URL}" + download_sbt "${BACKUP_URL}" "${JAR}" fi - if [ ! -f "${JAR}" ]; then - # We failed to download - printf "Our attempt to download sbt locally to ${JAR} failed. Please install sbt manually from https://www.scala-sbt.org/\n" - exit -1 + + if [[ ! -f "${JAR}" ]]; then + printf "Failed to download sbt. Please install sbt manually from https://www.scala-sbt.org/\n" + exit 1 fi printf "Launching sbt from ${JAR}\n" fi diff --git a/icebergShaded/generate_iceberg_jars.py b/icebergShaded/generate_iceberg_jars.py index 0c2579eb4e3..8bc2fc2dfae 100644 --- a/icebergShaded/generate_iceberg_jars.py +++ b/icebergShaded/generate_iceberg_jars.py @@ -19,6 +19,7 @@ import argparse import os import glob +import re import subprocess import shlex import shutil @@ -65,6 +66,28 @@ def iceberg_jars_exists(): return True +def add_google_maven_repo_to_gradle_config(): + with WorkingDirectory(iceberg_src_dir): + file_path = 'build.gradle' + + with open(file_path, 'r') as file: + content = file.read() + + # Define the old and new configurations + old_config = r'repositories {\n mavenCentral\(\)' + + new_config = 'repositories {\n maven {\n ' + \ + 'url "https://maven-central.storage-download.googleapis.com/maven2"\n }\n ' + \ + 'mavenCentral()' + + # Replace the old configuration with the new one + updated_content = re.sub(old_config, new_config, content, flags=re.DOTALL) + + # Write the updated content back to the file + with open(file_path, 'w') as file: + file.write(updated_content) + + def prepare_iceberg_source(): with WorkingDirectory(iceberg_root_dir): print(">>> Cloning Iceberg repo") @@ -92,6 +115,8 @@ def prepare_iceberg_source(): run_cmd("git add .") run_cmd("git commit -a -m 'applied %s'" % path.basename(patch_file)) + add_google_maven_repo_to_gradle_config() + def generate_iceberg_jars(): print(">>> Compiling JARs") From ed71f8a4b5327c067417526908c3c646fe3bf980 Mon Sep 17 00:00:00 2001 From: Scott Sandre Date: Thu, 23 Jan 2025 12:52:03 -0800 Subject: [PATCH 11/28] [Kernel] Clean up `FileNames.java` and add more tests (#4086) #### Which Delta project/connector is this regarding? - [ ] Spark - [ ] Standalone - [ ] Flink - [X] Kernel - [ ] Other (fill in here) ## Description Clean up `FileNames.java` and add more tests ## How was this patch tested? New UTs. ## Does this PR introduce _any_ user-facing changes? No. --- .../delta/kernel/internal/util/FileNames.java | 108 ++++++++++----- .../kernel/internal/util/FileNamesSuite.scala | 131 ++++++++++++++++-- 2 files changed, 189 insertions(+), 50 deletions(-) diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/FileNames.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/FileNames.java index 93b1e918ecd..f0077f840c1 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/FileNames.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/FileNames.java @@ -25,28 +25,71 @@ public final class FileNames { private FileNames() {} + //////////////////////////////////////////////// + // File name patterns and other static values // + //////////////////////////////////////////////// + + /** Example: 00000000000000000001.json */ private static final Pattern DELTA_FILE_PATTERN = Pattern.compile("\\d+\\.json"); - // Example: 00000000000000000001.dc0f9f58-a1a0-46fd-971a-bd8b2e9dbb81.json + /** Example: 00000000000000000001.dc0f9f58-a1a0-46fd-971a-bd8b2e9dbb81.json */ private static final Pattern UUID_DELTA_FILE_REGEX = Pattern.compile("(\\d+)\\.([^\\.]+)\\.json"); + /** + * Examples: + * + *

    + *
  • Classic V1 - 00000000000000000001.checkpoint.parquet + *
  • Multi-part V1 - 00000000000000000001.checkpoint.0000000001.0000000010.parquet + *
  • V2 JSON - 00000000000000000001.checkpoint.uuid-1234abcd.json + *
  • V2 Parquet - 00000000000000000001.checkpoint.uuid-1234abcd.parquet + *
+ */ private static final Pattern CHECKPOINT_FILE_PATTERN = Pattern.compile("(\\d+)\\.checkpoint((\\.\\d+\\.\\d+)?\\.parquet|\\.[^.]+\\.(json|parquet))"); + /** Example: 00000000000000000001.checkpoint.parquet */ private static final Pattern CLASSIC_CHECKPOINT_FILE_PATTERN = Pattern.compile("\\d+\\.checkpoint\\.parquet"); + /** + * Examples: + * + *
    + *
  • 00000000000000000001.checkpoint.dc0f9f58-a1a0-46fd-971a-bd8b2e9dbb81.json + *
  • 00000000000000000001.checkpoint.dc0f9f58-a1a0-46fd-971a-bd8b2e9dbb81.parquet + *
+ */ private static final Pattern V2_CHECKPOINT_FILE_PATTERN = Pattern.compile("(\\d+)\\.checkpoint\\.[^.]+\\.(json|parquet)"); + /** Example: 00000000000000000001.checkpoint.0000000020.0000000060.parquet */ private static final Pattern MULTI_PART_CHECKPOINT_FILE_PATTERN = Pattern.compile("(\\d+)\\.checkpoint\\.\\d+\\.\\d+\\.parquet"); public static final String SIDECAR_DIRECTORY = "_sidecars"; - /** Returns the delta (json format) path for a given delta file. */ - public static String deltaFile(Path path, long version) { - return String.format("%s/%020d.json", path, version); + //////////////////////// + // Version extractors // + //////////////////////// + + /** + * Get the version of the checkpoint, checksum or delta file. Throws an error if an unexpected + * file type is seen. These unexpected files should be filtered out to ensure forward + * compatibility in cases where new file types are added, but without an explicit protocol + * upgrade. + */ + public static long getFileVersion(Path path) { + if (isCheckpointFile(path.getName())) { + return checkpointVersion(path); + } else if (isCommitFile(path.getName())) { + return deltaVersion(path); + // } else if (isChecksumFile(path)) { + // checksumVersion(path); + } else { + throw new IllegalArgumentException( + String.format("Unexpected file type found in transaction log: %s", path)); + } } /** Returns the version for the given delta path. */ @@ -71,6 +114,16 @@ public static long checkpointVersion(String path) { return Long.parseLong(name.split("\\.")[0]); } + /////////////////////////////////// + // File path and prefix builders // + /////////////////////////////////// + + /** Returns the delta (json format) path for a given delta file. */ + public static String deltaFile(Path path, long version) { + return String.format("%s/%020d.json", path, version); + } + + /** Example: /a/_sidecars/3a0d65cd-4056-49b8-937b-95f9e3ee90e5.parquet */ public static String sidecarFile(Path path, String sidecar) { return String.format("%s/%s/%s", path.toString(), SIDECAR_DIRECTORY, sidecar); } @@ -106,7 +159,7 @@ public static Path topLevelV2CheckpointFile( /** Returns the path for a V2 sidecar file with a given UUID. */ public static Path v2CheckpointSidecarFile(Path path, String uuid) { - return new Path(String.format("%s/_sidecars/%s.parquet", path.toString(), uuid)); + return new Path(String.format("%s/%s/%s.parquet", path.toString(), SIDECAR_DIRECTORY, uuid)); } /** @@ -128,44 +181,29 @@ public static List checkpointFileWithParts(Path path, long version, int nu return output; } - public static boolean isCheckpointFile(String fileName) { - return CHECKPOINT_FILE_PATTERN.matcher(new Path(fileName).getName()).matches(); - } + ///////////////////////////// + // Is file checkers // + ///////////////////////////// - public static boolean isClassicCheckpointFile(String fileName) { - return CLASSIC_CHECKPOINT_FILE_PATTERN.matcher(fileName).matches(); + public static boolean isCheckpointFile(String path) { + return CHECKPOINT_FILE_PATTERN.matcher(new Path(path).getName()).matches(); } - public static boolean isMultiPartCheckpointFile(String fileName) { - return MULTI_PART_CHECKPOINT_FILE_PATTERN.matcher(fileName).matches(); + public static boolean isClassicCheckpointFile(String path) { + return CLASSIC_CHECKPOINT_FILE_PATTERN.matcher(new Path(path).getName()).matches(); } - public static boolean isV2CheckpointFile(String fileName) { - return V2_CHECKPOINT_FILE_PATTERN.matcher(fileName).matches(); + public static boolean isMultiPartCheckpointFile(String path) { + return MULTI_PART_CHECKPOINT_FILE_PATTERN.matcher(new Path(path).getName()).matches(); } - public static boolean isCommitFile(String fileName) { - String filename = new Path(fileName).getName(); - return DELTA_FILE_PATTERN.matcher(filename).matches() - || UUID_DELTA_FILE_REGEX.matcher(filename).matches(); + public static boolean isV2CheckpointFile(String path) { + return V2_CHECKPOINT_FILE_PATTERN.matcher(new Path(path).getName()).matches(); } - /** - * Get the version of the checkpoint, checksum or delta file. Throws an error if an unexpected - * file type is seen. These unexpected files should be filtered out to ensure forward - * compatibility in cases where new file types are added, but without an explicit protocol - * upgrade. - */ - public static long getFileVersion(Path path) { - if (isCheckpointFile(path.getName())) { - return checkpointVersion(path); - } else if (isCommitFile(path.getName())) { - return deltaVersion(path); - // } else if (isChecksumFile(path)) { - // checksumVersion(path); - } else { - throw new IllegalArgumentException( - String.format("Unexpected file type found in transaction log: %s", path)); - } + public static boolean isCommitFile(String path) { + final String fileName = new Path(path).getName(); + return DELTA_FILE_PATTERN.matcher(fileName).matches() + || UUID_DELTA_FILE_REGEX.matcher(fileName).matches(); } } diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/FileNamesSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/FileNamesSuite.scala index 8bb30795ba4..dd33865e78b 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/FileNamesSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/util/FileNamesSuite.scala @@ -23,29 +23,75 @@ import scala.collection.JavaConverters._ class FileNamesSuite extends AnyFunSuite { - test("isCheckpointFile") { - assert(isCheckpointFile("/a/123.checkpoint.parquet")) - assert(isCheckpointFile("/a/123.checkpoint.0000000001.0000000087.parquet")) - assert(isCheckpointFile("/a/000000010.checkpoint.80a083e8-7026.json")) - assert(isCheckpointFile("/a/000000010.checkpoint.80a083e8-7026.parquet")) - assert(!isCheckpointFile("/a/123.json")) - assert(!isCommitFile("/a/123.checkpoint.3.json")) - } + private val checkpointV1 = "/a/123.checkpoint.parquet" + private val checkpointMultiPart = "/a/123.checkpoint.0000000001.0000000087.parquet" + private val checkpointV2Json = "/a/000000010.checkpoint.80a083e8-7026.json" + private val checkpointV2Parquet = "/a/000000010.checkpoint.80a083e8-7026.parquet" + private val commitNormal = "/a/0000000088.json" + private val commitUUID = "/a/00000022.dc0f9f58-a1a0.json" + + ///////////////////////////// + // Version extractor tests // + ///////////////////////////// test("checkpointVersion") { - assert(checkpointVersion(new Path("/a/123.checkpoint.parquet")) == 123) - assert(checkpointVersion(new Path("/a/0.checkpoint.parquet")) == 0) - assert(checkpointVersion( - new Path("/a/00000000000000000151.checkpoint.parquet")) == 151) - assert(checkpointVersion( - new Path("/a/999.checkpoint.0000000090.0000000099.parquet")) == 999) - assert(checkpointVersion("/a/000000010.checkpoint.80a083e8-7026.json") == 10) + assert(checkpointVersion(new Path(checkpointV1)) == 123) + assert(checkpointVersion(new Path(checkpointMultiPart)) == 123) + assert(checkpointVersion(new Path(checkpointV2Json)) == 10) + assert(checkpointVersion(new Path(checkpointV2Parquet)) == 10) + } + + test("deltaVersion") { + assert(deltaVersion(new Path(commitNormal)) == 88) + assert(deltaVersion(new Path(commitUUID)) == 22) + } + + test("getFileVersion") { + assert(getFileVersion(new Path(checkpointV1)) == 123) + assert(getFileVersion(new Path(checkpointMultiPart)) == 123) + assert(getFileVersion(new Path(checkpointV2Json)) == 10) + assert(getFileVersion(new Path(checkpointV2Parquet)) == 10) + assert(getFileVersion(new Path(commitNormal)) == 88) + assert(getFileVersion(new Path(commitUUID)) == 22) + } + + ///////////////////////////////////////// + // File path and prefix builders tests // + ///////////////////////////////////////// + + test("deltaFile") { + assert(deltaFile(new Path("/a"), 1234) == "/a/00000000000000001234.json") + } + + test("sidecarFile") { + assert(sidecarFile(new Path("/a"), "7d17ac10.parquet") == "/a/_sidecars/7d17ac10.parquet") } test("listingPrefix") { assert(listingPrefix(new Path("/a"), 1234) == "/a/00000000000000001234.") } + test("checkpointFileSingular") { + assert( + checkpointFileSingular(new Path("/a"), 1234).toString == + "/a/00000000000000001234.checkpoint.parquet") + } + + test("topLevelV2CheckpointFile") { + assert( + topLevelV2CheckpointFile(new Path("/a"), 1234, "7d17ac10", "json").toString == + "/a/00000000000000001234.checkpoint.7d17ac10.json") + assert( + topLevelV2CheckpointFile(new Path("/a"), 1234, "7d17ac10", "parquet").toString == + "/a/00000000000000001234.checkpoint.7d17ac10.parquet") + } + + test("v2CheckpointSidecarFile") { + assert( + v2CheckpointSidecarFile(new Path("/a"), "7d17ac10").toString == + "/a/_sidecars/7d17ac10.parquet") + } + test("checkpointFileWithParts") { assert(checkpointFileWithParts(new Path("/a"), 1, 1).asScala == Seq( new Path("/a/00000000000000000001.checkpoint.0000000001.0000000001.parquet"))) @@ -59,4 +105,59 @@ class FileNamesSuite extends AnyFunSuite { new Path("/a/00000000000000000001.checkpoint.0000000004.0000000005.parquet"), new Path("/a/00000000000000000001.checkpoint.0000000005.0000000005.parquet"))) } + + /////////////////////////////////// + // Is file checkers tests // + /////////////////////////////////// + + test("is checkpoint file") { + // ===== V1 checkpoint ===== + // Positive cases + assert(isCheckpointFile(checkpointV1)) + assert(isCheckpointFile(new Path(checkpointV1).getName)) + assert(isClassicCheckpointFile(checkpointV1)) + assert(isClassicCheckpointFile(new Path(checkpointV1).getName)) + // Negative cases + assert(!isMultiPartCheckpointFile(checkpointV1)) + assert(!isV2CheckpointFile(checkpointV1)) + assert(!isCommitFile(checkpointV1)) + + // ===== Multipart checkpoint ===== + // Positive cases + assert(isCheckpointFile(checkpointMultiPart)) + assert(isCheckpointFile(new Path(checkpointMultiPart).getName)) + assert(isMultiPartCheckpointFile(checkpointMultiPart)) + assert(isMultiPartCheckpointFile(new Path(checkpointMultiPart).getName)) + // Negative cases + assert(!isClassicCheckpointFile(checkpointMultiPart)) + assert(!isV2CheckpointFile(checkpointMultiPart)) + assert(!isCommitFile(checkpointMultiPart)) + + // ===== V2 checkpoint ===== + // Positive cases + assert(isCheckpointFile(checkpointV2Json)) + assert(isCheckpointFile(new Path(checkpointV2Json).getName)) + assert(isV2CheckpointFile(checkpointV2Json)) + assert(isV2CheckpointFile(new Path(checkpointV2Json).getName)) + assert(isCheckpointFile(checkpointV2Parquet)) + assert(isCheckpointFile(new Path(checkpointV2Parquet).getName)) + assert(isV2CheckpointFile(checkpointV2Parquet)) + assert(isV2CheckpointFile(new Path(checkpointV2Parquet).getName)) + // Negative cases + assert(!isClassicCheckpointFile(checkpointV2Json)) + assert(!isClassicCheckpointFile(checkpointV2Parquet)) + assert(!isMultiPartCheckpointFile(checkpointV2Json)) + assert(!isMultiPartCheckpointFile(checkpointV2Parquet)) + assert(!isCommitFile(checkpointV2Json)) + assert(!isCommitFile(checkpointV2Parquet)) + + // ===== Others ===== + assert(!isCheckpointFile("/a/123.json")) + assert(!isCommitFile("/a/123.checkpoint.3.json")) + } + + test("is commit file") { + assert(isCommitFile(commitNormal)) + assert(isCommitFile(commitUUID)) + } } From cc85dfd19b66a78105c2b8e72189d53b174d95c7 Mon Sep 17 00:00:00 2001 From: Venki Korukanti Date: Thu, 23 Jan 2025 13:29:52 -0800 Subject: [PATCH 12/28] Revert "[Build] Changes to avoid making too many requests to central maven repo" (#4085) Reverts delta-io/delta#4082 (few unncessary changes are included). Will make a new PR. --- build/.sbt-launch-lib.bash.swo | Bin 20480 -> 0 bytes build/sbt-config/repositories | 2 +- build/sbt-launch-lib.bash | 50 ++++++++++--------------- icebergShaded/generate_iceberg_jars.py | 25 ------------- 4 files changed, 21 insertions(+), 56 deletions(-) delete mode 100644 build/.sbt-launch-lib.bash.swo diff --git a/build/.sbt-launch-lib.bash.swo b/build/.sbt-launch-lib.bash.swo deleted file mode 100644 index 5f16778328e16b9430ac66514fa92a4027cfd257..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 20480 zcmeI4Ym6jS6@Uxh$U^~F&=|ej)64_5x~B(LVU}JPc4uXm#|$&GE4aI9Yr1N>Yp1)a zsj8mYVRnGv6TuJ@AE-g&4^)hai5lY{e1=Cb_yRO0AfQAeLBT(S#0P%oR#iV{hnFE5 zlWO?ru3NXxJ@?#Ox6V0rhxyTcyDwGe7Df#|Z#ImJZv15JcgmZOF_s)}(Tbd~ zzH9}BML%dST3+Nz#s1IbqtoxX;@n;P-g`xV`TC=4HWpa)!?0*OjmR1+&bQr$T@2?V zvthNpYTazO^M!dUtgn}q9cLEEEbuZc(2f?&b51bM8Y}5reIU14opSQtmmwglnpq&T zKxTo=0+|Ie3uG3^ERb0sv%vqm1)|ogjce(|tCEgvO}>v=_x+mWcq;iWtULeefuffG0nHdJ5| zE`$j<8wOxAoC;4KPy6sF{0M#k--mC&t#BjMp$3!iT6l~;+yOVkBG_;WydI=~C&6>{ z^$GYadm87qxufd33mZyOM^r6vTFSgi4XL4R za)Wrf=>B4}TJnU54asx2=^;A&} z*K@{mQfCzz1F-_qgh*~TxfMc1pXFwjSLCTnko?ay4M(R=23df7Z<>lo<$cki{a-AR4ifO|P0e0N1 zOhz+SDTfIYoA;}hwq22SsC?nXj*ItBSN6_aqHko~pL7-^9Ut16>Z|*FdZ-q3R|;yz ziGmd|NOIzUHzYCD|*S}{?0?G@1&g*(fNR+*V1 zmMm4a^3SEaXJ)2mDp)+8i@J6^U`f5y;yG0=t6ZtDwXjtNPZj|tzv;=$dwzM{`$6&P zZGVH7U!dL5Lg}nR>6{H)UN5P)%>gy*M6ymbEgtAR?zo8>jd`oOD9VH~g?gq{IX`SI zFDi4F%9V3!NT$Mp0?TCR`i3bTz?QPcn5Fcv#Si%xzl-nqzq%kU?!AKF$;?S*$k}w2 zyV7s7Kxr?g&SO$l+d)IIWH00Rq>GeeZ&7Xi)axu~mT$8fQa4!*J44NLbxDRsyqk(V zZ{wB0Z%3_ml-$o+wrO6~c4;G)qRgrFR}BsxFk}wN=&1Z~`YbS)v~D9RUGStgFQrx8 zGt-pCOqr%Qdy(sV{0!03>nO5)V?B=6LgaO|FjD`q({b@Ul5Q7TRuIW89J`otX*oaE zDM1uonUUU@dbQThrhYKVLv#Iu(g^|-#RJezn64|1z>7U!JC`kq@7B3Ydk>}UgJZEw zIh2~Q3d!uTood4h9HqUNYSS*u52G{iN1e{Gs_vVpTr_p*WO3g0ic&pZ&tqTZGCy** z@DO)H+-07GIMF!5LEzf9-83Eo8ha3 zy@jdY9M?9qnLAAcDy#->E2`LTz)aFsEYNB$wfOVZ9OeSg!e+UABB5nP_1s900j}np z2+XBj9#C^rJExSXX6t^tLHHt}NKnI}!o_OhI>};{C=|ywdf13R0+zS)32c>&=D3=Z z(F|iDjAuK<17aj2-*1FCW{Zxhxsel`Q^X~UwNpIBB{-Vkw3a4IOz1XS-1jR8&<}gG z5C!5L^Z0D~f$yVSpz!Awd8UO0lrRQ17u#W=ht%a>mPzU;OlTZ+zQvjLD&vw!X|hQTRjf84y>w~GIq)7KBEpzTQ(Lqftdo3(=^a#YR_ zP=v!vh$6k)u-_y#$!CSyg`^VkK%5`brP_VNOU3)vsp(oM{x;4vq7W<9#oH1mFBPsM z(C9S5frvu3!G)lIjo6(ZUw}nTH3m=j5R-HjWPti1t#P#=l zqKe>r)vlX9$sS)P~Q+a7TMdJknJIC>i zbT95=YQ5%4YQWXzin!_BX_XpAS11ol+M8fgS1J#0I4V-b4J!-$b*BWF6{du?|N6#s zHj}!YY^@VoRgQZ&!dXg>`o!Qroum)i$T8-JgMe;re@G>3fK>Fg33{ z#)#FQSNhJr+Pneitr;uQ4cJDQQ1MV$Q9D6^_x;L*A#b8;g|-SIHzEwYo0 z2Z?5N9)JA1RP@@+^j@P$qpVF!O;ZQ=>)Msq8GSZmB6Qx^Y-AKIP9*BJsn`DG{T+GF z(^@XXMsf+K45a_K~#9y65H6QFbwx zj>rJZdx8;_{MewHXjngD)}2OcgkX_Ca9f-yE2VcwRg&$x?Xx%+!wvUH+sS&f+V+f`>QjSX({?gG=Ba*JOL#EWR2Crf&BgzCI}w9<06 zcNYGfk|YdzO1g48Ra zoPv^ODd9z6??maUA1iBK(F}Qg<}}xuirn6Fkj zuFCORQ@@^y+v}8)>Ks=2mJ>8pQH`k)SuHyy=c%(t&wV$AvXo1mvK;r;*sjL5_Eq4p z(60{QoQ7HxEh-OZ-2TI4}Q_*dG1?e}YfLryziL!M}*%Z-Z-K8GcIa zeizKZ=@E?b( z;UXANgL$*cSjLf8#YVCNr0_Q@=eSs=5(D`SE5nI+!J z3+>E#srAS<|0g?mUWWKraw}2V>?1z`zZ}Ex(;f#`zhP~B*D-g*(KE7dBKg-+3vtSla$u0bsc0YaK`8Gd!MYXCoDyjU~+K?&~$mYB+vUuzlv=i0h7z diff --git a/build/sbt-config/repositories b/build/sbt-config/repositories index f5fcaa71859..b12fece5fd5 100644 --- a/build/sbt-config/repositories +++ b/build/sbt-config/repositories @@ -2,7 +2,7 @@ local local-preloaded-ivy: file:///${sbt.preloaded-${sbt.global.base-${user.home}/.sbt}/preloaded/}, [organization]/[module]/[revision]/[type]s/[artifact](-[classifier]).[ext] local-preloaded: file:///${sbt.preloaded-${sbt.global.base-${user.home}/.sbt}/preloaded/} - gcs-maven-central-mirror: https://maven-central.storage-download.googleapis.com/maven2/ + gcs-maven-central-mirror: https://maven-central.storage-download.googleapis.com/repos/central/data/ maven-central typesafe-ivy-releases: https://repo.typesafe.com/typesafe/ivy-releases/, [organization]/[module]/[revision]/[type]s/[artifact](-[classifier]).[ext], bootOnly sbt-ivy-snapshots: https://repo.scala-sbt.org/scalasbt/ivy-snapshots/, [organization]/[module]/[revision]/[type]s/[artifact](-[classifier]).[ext], bootOnly diff --git a/build/sbt-launch-lib.bash b/build/sbt-launch-lib.bash index f2f501e0a7d..0d58bb7269c 100755 --- a/build/sbt-launch-lib.bash +++ b/build/sbt-launch-lib.bash @@ -36,52 +36,42 @@ dlog () { [[ $debug ]] && echoerr "$@" } -download_sbt () { - local url=$1 - local output=$2 - local temp_file="${output}.part" - - if [ $(command -v curl) ]; then - curl --fail --location --silent ${url} > "${temp_file}" &&\ - mv "${temp_file}" "${output}" - elif [ $(command -v wget) ]; then - wget --quiet ${url} -O "${temp_file}" &&\ - mv "${temp_file}" "${output}" - else - printf "You do not have curl or wget installed, unable to downlaod ${url}\n" - exit -1 - fi -} - - acquire_sbt_jar () { SBT_VERSION=`awk -F "=" '/sbt\.version/ {print $2}' ./project/build.properties` - # Set primary and fallback URLs + # Download sbt from mirror URL if the environment variable is provided if [[ "${SBT_VERSION}" == "0.13.18" ]] && [[ -n "${SBT_MIRROR_JAR_URL}" ]]; then URL1="${SBT_MIRROR_JAR_URL}" elif [[ "${SBT_VERSION}" == "1.5.5" ]] && [[ -n "${SBT_1_5_5_MIRROR_JAR_URL}" ]]; then URL1="${SBT_1_5_5_MIRROR_JAR_URL}" else - URL1=${DEFAULT_ARTIFACT_REPOSITORY:-https://maven-central.storage-download.googleapis.com/maven2/}org/scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch-${SBT_VERSION}.jar + URL1=${DEFAULT_ARTIFACT_REPOSITORY:-https://repo1.maven.org/maven2/}org/scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch-${SBT_VERSION}.jar fi - BACKUP_URL="https://repo1.maven.org/maven2/org/scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch-${SBT_VERSION}.jar" JAR=build/sbt-launch-${SBT_VERSION}.jar sbt_jar=$JAR if [[ ! -f "$sbt_jar" ]]; then + # Download sbt launch jar if it hasn't been downloaded yet + if [ ! -f "${JAR}" ]; then + # Download printf 'Attempting to fetch sbt from %s\n' "${URL1}" - download_sbt "${URL1}" "${JAR}" - - if [[ ! -f "${JAR}" ]]; then - printf 'Download from %s failed. Retrying from %s\n' "${URL1}" "${BACKUP_URL}" - download_sbt "${BACKUP_URL}" "${JAR}" + JAR_DL="${JAR}.part" + if [ $(command -v curl) ]; then + curl --fail --location --silent ${URL1} > "${JAR_DL}" &&\ + mv "${JAR_DL}" "${JAR}" + elif [ $(command -v wget) ]; then + wget --quiet ${URL1} -O "${JAR_DL}" &&\ + mv "${JAR_DL}" "${JAR}" + else + printf "You do not have curl or wget installed, please install sbt manually from https://www.scala-sbt.org/\n" + exit -1 fi - - if [[ ! -f "${JAR}" ]]; then - printf "Failed to download sbt. Please install sbt manually from https://www.scala-sbt.org/\n" - exit 1 + fi + if [ ! -f "${JAR}" ]; then + # We failed to download + printf "Our attempt to download sbt locally to ${JAR} failed. Please install sbt manually from https://www.scala-sbt.org/\n" + exit -1 fi printf "Launching sbt from ${JAR}\n" fi diff --git a/icebergShaded/generate_iceberg_jars.py b/icebergShaded/generate_iceberg_jars.py index 8bc2fc2dfae..0c2579eb4e3 100644 --- a/icebergShaded/generate_iceberg_jars.py +++ b/icebergShaded/generate_iceberg_jars.py @@ -19,7 +19,6 @@ import argparse import os import glob -import re import subprocess import shlex import shutil @@ -66,28 +65,6 @@ def iceberg_jars_exists(): return True -def add_google_maven_repo_to_gradle_config(): - with WorkingDirectory(iceberg_src_dir): - file_path = 'build.gradle' - - with open(file_path, 'r') as file: - content = file.read() - - # Define the old and new configurations - old_config = r'repositories {\n mavenCentral\(\)' - - new_config = 'repositories {\n maven {\n ' + \ - 'url "https://maven-central.storage-download.googleapis.com/maven2"\n }\n ' + \ - 'mavenCentral()' - - # Replace the old configuration with the new one - updated_content = re.sub(old_config, new_config, content, flags=re.DOTALL) - - # Write the updated content back to the file - with open(file_path, 'w') as file: - file.write(updated_content) - - def prepare_iceberg_source(): with WorkingDirectory(iceberg_root_dir): print(">>> Cloning Iceberg repo") @@ -115,8 +92,6 @@ def prepare_iceberg_source(): run_cmd("git add .") run_cmd("git commit -a -m 'applied %s'" % path.basename(patch_file)) - add_google_maven_repo_to_gradle_config() - def generate_iceberg_jars(): print(">>> Compiling JARs") From d326793aac8b7226f0da493f8c52dc57a969442a Mon Sep 17 00:00:00 2001 From: Scott Sandre Date: Thu, 23 Jan 2025 16:23:48 -0800 Subject: [PATCH 13/28] [Kernel] Consolidate listing utilities code and simplify SnapshotManager log segment error case (#4035) #### Which Delta project/connector is this regarding? - [ ] Spark - [ ] Standalone - [ ] Flink - [X] Kernel - [ ] Other (fill in here) ## Description We used to have several duplicated log directory listing utilities. This PR cleans that up so there's just one utility method. I also got rid of the listing utility that returns an `Optional>`. If we cannot find the `_delta_log` folder, we throw an exception. If the `_delta_log` folder is empty, we should return an empty list. ## How was this patch tested? Just a refactor. Existing UTs. ## Does this PR introduce _any_ user-facing changes? No. --- .../io/delta/kernel/internal/DeltaErrors.java | 5 + .../kernel/internal/DeltaLogActionUtils.java | 157 +++++++++++----- .../internal/snapshot/SnapshotManager.java | 167 +++--------------- .../delta/kernel/internal/util/FileNames.java | 5 + .../internal/DeltaLogActionUtilsSuite.scala | 89 +++++++++- .../internal/SnapshotManagerSuite.scala | 40 +++-- .../DeltaLogActionUtilsE2ESuite.scala | 66 +++++++ 7 files changed, 314 insertions(+), 215 deletions(-) create mode 100644 kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaLogActionUtilsE2ESuite.scala diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaErrors.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaErrors.java index 292d45a45b7..4c628c540ff 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaErrors.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaErrors.java @@ -33,6 +33,11 @@ public final class DeltaErrors { private DeltaErrors() {} + public static KernelException missingCheckpoint(String tablePath, long checkpointVersion) { + return new InvalidTableException( + tablePath, String.format("Missing checkpoint at version %s", checkpointVersion)); + } + public static KernelException versionBeforeFirstAvailableCommit( String tablePath, long versionToLoad, long earliestVersion) { String message = diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaLogActionUtils.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaLogActionUtils.java index 758b55aa15e..69389a9c806 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaLogActionUtils.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaLogActionUtils.java @@ -17,6 +17,7 @@ import static io.delta.kernel.internal.DeltaErrors.*; import static io.delta.kernel.internal.fs.Path.getName; +import static io.delta.kernel.internal.util.Preconditions.checkArgument; import io.delta.kernel.data.ColumnVector; import io.delta.kernel.data.ColumnarBatch; @@ -31,6 +32,7 @@ import io.delta.kernel.internal.lang.ListUtils; import io.delta.kernel.internal.replay.ActionsIterator; import io.delta.kernel.internal.util.FileNames; +import io.delta.kernel.internal.util.FileNames.DeltaLogFileType; import io.delta.kernel.types.*; import io.delta.kernel.utils.CloseableIterator; import io.delta.kernel.utils.FileStatus; @@ -94,14 +96,20 @@ public enum DeltaAction { */ public static List getCommitFilesForVersionRange( Engine engine, Path tablePath, long startVersion, long endVersion) { - // Validate arguments if (startVersion < 0 || endVersion < startVersion) { throw invalidVersionRange(startVersion, endVersion); } // Get any available commit files within the version range - List commitFiles = listCommitFiles(engine, tablePath, startVersion, endVersion); + final List commitFiles = + listDeltaLogFiles( + engine, + Collections.singleton(DeltaLogFileType.COMMIT), + tablePath, + startVersion, + Optional.of(endVersion), + false /* mustBeRecreatable */); // There are no available commit files within the version range. // This can be due to (1) an empty directory, (2) no valid delta files in the directory, @@ -172,6 +180,106 @@ public static CloseableIterator readCommitFiles( }); } + /** + * Returns the list of files of type $fileTypes in the _delta_log directory of the given + * $tablePath, in increasing order from $startVersion to the optional $endVersion. + * + * @throws TableNotFoundException if the table or its _delta_log does not exist + * @throws KernelException if mustBeRecreatable is true, endVersionOpt is present, and the + * _delta_log history has been truncated so that we cannot load the desired end version + */ + public static List listDeltaLogFiles( + Engine engine, + Set fileTypes, + Path tablePath, + long startVersion, + Optional endVersionOpt, + boolean mustBeRecreatable) { + checkArgument(!fileTypes.isEmpty(), "At least one file type must be provided"); + + endVersionOpt.ifPresent( + endVersion -> { + checkArgument( + endVersion >= startVersion, + "endVersion=%s provided is less than startVersion=%s", + endVersion, + startVersion); + }); + + final Path logPath = new Path(tablePath, "_delta_log"); + + logger.info( + "Listing log files types={} in path={} starting from {} and ending with {}", + fileTypes, + logPath, + startVersion, + endVersionOpt); + + final List output = new ArrayList<>(); + final long startTimeMillis = System.currentTimeMillis(); + + try (CloseableIterator fsIter = listLogDir(engine, tablePath, startVersion)) { + while (fsIter.hasNext()) { + final FileStatus fs = fsIter.next(); + + if (fileTypes.contains(DeltaLogFileType.COMMIT) + && FileNames.isCommitFile(getName(fs.getPath()))) { + // Here, we do nothing (we will consume this file). + } else if (fileTypes.contains(DeltaLogFileType.CHECKPOINT) + && FileNames.isCheckpointFile(getName(fs.getPath())) + && fs.getSize() > 0) { + // Checkpoint files of 0 size are invalid but may be ignored silently when read, hence we + // ignore them so that we never pick up such checkpoints. + // Here, we do nothing (we will consume this file). + } else { + logger.debug("Ignoring file {} as it is not of the desired type", fs.getPath()); + continue; // Here, we continue and skip this file. + } + + final long fileVersion = FileNames.getFileVersion(new Path(fs.getPath())); + + if (fileVersion < startVersion) { + throw new RuntimeException( + String.format( + "Listing files in %s with startVersion %s yet found file %s with version %s", + logPath, startVersion, fs.getPath(), fileVersion)); + } + + if (endVersionOpt.isPresent()) { + final long endVersion = endVersionOpt.get(); + + if (fileVersion > endVersion) { + if (mustBeRecreatable && output.isEmpty()) { + final long earliestVersion = + DeltaHistoryManager.getEarliestRecreatableCommit(engine, logPath); + throw DeltaErrors.versionBeforeFirstAvailableCommit( + tablePath.toString(), endVersion, earliestVersion); + } else { + logger.debug( + "Stopping listing; found file {} with version > {}=endVersion", + fs.getPath(), + endVersion); + break; + } + } + } + + output.add(fs); + } + } catch (IOException e) { + throw new UncheckedIOException("Unable to close resource", e); + } + + logger.info( + "{}: Took {} ms to list the commit files for versions [{}, {}]", + tablePath, + System.currentTimeMillis() - startTimeMillis, + startVersion, + endVersionOpt); + + return output; + } + ////////////////////// // Private helpers // ///////////////////// @@ -247,53 +355,10 @@ private static CloseableIterator listLogDir( "Listing from %s", FileNames.listingPrefix(logPath, startVersion)); } catch (FileNotFoundException e) { + // Did not find the _delta_log directory. throw new TableNotFoundException(tablePath.toString()); } catch (IOException io) { throw new UncheckedIOException("Failed to list the files in delta log", io); } } - - /** - * Returns a list of delta commit files found in the _delta_log directory between startVersion and - * endVersion (both inclusive). - * - * @throws TableNotFoundException if the _delta_log directory does not exist - */ - private static List listCommitFiles( - Engine engine, Path tablePath, long startVersion, long endVersion) { - - // TODO update to support coordinated commits; suggested to load the Snapshot at endVersion - // and get the backfilled/unbackfilled commits from the LogSegment to combine with commit files - // listed from [startVersion, LogSegment.checkpointVersion] - logger.info( - "{}: Listing the commit files for versions [{}, {}]", tablePath, startVersion, endVersion); - long startTimeMillis = System.currentTimeMillis(); - final List output = new ArrayList<>(); - try (CloseableIterator fsIter = listLogDir(engine, tablePath, startVersion)) { - while (fsIter.hasNext()) { - FileStatus fs = fsIter.next(); - if (!FileNames.isCommitFile(getName(fs.getPath()))) { - logger.debug("Ignoring non-commit file {}", fs.getPath()); - continue; - } - if (FileNames.getFileVersion(new Path(fs.getPath())) > endVersion) { - logger.debug( - "Stopping listing found file {} with version > {}=endVersion", - fs.getPath(), - endVersion); - break; - } - output.add(fs); - } - } catch (IOException e) { - throw new UncheckedIOException("Unable to close resource", e); - } - logger.info( - "{}: Took {} ms to list the commit files for versions [{}, {}]", - tablePath, - System.currentTimeMillis() - startTimeMillis, - startVersion, - endVersion); - return output; - } } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java index 427f2a1e989..857568d82e4 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java @@ -21,7 +21,6 @@ import static io.delta.kernel.internal.TableConfig.LOG_RETENTION; import static io.delta.kernel.internal.TableFeatures.validateWriteSupportedTable; import static io.delta.kernel.internal.checkpoints.Checkpointer.findLastCompleteCheckpointBefore; -import static io.delta.kernel.internal.fs.Path.getName; import static io.delta.kernel.internal.replay.LogReplayUtils.assertLogFilesBelongToTable; import static io.delta.kernel.internal.snapshot.MetadataCleanup.cleanupExpiredLogs; import static io.delta.kernel.internal.util.Preconditions.checkArgument; @@ -43,8 +42,8 @@ import io.delta.kernel.internal.replay.LogReplay; import io.delta.kernel.internal.util.Clock; import io.delta.kernel.internal.util.FileNames; +import io.delta.kernel.internal.util.FileNames.DeltaLogFileType; import io.delta.kernel.internal.util.Tuple2; -import io.delta.kernel.utils.CloseableIterator; import io.delta.kernel.utils.FileStatus; import java.io.*; import java.nio.file.FileAlreadyExistsException; @@ -266,117 +265,6 @@ private void registerHint(SnapshotHint newHint) { }); } - /** Get an iterator of files in the _delta_log directory starting with the startVersion. */ - private CloseableIterator listFrom(Engine engine, long startVersion) - throws IOException { - logger.debug("{}: startVersion: {}", tablePath, startVersion); - return wrapEngineExceptionThrowsIO( - () -> engine.getFileSystemClient().listFrom(FileNames.listingPrefix(logPath, startVersion)), - "Listing from %s", - FileNames.listingPrefix(logPath, startVersion)); - } - - /** - * Returns true if the given file name is delta log files. Delta log files can be delta commit - * file (e.g., 000000000.json), or checkpoint file. (e.g., - * 000000001.checkpoint.00001.00003.parquet) - * - * @param fileName Name of the file (not the full path) - * @return Boolean Whether the file is delta log files - */ - private boolean isDeltaCommitOrCheckpointFile(String fileName) { - return FileNames.isCheckpointFile(fileName) || FileNames.isCommitFile(fileName); - } - - /** - * Returns an iterator containing a list of files found in the _delta_log directory starting with - * the startVersion. Returns None if no files are found or the directory is missing. - */ - private Optional> listFromOrNone(Engine engine, long startVersion) { - // LIST the directory, starting from the provided lower bound (treat missing dir as empty). - // NOTE: "empty/missing" is _NOT_ equivalent to "contains no useful commit files." - try { - CloseableIterator results = listFrom(engine, startVersion); - if (results.hasNext()) { - return Optional.of(results); - } else { - return Optional.empty(); - } - } catch (FileNotFoundException e) { - return Optional.empty(); - } catch (IOException io) { - throw new UncheckedIOException("Failed to list the files in delta log", io); - } - } - - /** - * Returns the delta files and checkpoint files starting from the given `startVersion`. - * `versionToLoad` is an optional parameter to set the max bound. It's usually used to load a - * table snapshot for a specific version. If no delta or checkpoint files exist below the - * versionToLoad and at least one delta file exists, throws an exception that the state is not - * reconstructable. - * - * @param startVersion the version to start. Inclusive. - * @param versionToLoad the optional parameter to set the max version we should return. Inclusive. - * Must be >= startVersion if provided. - * @return Some array of files found (possibly empty, if no usable commit files are present), or - * None if the listing returned no files at all. - */ - protected final Optional> listDeltaAndCheckpointFiles( - Engine engine, long startVersion, Optional versionToLoad) { - versionToLoad.ifPresent( - v -> - checkArgument( - v >= startVersion, - "versionToLoad=%s provided is less than startVersion=%s", - v, - startVersion)); - logger.debug("startVersion: {}, versionToLoad: {}", startVersion, versionToLoad); - - return listFromOrNone(engine, startVersion) - .map( - fileStatusesIter -> { - final List output = new ArrayList<>(); - - while (fileStatusesIter.hasNext()) { - final FileStatus fileStatus = fileStatusesIter.next(); - final String fileName = getName(fileStatus.getPath()); - - // Pick up all checkpoint and delta files - if (!isDeltaCommitOrCheckpointFile(fileName)) { - continue; - } - - // Checkpoint files of 0 size are invalid but may be ignored silently when read, - // hence we drop them so that we never pick up such checkpoints. - if (FileNames.isCheckpointFile(fileName) && fileStatus.getSize() == 0) { - continue; - } - // Take files until the version we want to load - final boolean versionWithinRange = - versionToLoad - .map(v -> FileNames.getFileVersion(new Path(fileStatus.getPath())) <= v) - .orElse(true); - - if (!versionWithinRange) { - // If we haven't taken any files yet and the first file we see is greater - // than the versionToLoad then the versionToLoad is not reconstructable - // from the existing logs - if (output.isEmpty()) { - long earliestVersion = - DeltaHistoryManager.getEarliestRecreatableCommit(engine, logPath); - throw DeltaErrors.versionBeforeFirstAvailableCommit( - tablePath.toString(), versionToLoad.get(), earliestVersion); - } - break; - } - output.add(fileStatus); - } - - return output; - }); - } - /** * Load the Snapshot for this Delta table at initialization. This method uses the `lastCheckpoint` * file as a hint on where to start listing the transaction log directory. @@ -508,8 +396,15 @@ public Optional getLogSegmentForVersion( }); long startTimeMillis = System.currentTimeMillis(); - final Optional> newFiles = - listDeltaAndCheckpointFiles(engine, startVersion, versionToLoad); + final List newFiles = + DeltaLogActionUtils.listDeltaLogFiles( + engine, + new HashSet<>(Arrays.asList(DeltaLogFileType.COMMIT, DeltaLogFileType.CHECKPOINT)), + tablePath, + startVersion, + versionToLoad, + true /* mustBeRecreatable */); + logger.info( "{}: Took {}ms to list the files after starting checkpoint", tablePath, @@ -534,26 +429,22 @@ protected Optional getLogSegmentForVersion( Engine engine, Optional startCheckpointOpt, Optional versionToLoadOpt, - Optional> filesOpt) { - final List newFiles; - if (filesOpt.isPresent()) { - newFiles = filesOpt.get(); - } else { - // No files found even when listing from 0 => empty directory => - // table does not exist yet. - if (!startCheckpointOpt.isPresent()) { - return Optional.empty(); + List newFiles) { + if (newFiles.isEmpty()) { + if (startCheckpointOpt.isPresent()) { + // We either (a) determined this checkpoint version from the _LAST_CHECKPOINT file, or (b) + // found the last complete checkpoint before our versionToLoad. In either case, we didn't + // see the checkpoint file in the listing. + // TODO: for case (a), re-load the delta log but ignore the _LAST_CHECKPOINT file. + throw DeltaErrors.missingCheckpoint(tablePath.toString(), startCheckpointOpt.get()); + } else { + // Either no files found OR no *delta* files found even when listing from 0. This means that + // the delta table does not exist yet. + throw new TableNotFoundException( + tablePath.toString(), format("No delta files found in the directory: %s", logPath)); } - - // FIXME: We always write the commit and checkpoint files before updating - // _last_checkpoint. If the listing came up empty, then we either encountered a - // list-after-put inconsistency in the underlying log store, or somebody corrupted the - // table by deleting files. Either way, we can't safely continue. - // - // For now, we preserve existing behavior by returning Array.empty, which will trigger a - // recursive call to [[getLogSegmentForVersion]] below (same as before the refactor). - newFiles = Collections.emptyList(); } + logDebug( () -> format( @@ -561,16 +452,6 @@ protected Optional getLogSegmentForVersion( Arrays.toString( newFiles.stream().map(x -> new Path(x.getPath()).getName()).toArray()))); - if (newFiles.isEmpty() && !startCheckpointOpt.isPresent()) { - // We can't construct a snapshot because the directory contained no usable commit - // files... but we can't return Optional.empty either, because it was not truly empty. - throw new RuntimeException(format("No delta files found in the directory: %s", logPath)); - } else if (newFiles.isEmpty()) { - // The directory may be deleted and recreated and we may have stale state in our - // DeltaLog singleton, so try listing from the first version - return getLogSegmentForVersion(engine, Optional.empty(), versionToLoadOpt); - } - Tuple2, List> checkpointsAndDeltas = ListUtils.partition( newFiles, diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/FileNames.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/FileNames.java index f0077f840c1..bb121a94f79 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/FileNames.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/FileNames.java @@ -29,6 +29,11 @@ private FileNames() {} // File name patterns and other static values // //////////////////////////////////////////////// + public enum DeltaLogFileType { + COMMIT, + CHECKPOINT + } + /** Example: 00000000000000000001.json */ private static final Pattern DELTA_FILE_PATTERN = Pattern.compile("\\d+\\.json"); diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/DeltaLogActionUtilsSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/DeltaLogActionUtilsSuite.scala index 3846fd034d3..938e025d30c 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/DeltaLogActionUtilsSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/DeltaLogActionUtilsSuite.scala @@ -16,6 +16,7 @@ package io.delta.kernel.internal import java.io.FileNotFoundException +import java.util.{Collections, Optional} import scala.collection.JavaConverters._ import scala.reflect.ClassTag @@ -23,15 +24,17 @@ import scala.reflect.ClassTag import io.delta.kernel.exceptions.{InvalidTableException, KernelException, TableNotFoundException} import io.delta.kernel.internal.util.FileNames import io.delta.kernel.utils.FileStatus -import org.scalatest.funsuite.AnyFunSuite -import io.delta.kernel.internal.DeltaLogActionUtils.{getCommitFilesForVersionRange, verifyDeltaVersions} +import io.delta.kernel.internal.DeltaLogActionUtils.{getCommitFilesForVersionRange, listDeltaLogFiles, verifyDeltaVersions} +import io.delta.kernel.internal.fs.Path import io.delta.kernel.test.MockFileSystemClientUtils +import org.scalatest.funsuite.AnyFunSuite + class DeltaLogActionUtilsSuite extends AnyFunSuite with MockFileSystemClientUtils { - ////////////////////////////////////////////////////////////////////////////////// - // verifyDeltaVersions tests - ////////////////////////////////////////////////////////////////////////////////// + /////////////////////////////// + // verifyDeltaVersions tests // + /////////////////////////////// def getCommitFiles(versions: Seq[Long]): java.util.List[FileStatus] = { versions @@ -108,9 +111,9 @@ class DeltaLogActionUtilsSuite extends AnyFunSuite with MockFileSystemClientUtil } } - ////////////////////////////////////////////////////////////////////////////////// - // getCommitFilesForVersionRange tests - ////////////////////////////////////////////////////////////////////////////////// + ///////////////////////////////////////// + // getCommitFilesForVersionRange tests // + ///////////////////////////////////////// test("getCommitFilesForVersionRange: directory does not exist") { intercept[TableNotFoundException] { @@ -240,4 +243,74 @@ class DeltaLogActionUtilsSuite extends AnyFunSuite with MockFileSystemClientUtil endVersion = 0, expectedCommitFiles = deltaFileStatuses(Seq(0)) ) + + ///////////////////////////// + // listDeltaLogFiles tests // + ///////////////////////////// + + private val checkpointsAndDeltas = singularCheckpointFileStatuses(Seq(10)) ++ + deltaFileStatuses(Seq(10, 11, 12, 13, 14)) ++ + Seq(FileStatus.of(s"$logPath/00000000000000000014.crc", 0, 0)) ++ + multiCheckpointFileStatuses(Seq(14), 2) ++ + deltaFileStatuses(Seq(15, 16, 17)) ++ + v2CheckpointFileStatuses(Seq((17, false, 2)), "json").map(_._1) + + private def extractVersions(files: Seq[FileStatus]): Seq[Long] = { + files.map(fs => FileNames.getFileVersion(new Path(fs.getPath))) + } + + test("listDeltaLogFiles: no fileTypes provided") { + intercept[IllegalArgumentException] { + listDeltaLogFiles( + createMockFSListFromEngine(deltaFileStatuses(Seq(1, 2, 3))), + Collections.emptySet(), // No fileTypes provided! + dataPath, + 1, + Optional.empty(), + false /* mustBeRecreatable */ + ) + } + } + + test("listDeltaLogFiles: returns requested file type only") { + val commitFiles = listDeltaLogFiles( + createMockFSListFromEngine(checkpointsAndDeltas), + Set(FileNames.DeltaLogFileType.COMMIT).asJava, + dataPath, + 10, + Optional.empty(), + false /* mustBeRecreatable */ + ).asScala + + assert(commitFiles.forall(fs => FileNames.isCommitFile(fs.getPath))) + assert(extractVersions(commitFiles) == Seq(10, 11, 12, 13, 14, 15, 16, 17)) + + val checkpointFiles = listDeltaLogFiles( + createMockFSListFromEngine(checkpointsAndDeltas), + Set(FileNames.DeltaLogFileType.CHECKPOINT).asJava, + dataPath, + 10, + Optional.empty(), + false /* mustBeRecreatable */ + ).asScala + + assert(checkpointFiles.forall(fs => FileNames.isCheckpointFile(fs.getPath))) + assert(extractVersions(checkpointFiles) == Seq(10, 14, 14, 17)) + } + + test("listDeltaLogFiles: mustBeRecreatable") { + val exMsg = intercept[KernelException] { + listDeltaLogFiles( + createMockFSListFromEngine(checkpointsAndDeltas), + Set(FileNames.DeltaLogFileType.COMMIT, FileNames.DeltaLogFileType.CHECKPOINT).asJava, + dataPath, + 0, + Optional.of(4), + true /* mustBeRecreatable */ + ) + }.getMessage + assert(exMsg.contains("Cannot load table version 4 as the transaction log has been " + + "truncated due to manual deletion or the log/checkpoint retention policy. The earliest " + + "available version is 10")) + } } diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/SnapshotManagerSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/SnapshotManagerSuite.scala index d74a2a9a2df..c9a05385822 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/SnapshotManagerSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/SnapshotManagerSuite.scala @@ -20,8 +20,8 @@ import java.util.{Arrays, Collections, Optional} import scala.collection.JavaConverters._ import scala.reflect.ClassTag -import io.delta.kernel.data.{ColumnarBatch, ColumnVector} -import io.delta.kernel.exceptions.InvalidTableException +import io.delta.kernel.data.{ColumnVector, ColumnarBatch} +import io.delta.kernel.exceptions.{InvalidTableException, TableNotFoundException} import io.delta.kernel.expressions.Predicate import io.delta.kernel.internal.checkpoints.{CheckpointInstance, SidecarFile} import io.delta.kernel.internal.fs.Path @@ -436,25 +436,27 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { } test("getLogSegmentForVersion: empty delta log") { - // listDeltaAndCheckpointFiles = Optional.empty() - val logSegmentOpt = snapshotManager.getLogSegmentForVersion( - createMockFSListFromEngine(Seq.empty), - Optional.empty(), - Optional.empty() - ) - assert(!logSegmentOpt.isPresent()) + val exMsg = intercept[TableNotFoundException] { + snapshotManager.getLogSegmentForVersion( + createMockFSListFromEngine(Seq.empty), + Optional.empty(), + Optional.empty() + ) + }.getMessage + + assert(exMsg.contains("No delta files found in the directory")) } test("getLogSegmentForVersion: no delta files in the delta log") { // listDeltaAndCheckpointFiles = Optional.of(EmptyList) val files = Seq("foo", "notdelta.parquet", "foo.json", "001.checkpoint.00f.oo0.parquet") .map(FileStatus.of(_, 10, 10)) - testExpectedError[RuntimeException]( + testExpectedError[TableNotFoundException]( files, expectedErrorMessageContains = "No delta files found in the directory: /fake/path/to/table/_delta_log" ) - testExpectedError[RuntimeException]( + testExpectedError[TableNotFoundException]( files, versionToLoad = Optional.of(5), expectedErrorMessageContains = @@ -848,13 +850,15 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { } test("getLogSegmentForVersion: corrupt _last_checkpoint with empty delta log") { - // listDeltaAndCheckpointFiles = Optional.empty() - val logSegmentOpt = snapshotManager.getLogSegmentForVersion( - createMockFSListFromEngine(Seq.empty), - Optional.of(1), - Optional.empty() - ) - assert(!logSegmentOpt.isPresent()) + val exMsg = intercept[InvalidTableException] { + snapshotManager.getLogSegmentForVersion( + createMockFSListFromEngine(Seq.empty), + Optional.of(1), + Optional.empty() + ) + }.getMessage + + assert(exMsg.contains("Missing checkpoint at version 1")) } } diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaLogActionUtilsE2ESuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaLogActionUtilsE2ESuite.scala new file mode 100644 index 00000000000..84f9aebb3bc --- /dev/null +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaLogActionUtilsE2ESuite.scala @@ -0,0 +1,66 @@ +/* + * Copyright (2025) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.kernel.defaults + +import java.io.File +import java.util.Optional + +import scala.collection.JavaConverters._ + +import io.delta.kernel.defaults.utils.TestUtils +import io.delta.kernel.exceptions.TableNotFoundException +import io.delta.kernel.internal.DeltaLogActionUtils.listDeltaLogFiles +import io.delta.kernel.internal.fs.Path +import io.delta.kernel.internal.util.FileNames + +import org.scalatest.funsuite.AnyFunSuite + +/** Test suite for end-to-end cases. See also the mocked unit tests in DeltaLogActionUtilsSuite. */ +class DeltaLogActionUtilsE2ESuite extends AnyFunSuite with TestUtils { + test("listDeltaLogFiles: throws TableNotFoundException if _delta_log does not exist") { + withTempDir { tableDir => + intercept[TableNotFoundException] { + listDeltaLogFiles( + defaultEngine, + Set(FileNames.DeltaLogFileType.COMMIT, FileNames.DeltaLogFileType.CHECKPOINT).asJava, + new Path(tableDir.getAbsolutePath), + 0, + Optional.empty(), + true /* mustBeRecreatable */ + ) + } + } + } + + test("listDeltaLogFiles: returns empty list if _delta_log is empty") { + withTempDir { tableDir => + val logDir = new File(tableDir, "_delta_log") + assert(logDir.mkdirs() && logDir.isDirectory && logDir.listFiles().isEmpty) + + val result = listDeltaLogFiles( + defaultEngine, + Set(FileNames.DeltaLogFileType.COMMIT, FileNames.DeltaLogFileType.CHECKPOINT).asJava, + new Path(tableDir.getAbsolutePath), + 0, + Optional.empty(), + true /* mustBeRecreatable */ + ) + + assert(result.isEmpty) + } + } +} From ba8246e2a82efe6810aee94dfe578e0dfa082ed6 Mon Sep 17 00:00:00 2001 From: Chirag Singh <137233133+chirag-s-db@users.noreply.github.com> Date: Fri, 24 Jan 2025 10:16:34 -0800 Subject: [PATCH 14/28] [Spark] Minor test fix for partition-like data skipping (#4088) #### Which Delta project/connector is this regarding? - [x] Spark - [ ] Standalone - [ ] Flink - [ ] Kernel - [ ] Other (fill in here) ## Description Currently, we look at a query's optimized plan's expressions to get query's predicates (which we use to test data skipping). This isn't correct - it includes the projected columns in addition to the actual filters, which means that we might have incorrect validation (it might be too selective). Replace this by replacing with the split conjunctive predicates from the Filter of the query plan. ## How was this patch tested? Test-only change. ## Does this PR introduce _any_ user-facing changes? No --- .../sql/delta/stats/PartitionLikeDataSkippingSuite.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/stats/PartitionLikeDataSkippingSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/stats/PartitionLikeDataSkippingSuite.scala index 531f4fad93f..4b2bfa33688 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/stats/PartitionLikeDataSkippingSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/stats/PartitionLikeDataSkippingSuite.scala @@ -27,6 +27,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.SparkConf import org.apache.spark.sql.QueryTest import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.plans.logical.Filter import org.apache.spark.sql.functions.{array, col, concat, lit, struct} import org.apache.spark.sql.test.SharedSparkSession @@ -72,8 +73,9 @@ trait PartitionLikeDataSkippingSuiteBase val res = sql(query).collect() assert(res.sameElements(baseResult)) - val predicates = - sql(query).queryExecution.optimizedPlan.expressions.flatMap(splitConjunctivePredicates) + val predicates = sql(query).queryExecution.optimizedPlan.collect { + case Filter(condition, _) => condition + }.flatMap(splitConjunctivePredicates) val scanResult = DeltaLog.forTable(spark, TableIdentifier(tableName)) .update().filesForScan(predicates) assert(scanResult.files.length == expectedNumFiles) From 107b2d3c3279093ab6481503c399b828579a031a Mon Sep 17 00:00:00 2001 From: Venki Korukanti Date: Fri, 24 Jan 2025 13:21:00 -0800 Subject: [PATCH 15/28] [Build] Changes to avoid making too many requests to central maven repo (#4087) ## Description * Google maven repo URL is changed from `https://maven-central.storage-download.googleapis.com/repos/central/data/` to `https://maven-central.storage-download.googleapis.com/maven2/`. This caused more load on the central maven repo as the google maven repo is wrong. * Currently we hard coded the URL to central maven repo for downloading the `sbt-launch-x.x.x.jar` as part of the build start. Change it to google maven repo which seems have better rate limits than the central maven repo. Also add the central maven repo as a backup so that we have two options. Example retrying: ``` (base) delta2 % ./build/sbt clean package test:package Attempting to fetch sbt from https://eemaven-central.storage-download.googleapis.com/maven2/org/scala-sbt/sbt-launch/1.9.9/sbt-launch-1.9.9.jar Download from https://eemaven-central.storage-download.googleapis.com/maven2/org/scala-sbt/sbt-launch/1.9.9/sbt-launch-1.9.9.jar failed. Retrying from https://repo1.maven.org/maven2/org/scala-sbt/sbt-launch/1.9.9/sbt-launch-1.9.9.jar Launching sbt from build/sbt-launch-1.9.9.jar ``` * Add Google repo to iceberg build so that it is attempted first before the central maven repo and also as additional backup. * Testing * Without any repo changes: ``` (base) iceberg_src % rm -rf $HOME/.gradle/caches/ (base) iceberg_src % ./gradlew clean --refresh-dependencies :iceberg-core:build --info | grep Downloading | grep "mockito-core-4.0.0.jar" Downloading https://repo.maven.apache.org/maven2/org/mockito/mockito-core/4.0.0/mockito-core-4.0.0.jar to /Users/venkateshwar.korukanti/.gradle/.tmp/gradle_download6315884371983156241bin ``` * By adding google repo as a first in the list ``` (base) iceberg_src % git diff diff --git a/build.gradle b/build.gradle index 12ed701..c80bbd3 100644 --- a/build.gradle +++ b/build.gradle @@ -97,6 +97,9 @@ allprojects { group = "org.apache.iceberg" version = projectVersion repositories { + maven { + url "https://maven-central.storage-download.googleapis.com/maven2" + } mavenCentral() mavenLocal() } (base) iceberg_src % rm -rf $HOME/.gradle/caches/ (base) iceberg_src % ./gradlew clean --refresh-dependencies :iceberg-core:build --info | grep Downloading | grep "mockito-core-4.0.0.jar" Downloading https://maven-central.storage-download.googleapis.com/maven2/org/mockito/mockito-core/4.0.0/mockito-core-4.0.0.jar to /Users/venkateshwar.korukanti/.gradle/.tmp/gradle_download7065461132604582238bin ``` --- build/sbt-config/repositories | 2 +- build/sbt-launch-lib.bash | 50 +++++++++++++++----------- icebergShaded/generate_iceberg_jars.py | 24 +++++++++++++ 3 files changed, 55 insertions(+), 21 deletions(-) diff --git a/build/sbt-config/repositories b/build/sbt-config/repositories index b12fece5fd5..f5fcaa71859 100644 --- a/build/sbt-config/repositories +++ b/build/sbt-config/repositories @@ -2,7 +2,7 @@ local local-preloaded-ivy: file:///${sbt.preloaded-${sbt.global.base-${user.home}/.sbt}/preloaded/}, [organization]/[module]/[revision]/[type]s/[artifact](-[classifier]).[ext] local-preloaded: file:///${sbt.preloaded-${sbt.global.base-${user.home}/.sbt}/preloaded/} - gcs-maven-central-mirror: https://maven-central.storage-download.googleapis.com/repos/central/data/ + gcs-maven-central-mirror: https://maven-central.storage-download.googleapis.com/maven2/ maven-central typesafe-ivy-releases: https://repo.typesafe.com/typesafe/ivy-releases/, [organization]/[module]/[revision]/[type]s/[artifact](-[classifier]).[ext], bootOnly sbt-ivy-snapshots: https://repo.scala-sbt.org/scalasbt/ivy-snapshots/, [organization]/[module]/[revision]/[type]s/[artifact](-[classifier]).[ext], bootOnly diff --git a/build/sbt-launch-lib.bash b/build/sbt-launch-lib.bash index 0d58bb7269c..f2f501e0a7d 100755 --- a/build/sbt-launch-lib.bash +++ b/build/sbt-launch-lib.bash @@ -36,42 +36,52 @@ dlog () { [[ $debug ]] && echoerr "$@" } +download_sbt () { + local url=$1 + local output=$2 + local temp_file="${output}.part" + + if [ $(command -v curl) ]; then + curl --fail --location --silent ${url} > "${temp_file}" &&\ + mv "${temp_file}" "${output}" + elif [ $(command -v wget) ]; then + wget --quiet ${url} -O "${temp_file}" &&\ + mv "${temp_file}" "${output}" + else + printf "You do not have curl or wget installed, unable to downlaod ${url}\n" + exit -1 + fi +} + + acquire_sbt_jar () { SBT_VERSION=`awk -F "=" '/sbt\.version/ {print $2}' ./project/build.properties` - # Download sbt from mirror URL if the environment variable is provided + # Set primary and fallback URLs if [[ "${SBT_VERSION}" == "0.13.18" ]] && [[ -n "${SBT_MIRROR_JAR_URL}" ]]; then URL1="${SBT_MIRROR_JAR_URL}" elif [[ "${SBT_VERSION}" == "1.5.5" ]] && [[ -n "${SBT_1_5_5_MIRROR_JAR_URL}" ]]; then URL1="${SBT_1_5_5_MIRROR_JAR_URL}" else - URL1=${DEFAULT_ARTIFACT_REPOSITORY:-https://repo1.maven.org/maven2/}org/scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch-${SBT_VERSION}.jar + URL1=${DEFAULT_ARTIFACT_REPOSITORY:-https://maven-central.storage-download.googleapis.com/maven2/}org/scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch-${SBT_VERSION}.jar fi + BACKUP_URL="https://repo1.maven.org/maven2/org/scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch-${SBT_VERSION}.jar" JAR=build/sbt-launch-${SBT_VERSION}.jar sbt_jar=$JAR if [[ ! -f "$sbt_jar" ]]; then - # Download sbt launch jar if it hasn't been downloaded yet - if [ ! -f "${JAR}" ]; then - # Download printf 'Attempting to fetch sbt from %s\n' "${URL1}" - JAR_DL="${JAR}.part" - if [ $(command -v curl) ]; then - curl --fail --location --silent ${URL1} > "${JAR_DL}" &&\ - mv "${JAR_DL}" "${JAR}" - elif [ $(command -v wget) ]; then - wget --quiet ${URL1} -O "${JAR_DL}" &&\ - mv "${JAR_DL}" "${JAR}" - else - printf "You do not have curl or wget installed, please install sbt manually from https://www.scala-sbt.org/\n" - exit -1 - fi + download_sbt "${URL1}" "${JAR}" + + if [[ ! -f "${JAR}" ]]; then + printf 'Download from %s failed. Retrying from %s\n' "${URL1}" "${BACKUP_URL}" + download_sbt "${BACKUP_URL}" "${JAR}" fi - if [ ! -f "${JAR}" ]; then - # We failed to download - printf "Our attempt to download sbt locally to ${JAR} failed. Please install sbt manually from https://www.scala-sbt.org/\n" - exit -1 + + if [[ ! -f "${JAR}" ]]; then + printf "Failed to download sbt. Please install sbt manually from https://www.scala-sbt.org/\n" + exit 1 fi printf "Launching sbt from ${JAR}\n" fi diff --git a/icebergShaded/generate_iceberg_jars.py b/icebergShaded/generate_iceberg_jars.py index 0c2579eb4e3..2e9b37da487 100644 --- a/icebergShaded/generate_iceberg_jars.py +++ b/icebergShaded/generate_iceberg_jars.py @@ -19,6 +19,7 @@ import argparse import os import glob +import re import subprocess import shlex import shutil @@ -65,6 +66,27 @@ def iceberg_jars_exists(): return True +def add_google_maven_repo_to_gradle_config(): + with WorkingDirectory(iceberg_src_dir): + file_path = 'build.gradle' + + with open(file_path, 'r') as file: + content = file.read() + + # Define the old and new configurations + old_config = r'repositories {\n' + + new_config = 'repositories {\n maven {\n ' + \ + 'url "https://maven-central.storage-download.googleapis.com/maven2"\n }\n' + + # Replace the old configuration with the new one + updated_content = re.sub(old_config, new_config, content, flags=re.DOTALL) + + # Write the updated content back to the file + with open(file_path, 'w') as file: + file.write(updated_content) + + def prepare_iceberg_source(): with WorkingDirectory(iceberg_root_dir): print(">>> Cloning Iceberg repo") @@ -92,6 +114,8 @@ def prepare_iceberg_source(): run_cmd("git add .") run_cmd("git commit -a -m 'applied %s'" % path.basename(patch_file)) + add_google_maven_repo_to_gradle_config() + def generate_iceberg_jars(): print(">>> Compiling JARs") From 07691fac8e3cd00b55441f7f6e34de15b05d5ac6 Mon Sep 17 00:00:00 2001 From: Sumeet Varma Date: Fri, 24 Jan 2025 15:31:26 -0800 Subject: [PATCH 16/28] [Spark] Remove protocol and metadata from incremental checksum computation function (#4036) #### Which Delta project/connector is this regarding? - [x] Spark - [ ] Standalone - [ ] Flink - [ ] Kernel - [ ] Other (fill in here) ## Description This is a small refactoring PR that removes protocol and metadata from incremental checksum computation method since they can be easily fetched from the actions. ## How was this patch tested? Existing UTs ## Does this PR introduce _any_ user-facing changes? No --- .../org/apache/spark/sql/delta/Checksum.scala | 25 ++++++++++++++++--- .../sql/delta/OptimisticTransaction.scala | 4 +-- .../org/apache/spark/sql/delta/Snapshot.scala | 4 +-- 3 files changed, 25 insertions(+), 8 deletions(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/Checksum.scala b/spark/src/main/scala/org/apache/spark/sql/delta/Checksum.scala index 786494d62cd..02fde9acbc9 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/Checksum.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/Checksum.scala @@ -139,8 +139,8 @@ trait RecordChecksum extends DeltaLogging { * @param deltaLog The DeltaLog * @param versionToCompute The version for which we want to compute the checksum * @param actions The actions corresponding to the version `versionToCompute` - * @param metadata The metadata corresponding to the version `versionToCompute` - * @param protocol The protocol corresponding to the version `versionToCompute` + * @param metadataOpt The metadata corresponding to the version `versionToCompute` (if known) + * @param protocolOpt The protocol corresponding to the version `versionToCompute` (if known) * @param operationName The operation name corresponding to the version `versionToCompute` * @param txnIdOpt The transaction identifier for the version `versionToCompute` * @param previousVersionState Contains either the versionChecksum corresponding to @@ -156,8 +156,8 @@ trait RecordChecksum extends DeltaLogging { deltaLog: DeltaLog, versionToCompute: Long, actions: Seq[Action], - metadata: Metadata, - protocol: Protocol, + metadataOpt: Option[Metadata], + protocolOpt: Option[Protocol], operationName: String, txnIdOpt: Option[String], previousVersionState: Either[Snapshot, VersionChecksum], @@ -215,6 +215,23 @@ trait RecordChecksum extends DeltaLogging { RecordChecksum.operationNamesWhereAddFilesIgnoredForIncrementalCrc.contains(operationName) val ignoreRemoveFilesInOperation = RecordChecksum.operationNamesWhereRemoveFilesIgnoredForIncrementalCrc.contains(operationName) + // Retrieve protocol/metadata in order of precedence: + // 1. Use provided protocol/metadata if available + // 2. Look for a protocol/metadata action in the incremental set of actions to be applied + // 3. Use protocol/metadata from previous version's checksum + // 4. Return PROTOCOL_MISSING/METADATA_MISSING error if all attempts fail + val protocol = protocolOpt + .orElse(actions.collectFirst { case p: Protocol => p }) + .orElse(Option(oldVersionChecksum.protocol)) + .getOrElse { + return Left("PROTOCOL_MISSING") + } + val metadata = metadataOpt + .orElse(actions.collectFirst { case m: Metadata => m }) + .orElse(Option(oldVersionChecksum.metadata)) + .getOrElse { + return Left("METADATA_MISSING") + } val persistentDVsOnTableReadable = DeletionVectorUtils.deletionVectorsReadable(protocol, metadata) val persistentDVsOnTableWritable = diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala b/spark/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala index a5339127b70..f7fccc37f0e 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala @@ -2554,8 +2554,8 @@ trait OptimisticTransactionImpl extends TransactionalWrite deltaLog, attemptVersion, actions = currentTransactionInfo.finalActionsToCommit, - metadata = currentTransactionInfo.metadata, - protocol = currentTransactionInfo.protocol, + metadataOpt = Some(currentTransactionInfo.metadata), + protocolOpt = Some(currentTransactionInfo.protocol), operationName = currentTransactionInfo.op.name, txnIdOpt = Some(currentTransactionInfo.txnId), previousVersionState = scala.Left(snapshot), diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/Snapshot.scala b/spark/src/main/scala/org/apache/spark/sql/delta/Snapshot.scala index 930e1eed499..3a1bb5f3f19 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/Snapshot.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/Snapshot.scala @@ -384,7 +384,7 @@ class Snapshot( * checksum file. If the checksum file is not present or if the protocol or metadata is missing * this will return None. */ - protected def getProtocolMetadataAndIctFromCrc(): + protected def getProtocolMetadataAndIctFromCrc(checksumOpt: Option[VersionChecksum]): Option[Array[ReconstructedProtocolMetadataAndICT]] = { if (!spark.sessionState.conf.getConf( DeltaSQLConf.USE_PROTOCOL_AND_METADATA_FROM_CHECKSUM_ENABLED)) { @@ -431,7 +431,7 @@ class Snapshot( Array[ReconstructedProtocolMetadataAndICT] = { import implicits._ - getProtocolMetadataAndIctFromCrc().foreach { protocolMetadataAndIctFromCrc => + getProtocolMetadataAndIctFromCrc(checksumOpt).foreach { protocolMetadataAndIctFromCrc => return protocolMetadataAndIctFromCrc } From 57c7c87588ed39820322d1506fdcd3c2ad323a0f Mon Sep 17 00:00:00 2001 From: Scott Sandre Date: Fri, 24 Jan 2025 18:37:03 -0800 Subject: [PATCH 17/28] [Kernel] Minor SnapshotManager refactor followup (#4089) #### Which Delta project/connector is this regarding? - [ ] Spark - [ ] Standalone - [ ] Flink - [X] Kernel - [ ] Other (fill in here) ## Description Followup to #4035. Our SnapshotManager LogSegment construction logic is ... a bit messy, to say the least. This PR makes `getLogSegmentForVersion` _not_ return an Optional LogSegment. I also start documenting some of the key steps in constructing a LogSegment. More steps, refactors, and comments will come in future PRs. ## How was this patch tested? Mainly just a refactor. Existing UTs. ## Does this PR introduce _any_ user-facing changes? No. --- .../internal/snapshot/SnapshotManager.java | 210 ++++++++++-------- .../internal/SnapshotManagerSuite.scala | 44 ++-- 2 files changed, 134 insertions(+), 120 deletions(-) diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java index 857568d82e4..3aaa4d2dd71 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java @@ -20,7 +20,6 @@ import static io.delta.kernel.internal.TableConfig.EXPIRED_LOG_CLEANUP_ENABLED; import static io.delta.kernel.internal.TableConfig.LOG_RETENTION; import static io.delta.kernel.internal.TableFeatures.validateWriteSupportedTable; -import static io.delta.kernel.internal.checkpoints.Checkpointer.findLastCompleteCheckpointBefore; import static io.delta.kernel.internal.replay.LogReplayUtils.assertLogFilesBelongToTable; import static io.delta.kernel.internal.snapshot.MetadataCleanup.cleanupExpiredLogs; import static io.delta.kernel.internal.util.Preconditions.checkArgument; @@ -60,7 +59,7 @@ public class SnapshotManager { * The latest {@link SnapshotHint} for this table. The initial value inside the AtomicReference is * `null`. */ - private AtomicReference latestSnapshotHint; + private final AtomicReference latestSnapshotHint; private final Path logPath; private final Path tablePath; @@ -81,12 +80,18 @@ public SnapshotManager(Path logPath, Path tablePath) { * Construct the latest snapshot for given table. * * @param engine Instance of {@link Engine} to use. - * @return - * @throws TableNotFoundException + * @return the latest {@link Snapshot} of the table + * @throws TableNotFoundException if the table does not exist + * @throws InvalidTableException if the table is in an invalid state */ public Snapshot buildLatestSnapshot(Engine engine, SnapshotQueryContext snapshotContext) throws TableNotFoundException { - return getSnapshotAtInit(engine, snapshotContext); + final LogSegment logSegment = + getLogSegmentForVersion(engine, Optional.empty() /* versionToLoad */); + + snapshotContext.setVersion(logSegment.version); + + return createSnapshot(logSegment, engine, snapshotContext); } /** @@ -95,20 +100,15 @@ public Snapshot buildLatestSnapshot(Engine engine, SnapshotQueryContext snapshot * @param engine Instance of {@link Engine} to use. * @param version The snapshot version to construct * @return a {@link Snapshot} of the table at version {@code version} - * @throws TableNotFoundException + * @throws TableNotFoundException if the table does not exist + * @throws InvalidTableException if the table is in an invalid state */ public Snapshot getSnapshotAt(Engine engine, long version, SnapshotQueryContext snapshotContext) throws TableNotFoundException { + final LogSegment logSegment = + getLogSegmentForVersion(engine, Optional.of(version) /* versionToLoadOpt */); - Optional logSegmentOpt = - getLogSegmentForVersion( - engine, - Optional.empty(), /* startCheckpointOpt */ - Optional.of(version) /* versionToLoadOpt */); - - return logSegmentOpt - .map(logSegment -> createSnapshot(logSegment, engine, snapshotContext)) - .orElseThrow(() -> new TableNotFoundException(tablePath.toString())); + return createSnapshot(logSegment, engine, snapshotContext); } /** @@ -265,30 +265,6 @@ private void registerHint(SnapshotHint newHint) { }); } - /** - * Load the Snapshot for this Delta table at initialization. This method uses the `lastCheckpoint` - * file as a hint on where to start listing the transaction log directory. - */ - private SnapshotImpl getSnapshotAtInit(Engine engine, SnapshotQueryContext snapshotContext) - throws TableNotFoundException { - Checkpointer checkpointer = new Checkpointer(logPath); - Optional lastCheckpointOpt = checkpointer.readLastCheckpointFile(engine); - if (!lastCheckpointOpt.isPresent()) { - logger.warn( - "{}: Last checkpoint file is missing or corrupted. " - + "Will search for the checkpoint files directly.", - tablePath); - } - Optional logSegmentOpt = getLogSegmentFrom(engine, lastCheckpointOpt); - // We update the query context version with the resolved version from the log segment listing - // if it exists - logSegmentOpt.ifPresent(logSegment -> snapshotContext.setVersion(logSegment.version)); - - return logSegmentOpt - .map(logSegment -> createSnapshot(logSegment, engine, snapshotContext)) - .orElseThrow(() -> new TableNotFoundException(tablePath.toString())); - } - private SnapshotImpl createSnapshot( LogSegment initSegment, Engine engine, SnapshotQueryContext snapshotContext) { final String startingFromStr = @@ -341,78 +317,77 @@ private SnapshotImpl createSnapshot( } /** - * Get the LogSegment that will help in computing the Snapshot of the table at DeltaLog - * initialization, or None if the directory was empty/missing. + * Generates a {@link LogSegment} for the given `versionToLoadOpt`. If no `versionToLoadOpt` is + * provided, generates a {@code LogSegment} for the latest version of the table. + * + *

This primarily consists of three steps: * - * @param startingCheckpoint A checkpoint that we can start our listing from + *

    + *
  1. First, determine the starting checkpoint version that is at or before `versionToLoadOpt`. + * If no `versionToLoadOpt` is provided, will use the checkpoint pointed to by the + * _last_checkpoint file. + *
  2. Second, LIST the _delta_log for all delta and checkpoint files newer than the starting + * checkpoint version. + *
  3. Third, process and validate this list of _delta_log files to yield a {@code LogSegment}. + *
*/ - private Optional getLogSegmentFrom( - Engine engine, Optional startingCheckpoint) { - return getLogSegmentForVersion( - engine, startingCheckpoint.map(x -> x.version), Optional.empty()); + private LogSegment getLogSegmentForVersion(Engine engine, Optional versionToLoadOpt) { + final String versionToLoadStr = versionToLoadOpt.map(String::valueOf).orElse("latest"); + logger.info("Loading log segment for version {}", versionToLoadStr); + + //////////////////////////////////////////////////////////////////////////////////////////////// + // Step 1: Find the latest checkpoint version. If $versionToLoadOpt is empty, use the version // + // referenced by the _LAST_CHECKPOINT file. If $versionToLoad is present, search for // + // the previous latest complete checkpoint at or before $versionToLoad. // + //////////////////////////////////////////////////////////////////////////////////////////////// + + final Optional getStartCheckpointVersionOpt = + getStartCheckpointVersion(engine, versionToLoadOpt); + + // TODO: make this method *deep*. Conslidate all of the getLogSegment methods to one. + + return getLogSegmentForVersion(engine, getStartCheckpointVersionOpt, versionToLoadOpt); } /** - * Get a list of files that can be used to compute a Snapshot at version `versionToLoad`, if - * `versionToLoad` is not provided, will generate the list of files that are needed to load the - * latest version of the Delta table. This method also performs checks to ensure that the delta - * files are contiguous. - * - * @param startCheckpoint A potential start version to perform the listing of the DeltaLog, - * typically that of a known checkpoint. If this version's not provided, we will start listing - * from version 0. - * @param versionToLoad A specific version to load. Typically used with time travel and the Delta - * streaming source. If not provided, we will try to load the latest version of the table. - * @return Some LogSegment to build a Snapshot if files do exist after the given startCheckpoint. - * None, if the delta log directory was missing or empty. + * Helper function for the {@link #getLogSegmentForVersion(Engine, Optional)} above. Exposes the + * startCheckpoint param for testing. */ - public Optional getLogSegmentForVersion( - Engine engine, Optional startCheckpoint, Optional versionToLoad) { - // Only use startCheckpoint if it is <= versionToLoad - Optional startCheckpointToUse = - startCheckpoint.filter(v -> !versionToLoad.isPresent() || v <= versionToLoad.get()); - - // if we are loading a specific version and there is no usable starting checkpoint - // try to load a checkpoint that is <= version to load - if (!startCheckpointToUse.isPresent() && versionToLoad.isPresent()) { - long beforeVersion = versionToLoad.get() + 1; - long startTimeMillis = System.currentTimeMillis(); - startCheckpointToUse = - findLastCompleteCheckpointBefore(engine, logPath, beforeVersion).map(x -> x.version); - - logger.info( - "{}: Took {}ms to load last checkpoint before version {}", - tablePath, - System.currentTimeMillis() - startTimeMillis, - beforeVersion); - } - - long startVersion = - startCheckpointToUse.orElseGet( + @VisibleForTesting + public LogSegment getLogSegmentForVersion( + Engine engine, Optional startCheckpointVersionOpt, Optional versionToLoadOpt) { + ///////////////////////////////////////////////////////////////// + // Step 2: Determine the actual version to start listing from. // + ///////////////////////////////////////////////////////////////// + + final long listFromStartVersion = + startCheckpointVersionOpt.orElseGet( () -> { logger.warn( "{}: Starting checkpoint is missing. Listing from version as 0", tablePath); return 0L; }); - long startTimeMillis = System.currentTimeMillis(); + ///////////////////////////////////////////////////////////////// + // Step 3: List the files from $startVersion to $versionToLoad // + ///////////////////////////////////////////////////////////////// + + final long startTimeMillis = System.currentTimeMillis(); final List newFiles = DeltaLogActionUtils.listDeltaLogFiles( engine, new HashSet<>(Arrays.asList(DeltaLogFileType.COMMIT, DeltaLogFileType.CHECKPOINT)), tablePath, - startVersion, - versionToLoad, + listFromStartVersion, + versionToLoadOpt, true /* mustBeRecreatable */); - logger.info( "{}: Took {}ms to list the files after starting checkpoint", tablePath, System.currentTimeMillis() - startTimeMillis); - startTimeMillis = System.currentTimeMillis(); try { - return getLogSegmentForVersion(engine, startCheckpointToUse, versionToLoad, newFiles); + return constructLogSegmentFromFileList(startCheckpointVersionOpt, versionToLoadOpt, newFiles); } finally { logger.info( "{}: Took {}ms to construct a log segment", @@ -425,8 +400,7 @@ public Optional getLogSegmentForVersion( * Helper function for the getLogSegmentForVersion above. Called with a provided files list, and * will then try to construct a new LogSegment using that. */ - protected Optional getLogSegmentForVersion( - Engine engine, + private LogSegment constructLogSegmentFromFileList( Optional startCheckpointOpt, Optional versionToLoadOpt, List newFiles) { @@ -625,14 +599,56 @@ protected Optional getLogSegmentForVersion( }) .orElse(Collections.emptyList()); - return Optional.of( - new LogSegment( - logPath, - newVersion, - deltasAfterCheckpoint, - newCheckpointFiles, - newCheckpointOpt.map(x -> x.version), - lastCommitTimestamp)); + return new LogSegment( + logPath, + newVersion, + deltasAfterCheckpoint, + newCheckpointFiles, + newCheckpointOpt.map(x -> x.version), + lastCommitTimestamp); + } + + ///////////////////////// + // getLogSegment utils // + ///////////////////////// + + /** + * Determine the starting checkpoint version that is at or before `versionToLoadOpt`. If no + * `versionToLoadOpt` is provided, will use the checkpoint pointed to by the _last_checkpoint + * file. + */ + private Optional getStartCheckpointVersion(Engine engine, Optional versionToLoadOpt) { + return versionToLoadOpt + .map( + versionToLoad -> { + logger.info( + "Finding last complete checkpoint at or before version {}", versionToLoad); + final long startTimeMillis = System.currentTimeMillis(); + return Checkpointer.findLastCompleteCheckpointBefore( + engine, logPath, versionToLoad + 1) + .map(checkpointInstance -> checkpointInstance.version) + .map( + checkpointVersion -> { + checkArgument( + checkpointVersion <= versionToLoad, + "Last complete checkpoint version %s was not <= targetVersion %s", + checkpointVersion, + versionToLoad); + + logger.info( + "{}: Took {}ms to find last complete checkpoint <= targetVersion {}", + tablePath, + System.currentTimeMillis() - startTimeMillis, + versionToLoad); + + return checkpointVersion; + }); + }) + .orElseGet( + () -> { + logger.info("Loading last checkpoint from the _last_checkpoint file"); + return new Checkpointer(logPath).readLastCheckpointFile(engine).map(x -> x.version); + }); } /** diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/SnapshotManagerSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/SnapshotManagerSuite.scala index c9a05385822..5fa2cd09106 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/SnapshotManagerSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/SnapshotManagerSuite.scala @@ -215,14 +215,13 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { } }.getOrElse((Seq.empty, Seq.empty)) - val logSegmentOpt = snapshotManager.getLogSegmentForVersion( + val logSegment = snapshotManager.getLogSegmentForVersion( createMockFSListFromEngine(listFromProvider(deltas ++ checkpointFiles)("/"), new MockSidecarParquetHandler(expectedSidecars), new MockSidecarJsonHandler(expectedSidecars)), Optional.empty(), versionToLoad ) - assert(logSegmentOpt.isPresent()) val expectedDeltas = deltaFileStatuses( deltaVersions.filter { v => @@ -241,7 +240,7 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { }.getOrElse(Seq.empty) checkLogSegment( - logSegmentOpt.get(), + logSegment, expectedVersion = versionToLoad.orElse(deltaVersions.max), expectedDeltas = expectedDeltas, expectedCheckpoints = expectedCheckpoints, @@ -490,14 +489,13 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { listFromProvider(files)(filePath) } for (checkpointV <- Seq(10, 20)) { - val logSegmentOpt = snapshotManager.getLogSegmentForVersion( + val logSegment = snapshotManager.getLogSegmentForVersion( createMockFSListFromEngine(listFrom(checkpointV)(_)), Optional.of(checkpointV), Optional.empty() ) - assert(logSegmentOpt.isPresent()) checkLogSegment( - logSegmentOpt.get(), + logSegment, expectedVersion = 24, expectedDeltas = deltaFileStatuses(21L until 25L), expectedCheckpoints = singularCheckpointFileStatuses(Seq(20L)), @@ -524,15 +522,8 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { } test("getLogSegmentForVersion: versionToLoad not constructable from history") { - val files = deltaFileStatuses(20L until 25L) ++ singularCheckpointFileStatuses(Seq(20L)) testExpectedError[RuntimeException]( - files, - versionToLoad = Optional.of(15), - expectedErrorMessageContains = "Cannot load table version 15" - ) - testExpectedError[RuntimeException]( - files, - startCheckpoint = Optional.of(20), + deltaFileStatuses(20L until 25L) ++ singularCheckpointFileStatuses(Seq(20L)), versionToLoad = Optional.of(15), expectedErrorMessageContains = "Cannot load table version 15" ) @@ -744,12 +735,6 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { versionToLoad = Optional.of(17), expectedErrorMessageContains = "missing log file for version 0" ) - testExpectedError[InvalidTableException]( - deltaFileStatuses(15L until 25L) ++ singularCheckpointFileStatuses(Seq(20L)), - startCheckpoint = Optional.of(20), - versionToLoad = Optional.of(17), - expectedErrorMessageContains = "missing log file for version 0" - ) testExpectedError[InvalidTableException]( deltaFileStatuses((0L until 5L) ++ (6L until 9L)), expectedErrorMessageContains = "are not contiguous" @@ -830,15 +815,14 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { .take(4) val checkpoints = singularCheckpointFileStatuses(validVersions) val deltas = deltaFileStatuses(deltaVersions) - val logSegmentOpt = snapshotManager.getLogSegmentForVersion( + val logSegment = snapshotManager.getLogSegmentForVersion( createMockFSListFromEngine(deltas ++ corruptedCheckpoint ++ checkpoints), Optional.empty(), Optional.empty() ) val checkpointVersion = validVersions.sorted.lastOption - assert(logSegmentOpt.isPresent()) checkLogSegment( - logSegment = logSegmentOpt.get(), + logSegment, expectedVersion = deltaVersions.max, expectedDeltas = deltaFileStatuses( deltaVersions.filter(_ > checkpointVersion.getOrElse(-1L))), @@ -860,6 +844,20 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { assert(exMsg.contains("Missing checkpoint at version 1")) } + + test("getLogSegmentForVersion: startCheckpoint is greater than versionToLoad") { + val exMsg = intercept[IllegalArgumentException] { + snapshotManager.getLogSegmentForVersion( + createMockFSListFromEngine( + singularCheckpointFileStatuses(Seq(10)) ++ deltaFileStatuses(10L until 15L) + ), + Optional.of(10), // startCheckpoint + Optional.of(7) // versionToLoad + ) + }.getMessage + + assert(exMsg.contains("endVersion=7 provided is less than startVersion=10")) + } } trait SidecarIteratorProvider extends VectorTestUtils { From fa02a59ba861407c2f5a278eca8227d7b2d71178 Mon Sep 17 00:00:00 2001 From: Johan Lasperas Date: Mon, 27 Jan 2025 19:01:22 +0100 Subject: [PATCH 18/28] [Spark] Add assertion after resolving CHECK constraint child expressions (#4084) ## Description This adds an assertion after we attempt resolution to log additional information. The assertion will fail in testing mode but only emit an event outside of testing. This can later be turned into an actual exception if the data collected shows it's the right thing to do. ## How was this patch tested? N/A, logging only --- .../DeltaInvariantCheckerExec.scala | 22 +++++++++++++++++-- 1 file changed, 20 insertions(+), 2 deletions(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/constraints/DeltaInvariantCheckerExec.scala b/spark/src/main/scala/org/apache/spark/sql/delta/constraints/DeltaInvariantCheckerExec.scala index 166886f436a..a6dd6ccc8ae 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/constraints/DeltaInvariantCheckerExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/constraints/DeltaInvariantCheckerExec.scala @@ -20,6 +20,7 @@ import scala.collection.mutable import org.apache.spark.sql.delta.{DeltaErrors, DeltaIllegalStateException} import org.apache.spark.sql.delta.constraints.Constraints.{Check, NotNull} +import org.apache.spark.sql.delta.metering.DeltaLogging import org.apache.spark.sql.delta.schema.SchemaUtils import org.apache.spark.sql.delta.util.AnalysisHelper @@ -100,7 +101,7 @@ case class DeltaInvariantCheckerExec( copy(child = newChild) } -object DeltaInvariantCheckerExec { +object DeltaInvariantCheckerExec extends DeltaLogging { // Specialized optimizer to run necessary rules so that the check expressions can be evaluated. object DeltaInvariantCheckerOptimizer @@ -172,7 +173,7 @@ object DeltaInvariantCheckerExec { val wrappedPlan: LogicalPlan = ExpressionLogicalPlanWrapper(attributesExtracted) val analyzedLogicalPlan = spark.sessionState.analyzer.execute(wrappedPlan) val optimizedLogicalPlan = DeltaInvariantCheckerOptimizer.execute(analyzedLogicalPlan) - optimizedLogicalPlan match { + val resolvedExpr = optimizedLogicalPlan match { case ExpressionLogicalPlanWrapper(e) => e // This should never happen. case plan => throw new DeltaIllegalStateException( @@ -181,6 +182,23 @@ object DeltaInvariantCheckerExec { "Applying type casting resulted in a bad plan rather than a simple expression.\n" + s"Plan:${plan.prettyJson}\n")) } + // Cap the maximum length when logging an unresolved expression to avoid issues. This is a + // CHECK constraint expression and should be relatively simple. + val MAX_OUTPUT_LENGTH = 10 * 1024 + deltaAssert( + resolvedExpr.resolved, + name = "invariant.unresolvedExpression", + msg = s"CHECK constraint child expression was not properly resolved", + data = Map( + "name" -> name, + "checkExpr" -> expr.treeString.take(MAX_OUTPUT_LENGTH), + "attributesExtracted" -> attributesExtracted.treeString.take(MAX_OUTPUT_LENGTH), + "analyzedLogicalPlan" -> analyzedLogicalPlan.treeString.take(MAX_OUTPUT_LENGTH), + "optimizedLogicalPlan" -> optimizedLogicalPlan.treeString.take(MAX_OUTPUT_LENGTH), + "resolvedExpr" -> resolvedExpr.treeString.take(MAX_OUTPUT_LENGTH) + ) + ) + resolvedExpr } CheckDeltaInvariant(executableExpr, columnExtractors.toMap, constraint) From 0b7eed9fc784340c0c10c3e5e5a1a991fe8d3054 Mon Sep 17 00:00:00 2001 From: Xin Huang <42597328+huan233usc@users.noreply.github.com> Date: Mon, 27 Jan 2025 15:08:32 -0800 Subject: [PATCH 19/28] [Kernel]Collect sum of added file size in Transaction metric (#4092) #### Which Delta project/connector is this regarding? - [ ] Spark - [ ] Standalone - [ ] Flink - [x] Kernel - [ ] Other (fill in here) ## Description This PR collects the metric for added file size. Together with the added file count, they will be used further for generating CRC files See https://github.com/delta-io/delta/pull/4091 for e2e poc ## How was this patch tested? Unit test ## Does this PR introduce _any_ user-facing changes? No --- .../io/delta/kernel/internal/TransactionImpl.java | 2 ++ .../internal/metrics/TransactionMetrics.java | 14 ++++++++++++-- .../kernel/metrics/TransactionMetricsResult.java | 9 ++++++++- .../metrics/MetricsReportSerializerSuite.scala | 7 +++++-- .../defaults/metrics/TransactionReportSuite.scala | 14 +++++++++++--- 5 files changed, 38 insertions(+), 8 deletions(-) diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionImpl.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionImpl.java index 97f07f544dc..3f9d7706f81 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionImpl.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionImpl.java @@ -341,6 +341,8 @@ private TransactionCommitResult doCommit( transactionMetrics.totalActionsCounter.increment(); if (!action.isNullAt(ADD_FILE_ORDINAL)) { transactionMetrics.addFilesCounter.increment(); + transactionMetrics.addFilesSizeInBytesCounter.increment( + new AddFile(action.getStruct(ADD_FILE_ORDINAL)).getSize()); } else if (!action.isNullAt(REMOVE_FILE_ORDINAL)) { transactionMetrics.removeFilesCounter.increment(); } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/metrics/TransactionMetrics.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/metrics/TransactionMetrics.java index ecd697cf584..e0b33f35cf4 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/metrics/TransactionMetrics.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/metrics/TransactionMetrics.java @@ -36,6 +36,8 @@ public class TransactionMetrics { public final Counter removeFilesCounter = new Counter(); public final Counter totalActionsCounter = new Counter(); + // TODO: collect removed file's total size. + public final Counter addFilesSizeInBytesCounter = new Counter(); /** * Resets the action counters (addFilesCounter, removeFilesCounter and totalActionsCounter) to 0. @@ -45,6 +47,7 @@ public class TransactionMetrics { */ public void resetActionCounters() { addFilesCounter.reset(); + addFilesSizeInBytesCounter.reset(); removeFilesCounter.reset(); totalActionsCounter.reset(); } @@ -55,6 +58,7 @@ public TransactionMetricsResult captureTransactionMetricsResult() { final long totalCommitDurationNs = totalCommitTimer.totalDurationNs(); final long numCommitAttempts = commitAttemptsCounter.value(); final long numAddFiles = addFilesCounter.value(); + final long totalAddFilesSizeInBytes = addFilesSizeInBytesCounter.value(); final long numRemoveFiles = removeFilesCounter.value(); final long numTotalActions = totalActionsCounter.value(); @@ -82,6 +86,11 @@ public long getNumRemoveFiles() { public long getNumTotalActions() { return numTotalActions; } + + @Override + public long getTotalAddFilesSizeInBytes() { + return totalAddFilesSizeInBytes; + } }; } @@ -89,11 +98,12 @@ public long getNumTotalActions() { public String toString() { return String.format( "TransactionMetrics(totalCommitTimer=%s, commitAttemptsCounter=%s, addFilesCounter=%s, " - + "removeFilesCounter=%s, totalActionsCounter=%s)", + + "removeFilesCounter=%s, totalActionsCounter=%s, totalAddFilesSizeInBytes=%s)", totalCommitTimer, commitAttemptsCounter, addFilesCounter, removeFilesCounter, - totalActionsCounter); + totalActionsCounter, + addFilesSizeInBytesCounter); } } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/metrics/TransactionMetricsResult.java b/kernel/kernel-api/src/main/java/io/delta/kernel/metrics/TransactionMetricsResult.java index d582cbaf987..d357bed3e0f 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/metrics/TransactionMetricsResult.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/metrics/TransactionMetricsResult.java @@ -23,7 +23,8 @@ "numCommitAttempts", "numAddFiles", "numRemoveFiles", - "numTotalActions" + "numTotalActions", + "totalAddFilesSizeInBytes" }) public interface TransactionMetricsResult { @@ -50,4 +51,10 @@ public interface TransactionMetricsResult { * transaction this metric may be incomplete. */ long getNumTotalActions(); + + /** + * @return the sum of size of added files committed in this transaction. For a failed transaction + * this metric may be incomplete. + */ + long getTotalAddFilesSizeInBytes(); } diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/metrics/MetricsReportSerializerSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/metrics/MetricsReportSerializerSuite.scala index 68ece3fe59c..d88cbbab4bb 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/metrics/MetricsReportSerializerSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/metrics/MetricsReportSerializerSuite.scala @@ -117,7 +117,8 @@ class MetricsReportSerializerSuite extends AnyFunSuite { |"numCommitAttempts":${transactionMetrics.getNumCommitAttempts}, |"numAddFiles":${transactionMetrics.getNumAddFiles}, |"numRemoveFiles":${transactionMetrics.getNumRemoveFiles}, - |"numTotalActions":${transactionMetrics.getNumTotalActions} + |"numTotalActions":${transactionMetrics.getNumTotalActions}, + |"totalAddFilesSizeInBytes":${transactionMetrics.getTotalAddFilesSizeInBytes} |} |} |""".stripMargin.replaceAll("\n", "") @@ -135,6 +136,7 @@ class MetricsReportSerializerSuite extends AnyFunSuite { transactionMetrics1.commitAttemptsCounter.increment(2) transactionMetrics1.addFilesCounter.increment(82) transactionMetrics1.totalActionsCounter.increment(90) + transactionMetrics1.addFilesSizeInBytesCounter.increment(100) val transactionReport1 = new TransactionReportImpl( "/table/path", @@ -163,7 +165,8 @@ class MetricsReportSerializerSuite extends AnyFunSuite { |"numCommitAttempts":2, |"numAddFiles":82, |"numRemoveFiles":0, - |"numTotalActions":90 + |"numTotalActions":90, + |"totalAddFilesSizeInBytes":100 |} |} |""".stripMargin.replaceAll("\n", "") diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/metrics/TransactionReportSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/metrics/TransactionReportSuite.scala index 93f91995a62..0b8cd546ed4 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/metrics/TransactionReportSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/metrics/TransactionReportSuite.scala @@ -100,6 +100,7 @@ class TransactionReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectedNumTotalActions: Long = 0, expectedCommitVersion: Option[Long] = None, expectedNumAttempts: Long = 1, + expectedTotalAddFilesSizeInBytes: Long = 0, buildTransaction: (TransactionBuilder, Engine) => Transaction = (tb, e) => tb.build(e), engineInfo: String = "test-engine-info", operation: Operation = Operation.MANUAL_UPDATE @@ -146,6 +147,8 @@ class TransactionReportSuite extends AnyFunSuite with MetricsReportTestUtils { assert(transactionReport.getTransactionMetrics.getNumCommitAttempts == expectedNumAttempts) assert(transactionReport.getTransactionMetrics.getNumAddFiles == expectedNumAddFiles) + assert(transactionReport.getTransactionMetrics.getTotalAddFilesSizeInBytes + == expectedTotalAddFilesSizeInBytes) assert(transactionReport.getTransactionMetrics.getNumRemoveFiles == expectedNumRemoveFiles) assert(transactionReport.getTransactionMetrics.getNumTotalActions == expectedNumTotalActions) } @@ -173,7 +176,8 @@ class TransactionReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectedBaseSnapshotVersion = 0, expectedNumAddFiles = 1, expectedNumTotalActions = 2, // commitInfo + addFile - expectedCommitVersion = Some(1) + expectedCommitVersion = Some(1), + expectedTotalAddFilesSizeInBytes = 100 ) // Commit 2 AddFiles @@ -186,6 +190,7 @@ class TransactionReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectedNumTotalActions = 3, // commitInfo + addFile expectedCommitVersion = Some(2), engineInfo = "foo", + expectedTotalAddFilesSizeInBytes = 200, operation = Operation.WRITE ) @@ -232,7 +237,8 @@ class TransactionReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectedBaseSnapshotVersion = 0, expectedNumAddFiles = 2, expectedNumTotalActions = 3, // commitInfo + addFile - expectedCommitVersion = Some(1) + expectedCommitVersion = Some(1), + expectedTotalAddFilesSizeInBytes = 200 ) } } @@ -249,6 +255,7 @@ class TransactionReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectedNumAddFiles = 1, expectedNumTotalActions = 4, // protocol, metadata, commitInfo expectedCommitVersion = Some(0), + expectedTotalAddFilesSizeInBytes = 100, buildTransaction = (transBuilder, engine) => { transBuilder .withSchema(engine, new StructType().add("id", IntegerType.INTEGER)) @@ -304,7 +311,8 @@ class TransactionReportSuite extends AnyFunSuite with MetricsReportTestUtils { expectedNumAddFiles = 1, expectedNumTotalActions = 2, // commitInfo + removeFile expectedCommitVersion = Some(2), - expectedNumAttempts = 2 + expectedNumAttempts = 2, + expectedTotalAddFilesSizeInBytes = 100 ) } } From 2ffdbbf2748a2355fa45b9795787091ca4795567 Mon Sep 17 00:00:00 2001 From: Hao Jiang Date: Tue, 28 Jan 2025 13:00:56 -0500 Subject: [PATCH 20/28] [UniForm] Cast Iceberg TIME to Spark Long (#4093) #### Which Delta project/connector is this regarding? - [x] Spark - [ ] Standalone - [ ] Flink - [ ] Kernel - [ ] Other (fill in here) ## Description This PR introduces the feature that allows UniForm to cast Iceberg TIME type to Spark Long data type. The result will be represented as the microseconds since midnight. ## How was this patch tested? UT ## Does this PR introduce _any_ user-facing changes? No --- .../spark/sql/delta/IcebergSchemaUtils.scala | 15 ++- .../apache/spark/sql/delta/IcebergTable.scala | 29 ++-- .../delta/TypeToSparkTypeWithCustomCast.scala | 127 ++++++++++++++++++ .../spark/sql/delta/CloneIcebergSuite.scala | 56 ++++++++ .../sql/delta/NonSparkIcebergTestUtils.scala | 89 ++++++++++++ .../spark/sql/delta/DeltaAnalysis.scala | 4 +- .../apache/spark/sql/delta/DeltaConfig.scala | 7 + .../delta/commands/CloneTableCommand.scala | 4 +- .../delta/commands/convert/ConvertUtils.scala | 14 +- .../sql/delta/sources/DeltaSQLConf.scala | 8 ++ 10 files changed, 330 insertions(+), 23 deletions(-) create mode 100644 iceberg/src/main/scala/org/apache/spark/sql/delta/TypeToSparkTypeWithCustomCast.scala create mode 100644 iceberg/src/test/scala/org/apache/spark/sql/delta/NonSparkIcebergTestUtils.scala diff --git a/iceberg/src/main/scala/org/apache/spark/sql/delta/IcebergSchemaUtils.scala b/iceberg/src/main/scala/org/apache/spark/sql/delta/IcebergSchemaUtils.scala index 3c2f0cf0d81..3895850f365 100644 --- a/iceberg/src/main/scala/org/apache/spark/sql/delta/IcebergSchemaUtils.scala +++ b/iceberg/src/main/scala/org/apache/spark/sql/delta/IcebergSchemaUtils.scala @@ -20,6 +20,7 @@ import org.apache.spark.sql.delta.DeltaColumnMapping import org.apache.spark.sql.delta.schema.SchemaMergingUtils import org.apache.iceberg.Schema import org.apache.iceberg.spark.SparkSchemaUtil +import org.apache.iceberg.types.TypeUtil import org.apache.spark.sql.types.{MetadataBuilder, StructType} @@ -29,12 +30,18 @@ object IcebergSchemaUtils { * Given an iceberg schema, convert it to a Spark schema. This conversion will keep the Iceberg * column IDs (used to read Parquet files) in the field metadata * - * @param icebergSchema - * @return StructType for the converted schema + * @param icebergSchema Iceberg schema + * @param castTimeType cast Iceberg TIME type to Spark Long + * @return Spark schema converted from Iceberg schema */ - def convertIcebergSchemaToSpark(icebergSchema: Schema): StructType = { + def convertIcebergSchemaToSpark(icebergSchema: Schema, + castTimeType: Boolean = false): StructType = { // Convert from Iceberg schema to Spark schema but without the column IDs - val baseConvertedSchema = SparkSchemaUtil.convert(icebergSchema) + val baseConvertedSchema = if (castTimeType) { + TypeUtil.visit(icebergSchema, new TypeToSparkTypeWithCustomCast()).asInstanceOf[StructType] + } else { + SparkSchemaUtil.convert(icebergSchema) + } // For each field, find the column ID (fieldId) and add to the StructField metadata SchemaMergingUtils.transformColumns(baseConvertedSchema) { (path, field, _) => diff --git a/iceberg/src/main/scala/org/apache/spark/sql/delta/IcebergTable.scala b/iceberg/src/main/scala/org/apache/spark/sql/delta/IcebergTable.scala index 635596b064e..5e233ec3bd8 100644 --- a/iceberg/src/main/scala/org/apache/spark/sql/delta/IcebergTable.scala +++ b/iceberg/src/main/scala/org/apache/spark/sql/delta/IcebergTable.scala @@ -20,7 +20,7 @@ import java.util.Locale import scala.collection.JavaConverters._ -import org.apache.spark.sql.delta.{DeltaColumnMapping, DeltaColumnMappingMode, DeltaConfigs, IdMapping, SerializableFileStatus} +import org.apache.spark.sql.delta.{DeltaColumnMapping, DeltaColumnMappingMode, DeltaConfigs, IdMapping, SerializableFileStatus, Snapshot} import org.apache.spark.sql.delta.schema.SchemaMergingUtils import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.iceberg.{Table, TableProperties} @@ -36,7 +36,7 @@ import org.apache.spark.sql.types.StructType * A target Iceberg table for conversion to a Delta table. * * @param icebergTable the Iceberg table underneath. - * @param existingSchema schema used for incremental update, none for initial conversion. + * @param deltaSnapshot the delta snapshot used for incremental update, none for initial conversion. * @param convertStats flag for disabling convert iceberg stats directly into Delta stats. * If you wonder why we need this flag, you are not alone. * This flag is only used by the old, obsolete, legacy command @@ -49,23 +49,31 @@ import org.apache.spark.sql.types.StructType class IcebergTable( spark: SparkSession, icebergTable: Table, - existingSchema: Option[StructType], + deltaSnapshot: Option[Snapshot], convertStats: Boolean) extends ConvertTargetTable { - def this(spark: SparkSession, basePath: String, existingSchema: Option[StructType], + def this(spark: SparkSession, basePath: String, deltaTable: Option[Snapshot], convertStats: Boolean = true) = // scalastyle:off deltahadoopconfiguration this(spark, new HadoopTables(spark.sessionState.newHadoopConf).load(basePath), - existingSchema, convertStats) + deltaTable, convertStats) // scalastyle:on deltahadoopconfiguration + protected val existingSchema: Option[StructType] = deltaSnapshot.map(_.schema) + private val partitionEvolutionEnabled = spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_CONVERT_ICEBERG_PARTITION_EVOLUTION_ENABLED) private val bucketPartitionEnabled = spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_CONVERT_ICEBERG_BUCKET_PARTITION_ENABLED) - private val fieldPathToPhysicalName = + // When a table is CLONED/federated with the session conf ON, it will have the table property + // set and will continue to support CAST TIME TYPE even when later the session conf is OFF. + private val castTimeType = + spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_CONVERT_ICEBERG_CAST_TIME_TYPE) || + deltaSnapshot.exists(s => DeltaConfigs.CAST_ICEBERG_TIME_TYPE.fromMetaData(s.metadata)) + + protected val fieldPathToPhysicalName = existingSchema.map { SchemaMergingUtils.explode(_).collect { case (path, field) if DeltaColumnMapping.hasPhysicalName(field) => @@ -76,7 +84,7 @@ class IcebergTable( private val convertedSchema = { // Reuse physical names of existing columns. val mergedSchema = DeltaColumnMapping.setPhysicalNames( - IcebergSchemaUtils.convertIcebergSchemaToSpark(icebergTable.schema()), + IcebergSchemaUtils.convertIcebergSchemaToSpark(icebergTable.schema(), castTimeType), fieldPathToPhysicalName) // Assign physical names to new columns. @@ -88,8 +96,13 @@ class IcebergTable( override val properties: Map[String, String] = { val maxSnapshotAgeMs = PropertyUtil.propertyAsLong(icebergTable.properties, TableProperties.MAX_SNAPSHOT_AGE_MS, TableProperties.MAX_SNAPSHOT_AGE_MS_DEFAULT) + val castTimeTypeConf = if (castTimeType) { + Some((DeltaConfigs.CAST_ICEBERG_TIME_TYPE.key -> "true")) + } else { + None + } icebergTable.properties().asScala.toMap + (DeltaConfigs.COLUMN_MAPPING_MODE.key -> "id") + - (DeltaConfigs.LOG_RETENTION.key -> s"$maxSnapshotAgeMs millisecond") + (DeltaConfigs.LOG_RETENTION.key -> s"$maxSnapshotAgeMs millisecond") ++ castTimeTypeConf } override val partitionSchema: StructType = { diff --git a/iceberg/src/main/scala/org/apache/spark/sql/delta/TypeToSparkTypeWithCustomCast.scala b/iceberg/src/main/scala/org/apache/spark/sql/delta/TypeToSparkTypeWithCustomCast.scala new file mode 100644 index 00000000000..67530cd6fe1 --- /dev/null +++ b/iceberg/src/main/scala/org/apache/spark/sql/delta/TypeToSparkTypeWithCustomCast.scala @@ -0,0 +1,127 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.delta.commands.convert + +import java.util + +import scala.collection.JavaConverters._ + +import org.apache.iceberg.MetadataColumns +import org.apache.iceberg.Schema +import org.apache.iceberg.relocated.com.google.common.collect.Lists +import org.apache.iceberg.types.Type +import org.apache.iceberg.types.Type.TypeID._ +import org.apache.iceberg.types.Types +import org.apache.iceberg.types.TypeUtil + +import org.apache.spark.sql.types.ArrayType +import org.apache.spark.sql.types.BinaryType +import org.apache.spark.sql.types.BooleanType +import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.types.DateType +import org.apache.spark.sql.types.DecimalType +import org.apache.spark.sql.types.DoubleType +import org.apache.spark.sql.types.FloatType +import org.apache.spark.sql.types.IntegerType +import org.apache.spark.sql.types.LongType +import org.apache.spark.sql.types.MapType +import org.apache.spark.sql.types.Metadata +import org.apache.spark.sql.types.MetadataBuilder +import org.apache.spark.sql.types.StringType +import org.apache.spark.sql.types.StructField +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types.TimestampNTZType +import org.apache.spark.sql.types.TimestampType + +/** + * This class is copied from [[org.apache.iceberg.spark.TypeToSparkType]] to + * add custom type casting. Currently, it supports the following casting + * * Iceberg TIME -> Spark Long + * + */ +class TypeToSparkTypeWithCustomCast extends TypeUtil.SchemaVisitor[DataType] { + + val METADATA_COL_ATTR_KEY = "__metadata_col"; + + override def schema(schema: Schema, structType: DataType): DataType = structType + + override def struct(struct: Types.StructType, fieldResults: util.List[DataType]): DataType = { + val fields = struct.fields(); + val sparkFields: util.List[StructField] = + Lists.newArrayListWithExpectedSize(fieldResults.size()) + for (i <- 0 until fields.size()) { + val field = fields.get(i) + val `type` = fieldResults.get(i) + val metadata = fieldMetadata(field.fieldId()) + var sparkField = StructField.apply(field.name(), `type`, field.isOptional(), metadata) + if (field.doc() != null) { + sparkField = sparkField.withComment(field.doc()) + } + sparkFields.add(sparkField) + } + + StructType.apply(sparkFields) + } + + override def field(field: Types.NestedField, fieldResult: DataType): DataType = fieldResult + + override def list(list: Types.ListType, elementResult: DataType): DataType = + ArrayType.apply(elementResult, list.isElementOptional()) + + override def map(map: Types.MapType, keyResult: DataType, valueResult: DataType): DataType = + MapType.apply(keyResult, valueResult, map.isValueOptional()) + + override def primitive(primitive: Type.PrimitiveType): DataType = { + primitive.typeId() match { + case BOOLEAN => BooleanType + case INTEGER => IntegerType + case LONG => LongType + case FLOAT => FloatType + case DOUBLE => DoubleType + case DATE => DateType + // This line is changed to allow casting TIME to Spark Long. + // The result is microseconds since midnight. + case TIME => LongType + case TIMESTAMP => + val ts = primitive.asInstanceOf[Types.TimestampType] + if (ts.shouldAdjustToUTC()) { + TimestampType + } else { + TimestampNTZType + } + case STRING => StringType + case UUID => // use String + StringType + case FIXED => BinaryType + case BINARY => BinaryType + case DECIMAL => + val decimal = primitive.asInstanceOf[Types.DecimalType] + DecimalType.apply(decimal.precision(), decimal.scale()); + case _ => + throw new UnsupportedOperationException( + "Cannot convert unknown type to Spark: " + primitive); + } + } + + private def fieldMetadata(fieldId: Int): Metadata = { + if (MetadataColumns.metadataFieldIds().contains(fieldId)) { + return new MetadataBuilder().putBoolean(METADATA_COL_ATTR_KEY, value = true).build() + } + + Metadata.empty + } +} diff --git a/iceberg/src/test/scala/org/apache/spark/sql/delta/CloneIcebergSuite.scala b/iceberg/src/test/scala/org/apache/spark/sql/delta/CloneIcebergSuite.scala index 083b7c8683f..ed561685d61 100644 --- a/iceberg/src/test/scala/org/apache/spark/sql/delta/CloneIcebergSuite.scala +++ b/iceberg/src/test/scala/org/apache/spark/sql/delta/CloneIcebergSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.delta // scalastyle:off import.ordering.noEmptyLine import java.sql.Date +import java.time.LocalTime import scala.collection.JavaConverters._ import scala.util.Try @@ -26,7 +27,10 @@ import org.apache.spark.sql.delta.commands.convert.ConvertUtils import org.apache.spark.sql.delta.schema.SchemaMergingUtils import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.delta.stats.StatisticsCollection +import org.apache.iceberg.Schema import org.apache.iceberg.hadoop.HadoopTables +import org.apache.iceberg.types.Types +import org.apache.iceberg.types.Types.NestedField import org.apache.spark.SparkConf import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} @@ -451,6 +455,58 @@ class CloneIcebergByPathSuite extends CloneIcebergSuiteBase } } +/** + * This suite test features in Iceberg that is not directly supported by Spark. + * See also [[NonSparkIcebergTestUtils]]. + * We do not put these tests in or extend from [[CloneIcebergSuiteBase]] because they + * use non-Spark way to create test data. + */ +class CloneNonSparkIcebergByPathSuite extends QueryTest + with ConvertIcebergToDeltaUtils { + + protected val cloneTable = "clone" + + private def sourceIdentifier: String = s"iceberg.`$tablePath`" + + private def runCreateOrReplace(mode: String, source: String): DataFrame = { + Try(spark.sql(s"DELETE FROM $cloneTable")) + spark.sql(s"CREATE OR REPLACE TABLE $cloneTable $mode CLONE $source") + } + + private val mode = "SHALLOW" + + test("cast Iceberg TIME to Spark long") { + withTable(table, cloneTable) { + val schema = new Schema( + Seq[NestedField]( + NestedField.required(1, "id", Types.IntegerType.get), + NestedField.required(2, "event_time", Types.TimeType.get) + ).asJava + ) + val rows = Seq( + Map( + "id" -> 1, + "event_time" -> LocalTime.of(14, 30, 11) + ) + ) + NonSparkIcebergTestUtils.createIcebergTable(spark, tablePath, schema, rows) + intercept[UnsupportedOperationException] { + runCreateOrReplace(mode, sourceIdentifier) + } + withSQLConf(DeltaSQLConf.DELTA_CONVERT_ICEBERG_CAST_TIME_TYPE.key -> "true") { + runCreateOrReplace(mode, sourceIdentifier) + val expectedMicrosec = (14 * 3600 + 30 * 60 + 11) * 1000000L + checkAnswer(spark.table(cloneTable), Row(1, expectedMicrosec) :: Nil) + val clonedDeltaTable = DeltaLog.forTable( + spark, + spark.sessionState.catalog.getTableMetadata(TableIdentifier(cloneTable)) + ) + assert(DeltaConfigs.CAST_ICEBERG_TIME_TYPE.fromMetaData(clonedDeltaTable.update().metadata)) + } + } + } +} + class CloneIcebergByNameSuite extends CloneIcebergSuiteBase { override def sourceIdentifier: String = table diff --git a/iceberg/src/test/scala/org/apache/spark/sql/delta/NonSparkIcebergTestUtils.scala b/iceberg/src/test/scala/org/apache/spark/sql/delta/NonSparkIcebergTestUtils.scala new file mode 100644 index 00000000000..95f608f0aad --- /dev/null +++ b/iceberg/src/test/scala/org/apache/spark/sql/delta/NonSparkIcebergTestUtils.scala @@ -0,0 +1,89 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.delta + +import scala.collection.JavaConverters._ + +import org.apache.iceberg.{DataFile, DataFiles, Files, PartitionSpec, Schema, Table} +import org.apache.iceberg.data.GenericRecord +import org.apache.iceberg.data.parquet.GenericParquetWriter +import org.apache.iceberg.hadoop.HadoopTables +import org.apache.iceberg.io.FileAppender +import org.apache.iceberg.parquet.Parquet +import org.apache.iceberg.types.Types +import org.apache.iceberg.types.Types.NestedField + +import org.apache.spark.sql.SparkSession + +object NonSparkIcebergTestUtils { + + /** + * Create an Iceberg table with formats/data types not supported by Spark. + * This is primarily used for compatibility tests. It includes the following features + * * TIME data type that is not supported by Spark. + * @param location Iceberg table root path + * @param schema Iceberg table schema + * @param rows Data rows we write into the table + */ + def createIcebergTable( + spark: SparkSession, + location: String, + schema: Schema, + rows: Seq[Map[String, Any]]): Table = { + // scalastyle:off deltahadoopconfiguration + val tables = new HadoopTables(spark.sessionState.newHadoopConf()) + // scalastyle:on deltahadoopconfiguration + val table = tables.create( + schema, + PartitionSpec.unpartitioned(), + location + ) + + val records = rows.map { row => + val record = GenericRecord.create(schema) + row.foreach { + case (key, value) => record.setField(key, value) + } + record + } + + val parquetLocation = location + "/data/001.parquet" + + val fileAppender: FileAppender[GenericRecord] = Parquet + .write(table.io().newOutputFile(parquetLocation)) + .schema(schema) + .createWriterFunc(GenericParquetWriter.buildWriter) + .overwrite() + .build(); + try { + fileAppender.addAll(records.asJava) + } finally { + fileAppender.close + } + + val dataFile = DataFiles.builder(PartitionSpec.unpartitioned()) + .withInputFile(table.io().newInputFile(parquetLocation)) + .withMetrics(fileAppender.metrics()) + .build(); + + table + .newAppend + .appendFile(dataFile) + .commit + table + } +} diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaAnalysis.scala b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaAnalysis.scala index 0c567eebac0..10d3b6b0e9f 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaAnalysis.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaAnalysis.scala @@ -296,7 +296,7 @@ class DeltaAnalysis(session: SparkSession) resolveCloneCommand( cloneStatement.target, CloneIcebergSource( - table.tableIdentifier, sparkTable = None, tableSchema = None, session), + table.tableIdentifier, sparkTable = None, deltaSnapshot = None, session), cloneStatement) case DataSourceV2Relation(table, _, _, _, _) @@ -313,7 +313,7 @@ class DeltaAnalysis(session: SparkSession) } resolveCloneCommand( cloneStatement.target, - CloneIcebergSource(tableIdent, Some(table), tableSchema = None, session), + CloneIcebergSource(tableIdent, Some(table), deltaSnapshot = None, session), cloneStatement) case u: UnresolvedRelation => diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaConfig.scala b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaConfig.scala index 23deca4ba9c..b70ca428b39 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaConfig.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaConfig.scala @@ -755,6 +755,13 @@ trait DeltaConfigsBase extends DeltaLogging { helpMessage = "needs to be a boolean." ) + val CAST_ICEBERG_TIME_TYPE = buildConfig[Boolean]( + key = "castIcebergTimeType", + defaultValue = "false", + fromString = _.toBoolean, + validationFunction = _ => true, + helpMessage = "Casting Iceberg TIME type to Spark Long type enabled" + ) /** * Enable optimized writes into a Delta table. Optimized writes adds an adaptive shuffle before * the write to write compacted files into a Delta table during a write. diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/commands/CloneTableCommand.scala b/spark/src/main/scala/org/apache/spark/sql/delta/commands/CloneTableCommand.scala index bbfded0256a..94f6afbdf9f 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/commands/CloneTableCommand.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/commands/CloneTableCommand.scala @@ -282,11 +282,11 @@ case class CloneParquetSource( case class CloneIcebergSource( tableIdentifier: TableIdentifier, sparkTable: Option[Table], - tableSchema: Option[StructType], + deltaSnapshot: Option[Snapshot], spark: SparkSession) extends CloneConvertedSource(spark) { override lazy val convertTargetTable: ConvertTargetTable = - ConvertUtils.getIcebergTable(spark, tableIdentifier.table, sparkTable, tableSchema) + ConvertUtils.getIcebergTable(spark, tableIdentifier.table, sparkTable, deltaSnapshot) override def format: String = CloneSourceFormat.ICEBERG diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/commands/convert/ConvertUtils.scala b/spark/src/main/scala/org/apache/spark/sql/delta/commands/convert/ConvertUtils.scala index 482f165fcfa..0fabda5c3e9 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/commands/convert/ConvertUtils.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/commands/convert/ConvertUtils.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.delta.commands.convert import java.lang.reflect.InvocationTargetException -import org.apache.spark.sql.delta.{DeltaColumnMapping, DeltaErrors, SerializableFileStatus} +import org.apache.spark.sql.delta.{DeltaColumnMapping, DeltaErrors, SerializableFileStatus, Snapshot} import org.apache.spark.sql.delta.actions.AddFile import org.apache.spark.sql.delta.metering.DeltaLogging import org.apache.spark.sql.delta.schema.SchemaMergingUtils @@ -70,7 +70,7 @@ trait ConvertUtilsBase extends DeltaLogging { * @param spark: the spark session to use. * @param targetDir: the target directory of the Iceberg table. * @param sparkTable: the optional V2 table interface of the Iceberg table. - * @param tableSchema: the existing converted Delta table schema (if exists) of the Iceberg table. + * @param deltaTable: the existing converted Delta table (if exists) of the Iceberg table. * @param collectStats: collect column stats on convert * @return a target Iceberg table. */ @@ -78,7 +78,7 @@ trait ConvertUtilsBase extends DeltaLogging { spark: SparkSession, targetDir: String, sparkTable: Option[Table], - tableSchema: Option[StructType], + deltaSnapshot: Option[Snapshot], collectStats: Boolean = true): ConvertTargetTable = { try { val convertIcebergStats = collectStats && @@ -88,18 +88,18 @@ trait ConvertUtilsBase extends DeltaLogging { val constFromTable = clazz.getConstructor( classOf[SparkSession], Utils.classForName(icebergLibTableClassPath), - classOf[Option[StructType]], + classOf[Option[Snapshot]], java.lang.Boolean.TYPE ) val method = sparkTable.get.getClass.getMethod("table") - constFromTable.newInstance(spark, method.invoke(sparkTable.get), tableSchema, + constFromTable.newInstance(spark, method.invoke(sparkTable.get), deltaSnapshot, java.lang.Boolean.valueOf(convertIcebergStats)) } else { val baseDir = getQualifiedPath(spark, new Path(targetDir)).toString val constFromPath = clazz.getConstructor( - classOf[SparkSession], classOf[String], classOf[Option[StructType]], + classOf[SparkSession], classOf[String], classOf[Option[Snapshot]], java.lang.Boolean.TYPE) - constFromPath.newInstance(spark, baseDir, tableSchema, + constFromPath.newInstance(spark, baseDir, deltaSnapshot, java.lang.Boolean.valueOf(convertIcebergStats)) } } catch { diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala b/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala index 3a65a3660b8..d43dcf0cb34 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala @@ -1553,6 +1553,14 @@ trait DeltaSQLConfBase { .booleanConf .createWithDefault(false) + val DELTA_CONVERT_ICEBERG_CAST_TIME_TYPE = { + buildConf("convert.iceberg.castTimeType") + .internal() + .doc("Cast Iceberg TIME type to Spark Long when converting to Delta") + .booleanConf + .createWithDefault(false) + } + final object NonDeterministicPredicateWidening { final val OFF = "off" final val LOGGING = "logging" From 0c588369b57a8911c92349c44e696f81c86848e4 Mon Sep 17 00:00:00 2001 From: Sumeet Varma Date: Tue, 28 Jan 2025 15:17:32 -0800 Subject: [PATCH 21/28] [Spark] Skip non-deterministic filters to prevent incorrect file pruning in Delta queries (#4095) #### Which Delta project/connector is this regarding? - [x] Spark - [ ] Standalone - [ ] Flink - [ ] Kernel - [ ] Other (fill in here) ## Description Currently, PrepareDeltaScan does not exclude non-deterministic filters (e.g. rand() < 0.25). So, it ends up creating a file index with these non-deterministic filters applied. The same filters are reapplied downstream which causes them to be applied twice and the final result being unexpected. In general, PrepareDeltaScan should not skip entire files/partitions because of non-deterministic filters. ## How was this patch tested? Build ## Does this PR introduce _any_ user-facing changes? No --- .../spark/sql/delta/stats/PrepareDeltaScan.scala | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/stats/PrepareDeltaScan.scala b/spark/src/main/scala/org/apache/spark/sql/delta/stats/PrepareDeltaScan.scala index 341cecc3ec9..76552115bfe 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/stats/PrepareDeltaScan.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/stats/PrepareDeltaScan.scala @@ -114,20 +114,22 @@ trait PrepareDeltaScanBase extends Rule[LogicalPlan] limitOpt: Option[Int], filters: Seq[Expression], delta: LogicalRelation): DeltaScan = { + // Remove non-deterministic filters (e.g., rand() < 0.25) to prevent incorrect file pruning. + val deterministicFilters = filters.filter(_.deterministic) withStatusCode("DELTA", "Filtering files for query") { if (limitOpt.nonEmpty) { // If we trigger limit push down, the filters must be partition filters. Since // there are no data filters, we don't need to apply Generated Columns // optimization. See `DeltaTableScan` for more details. - return scanGenerator.filesForScan(limitOpt.get, filters) + return scanGenerator.filesForScan(limitOpt.get, deterministicFilters) } val filtersForScan = if (!GeneratedColumn.partitionFilterOptimizationEnabled(spark)) { - filters + deterministicFilters } else { val generatedPartitionFilters = GeneratedColumn.generatePartitionFilters( - spark, scanGenerator.snapshotToScan, filters, delta) - filters ++ generatedPartitionFilters + spark, scanGenerator.snapshotToScan, deterministicFilters, delta) + deterministicFilters ++ generatedPartitionFilters } scanGenerator.filesForScan(filtersForScan) } From c04cb063d0fe52062d538e20db18af63e3b72b6f Mon Sep 17 00:00:00 2001 From: Scott Sandre Date: Tue, 28 Jan 2025 19:27:23 -0800 Subject: [PATCH 22/28] [Kernel] Further refactor and cleanup SnapshotManager (#4090) #### Which Delta project/connector is this regarding? - [ ] Spark - [ ] Standalone - [ ] Flink - [X] Kernel - [ ] Other (fill in here) ## Description This PR further refactors and cleans up SnapshotManager. We consolidate all of our `getLogSegment` methods to just one. It has a deep implementation. This PR adds nice, clean header blocks to describe the steps during log segment construction. ## How was this patch tested? Refactor. Update existing UTs. ## Does this PR introduce _any_ user-facing changes? No. --- .../io/delta/kernel/internal/DeltaErrors.java | 2 +- .../internal/snapshot/SnapshotManager.java | 367 ++++++++---------- .../internal/SnapshotManagerSuite.scala | 307 ++++++--------- .../test/MockFileSystemClientUtils.scala | 19 +- 4 files changed, 292 insertions(+), 403 deletions(-) diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaErrors.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaErrors.java index 4c628c540ff..702b68dabf6 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaErrors.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaErrors.java @@ -49,7 +49,7 @@ public static KernelException versionBeforeFirstAvailableCommit( return new KernelException(message); } - public static KernelException versionAfterLatestCommit( + public static KernelException versionToLoadAfterLatestCommit( String tablePath, long versionToLoad, long latestVersion) { String message = String.format( diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java index 3aaa4d2dd71..4f305ad9e0f 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java @@ -48,7 +48,6 @@ import java.nio.file.FileAlreadyExistsException; import java.util.*; import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Supplier; import java.util.stream.Collectors; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -118,7 +117,8 @@ public Snapshot getSnapshotAt(Engine engine, long version, SnapshotQueryContext * @param millisSinceEpochUTC timestamp to fetch the snapshot for in milliseconds since the unix * epoch * @return a {@link Snapshot} of the table at the provided timestamp - * @throws TableNotFoundException + * @throws TableNotFoundException if the table does not exist + * @throws InvalidTableException if the table is in an invalid state */ public Snapshot getSnapshotForTimestamp( Engine engine, long millisSinceEpochUTC, SnapshotQueryContext snapshotContext) @@ -215,18 +215,12 @@ public void checkpoint(Engine engine, Clock clock, long version) //////////////////// /** - * Given a list of delta versions, verifies that they are (1) contiguous, (2) start with - * expectedStartVersion (if provided), and (3) end with expectedEndVersionOpt (if provided). - * Throws an exception if any of these are not true. + * Verify that a list of delta versions is contiguous. * - * @param versions List of versions in sorted increasing order according + * @throws InvalidTableException if the versions are not contiguous */ @VisibleForTesting - public static void verifyDeltaVersions( - List versions, - Optional expectedStartVersion, - Optional expectedEndVersion, - Path tablePath) { + public static void verifyDeltaVersionsContiguous(List versions, Path tablePath) { for (int i = 1; i < versions.size(); i++) { if (versions.get(i) != versions.get(i - 1) + 1) { throw new InvalidTableException( @@ -234,20 +228,6 @@ public static void verifyDeltaVersions( String.format("Missing delta files: versions are not contiguous: (%s)", versions)); } } - expectedStartVersion.ifPresent( - v -> { - checkArgument( - !versions.isEmpty() && Objects.equals(versions.get(0), v), - "Did not get the first delta file version %s to compute Snapshot", - v); - }); - expectedEndVersion.ifPresent( - v -> { - checkArgument( - !versions.isEmpty() && Objects.equals(ListUtils.getLast(versions), v), - "Did not get the last delta file version %s to compute Snapshot", - v); - }); } /** @@ -331,7 +311,8 @@ private SnapshotImpl createSnapshot( *
  • Third, process and validate this list of _delta_log files to yield a {@code LogSegment}. * */ - private LogSegment getLogSegmentForVersion(Engine engine, Optional versionToLoadOpt) { + @VisibleForTesting + public LogSegment getLogSegmentForVersion(Engine engine, Optional versionToLoadOpt) { final String versionToLoadStr = versionToLoadOpt.map(String::valueOf).orElse("latest"); logger.info("Loading log segment for version {}", versionToLoadStr); @@ -341,39 +322,32 @@ private LogSegment getLogSegmentForVersion(Engine engine, Optional version // the previous latest complete checkpoint at or before $versionToLoad. // //////////////////////////////////////////////////////////////////////////////////////////////// - final Optional getStartCheckpointVersionOpt = + final Optional startCheckpointVersionOpt = getStartCheckpointVersion(engine, versionToLoadOpt); - // TODO: make this method *deep*. Conslidate all of the getLogSegment methods to one. - - return getLogSegmentForVersion(engine, getStartCheckpointVersionOpt, versionToLoadOpt); - } - - /** - * Helper function for the {@link #getLogSegmentForVersion(Engine, Optional)} above. Exposes the - * startCheckpoint param for testing. - */ - @VisibleForTesting - public LogSegment getLogSegmentForVersion( - Engine engine, Optional startCheckpointVersionOpt, Optional versionToLoadOpt) { ///////////////////////////////////////////////////////////////// // Step 2: Determine the actual version to start listing from. // ///////////////////////////////////////////////////////////////// final long listFromStartVersion = - startCheckpointVersionOpt.orElseGet( - () -> { - logger.warn( - "{}: Starting checkpoint is missing. Listing from version as 0", tablePath); - return 0L; - }); + startCheckpointVersionOpt + .map( + version -> { + logger.info("Found a complete checkpoint at version {}.", version); + return version; + }) + .orElseGet( + () -> { + logger.warn("Cannot find a complete checkpoint. Listing from version 0."); + return 0L; + }); ///////////////////////////////////////////////////////////////// // Step 3: List the files from $startVersion to $versionToLoad // ///////////////////////////////////////////////////////////////// final long startTimeMillis = System.currentTimeMillis(); - final List newFiles = + final List listedFileStatuses = DeltaLogActionUtils.listDeltaLogFiles( engine, new HashSet<>(Arrays.asList(DeltaLogFileType.COMMIT, DeltaLogFileType.CHECKPOINT)), @@ -386,31 +360,17 @@ public LogSegment getLogSegmentForVersion( tablePath, System.currentTimeMillis() - startTimeMillis); - try { - return constructLogSegmentFromFileList(startCheckpointVersionOpt, versionToLoadOpt, newFiles); - } finally { - logger.info( - "{}: Took {}ms to construct a log segment", - tablePath, - System.currentTimeMillis() - startTimeMillis); - } - } + //////////////////////////////////////////////////////////////////////// + // Step 4: Perform some basic validations on the listed file statuses // + //////////////////////////////////////////////////////////////////////// - /** - * Helper function for the getLogSegmentForVersion above. Called with a provided files list, and - * will then try to construct a new LogSegment using that. - */ - private LogSegment constructLogSegmentFromFileList( - Optional startCheckpointOpt, - Optional versionToLoadOpt, - List newFiles) { - if (newFiles.isEmpty()) { - if (startCheckpointOpt.isPresent()) { + if (listedFileStatuses.isEmpty()) { + if (startCheckpointVersionOpt.isPresent()) { // We either (a) determined this checkpoint version from the _LAST_CHECKPOINT file, or (b) // found the last complete checkpoint before our versionToLoad. In either case, we didn't // see the checkpoint file in the listing. - // TODO: for case (a), re-load the delta log but ignore the _LAST_CHECKPOINT file. - throw DeltaErrors.missingCheckpoint(tablePath.toString(), startCheckpointOpt.get()); + // TODO: throw a more specific error based on case (a) or (b) + throw DeltaErrors.missingCheckpoint(tablePath.toString(), startCheckpointVersionOpt.get()); } else { // Either no files found OR no *delta* files found even when listing from 0. This means that // the delta table does not exist yet. @@ -419,192 +379,192 @@ private LogSegment constructLogSegmentFromFileList( } } - logDebug( - () -> - format( - "newFiles: %s", - Arrays.toString( - newFiles.stream().map(x -> new Path(x.getPath()).getName()).toArray()))); + logDebugFileStatuses("listedFileStatuses", listedFileStatuses); + + //////////////////////////////////////////////////////////////////////////// + // Step 5: Partition $listedFileStatuses into the checkpoints and deltas. // + //////////////////////////////////////////////////////////////////////////// - Tuple2, List> checkpointsAndDeltas = + final Tuple2, List> listedCheckpointAndDeltaFileStatuses = ListUtils.partition( - newFiles, + listedFileStatuses, fileStatus -> FileNames.isCheckpointFile(new Path(fileStatus.getPath()).getName())); - final List checkpoints = checkpointsAndDeltas._1; - final List deltas = checkpointsAndDeltas._2; - - logDebug( - () -> - format( - "\ncheckpoints: %s\ndeltas: %s", - Arrays.toString( - checkpoints.stream().map(x -> new Path(x.getPath()).getName()).toArray()), - Arrays.toString( - deltas.stream().map(x -> new Path(x.getPath()).getName()).toArray()))); - - // Find the latest checkpoint in the listing that is not older than the versionToLoad - final CheckpointInstance maxCheckpoint = - versionToLoadOpt.map(CheckpointInstance::new).orElse(CheckpointInstance.MAX_VALUE); - logger.debug("lastCheckpoint: {}", maxCheckpoint); + final List listedCheckpointFileStatuses = listedCheckpointAndDeltaFileStatuses._1; + final List listedDeltaFileStatuses = listedCheckpointAndDeltaFileStatuses._2; + + logDebugFileStatuses("listedCheckpointFileStatuses", listedCheckpointFileStatuses); + logDebugFileStatuses("listedDeltaFileStatuses", listedDeltaFileStatuses); - final List checkpointFiles = - checkpoints.stream() + ///////////////////////////////////////////////////////////////////////////////////////////// + // Step 6: Determine the latest complete checkpoint version. The intuition here is that we // + // LISTed from the startingCheckpoint but may have found a newer complete // + // checkpoint. // + ///////////////////////////////////////////////////////////////////////////////////////////// + + final List listedCheckpointInstances = + listedCheckpointFileStatuses.stream() .map(f -> new CheckpointInstance(f.getPath())) .collect(Collectors.toList()); - logDebug(() -> format("checkpointFiles: %s", Arrays.toString(checkpointFiles.toArray()))); - final Optional newCheckpointOpt = - Checkpointer.getLatestCompleteCheckpointFromList(checkpointFiles, maxCheckpoint); - logger.debug("newCheckpointOpt: {}", newCheckpointOpt); + final CheckpointInstance notLaterThanCheckpoint = + versionToLoadOpt.map(CheckpointInstance::new).orElse(CheckpointInstance.MAX_VALUE); - final long newCheckpointVersion = - newCheckpointOpt - .map(c -> c.version) - .orElseGet( - () -> { - // If we do not have any checkpoint, pass new checkpoint version as -1 so that - // first delta version can be 0. - startCheckpointOpt.map( - startCheckpoint -> { - // `startCheckpointOpt` was given but no checkpoint found on delta log. - // This means that the last checkpoint we thought should exist (the - // `_last_checkpoint` file) no longer exists. - // Try to look up another valid checkpoint and create `LogSegment` from it. - // - // FIXME: Something has gone very wrong if the checkpoint doesn't - // exist at all. This code should only handle rejected incomplete - // checkpoints. - final long snapshotVersion = - versionToLoadOpt.orElseGet( - () -> { - final FileStatus lastDelta = deltas.get(deltas.size() - 1); - return FileNames.deltaVersion(new Path(lastDelta.getPath())); - }); - - return getLogSegmentWithMaxExclusiveCheckpointVersion( - snapshotVersion, startCheckpoint) - .orElseThrow( - () -> - // No alternative found, but the directory contains files - // so we cannot return None. - new RuntimeException( - format( - "Checkpoint file to load version: %s is missing.", - startCheckpoint))); - }); + final Optional latestCompleteCheckpointOpt = + Checkpointer.getLatestCompleteCheckpointFromList( + listedCheckpointInstances, notLaterThanCheckpoint); - return -1L; - }); - logger.debug("newCheckpointVersion: {}", newCheckpointVersion); + if (!latestCompleteCheckpointOpt.isPresent() && startCheckpointVersionOpt.isPresent()) { + // In Step 1 we found a $startCheckpointVersion but now our LIST of the file system doesn't + // see it. This means that the checkpoint we thought should exist no longer does. + throw DeltaErrors.missingCheckpoint(tablePath.toString(), startCheckpointVersionOpt.get()); + } + + final long latestCompleteCheckpointVersion = + latestCompleteCheckpointOpt.map(x -> x.version).orElse(-1L); + + logger.info("Latest complete checkpoint version: {}", latestCompleteCheckpointVersion); + + ///////////////////////////////////////////////////////////////////////////////////////////// + // Step 7: Grab all deltas in range [$latestCompleteCheckpointVersion + 1, $versionToLoad] // + ///////////////////////////////////////////////////////////////////////////////////////////// - // TODO: we can calculate deltasAfterCheckpoint and deltaVersions more efficiently - // If there is a new checkpoint, start new lineage there. If `newCheckpointVersion` is -1, - // it will list all existing delta files. final List deltasAfterCheckpoint = - deltas.stream() + listedDeltaFileStatuses.stream() .filter( - fileStatus -> - FileNames.deltaVersion(new Path(fileStatus.getPath())) > newCheckpointVersion) + fs -> { + final long deltaVersion = FileNames.deltaVersion(new Path(fs.getPath())); + return latestCompleteCheckpointVersion + 1 <= deltaVersion + && deltaVersion <= versionToLoadOpt.orElse(Long.MAX_VALUE); + }) .collect(Collectors.toList()); - logDebug( - () -> - format( - "deltasAfterCheckpoint: %s", - Arrays.toString( - deltasAfterCheckpoint.stream() - .map(x -> new Path(x.getPath()).getName()) - .toArray()))); + logDebugFileStatuses("deltasAfterCheckpoint", deltasAfterCheckpoint); + + //////////////////////////////////////////////////////////////////// + // Step 8: Determine the version of the snapshot we can now load. // + //////////////////////////////////////////////////////////////////// final List deltaVersionsAfterCheckpoint = deltasAfterCheckpoint.stream() .map(fileStatus -> FileNames.deltaVersion(new Path(fileStatus.getPath()))) .collect(Collectors.toList()); - logDebug( - () -> format("deltaVersions: %s", Arrays.toString(deltaVersionsAfterCheckpoint.toArray()))); - final long newVersion = deltaVersionsAfterCheckpoint.isEmpty() - ? newCheckpointOpt.get().version + ? latestCompleteCheckpointVersion : ListUtils.getLast(deltaVersionsAfterCheckpoint); - // There should be a delta file present for the newVersion that we are loading - // (Even if `deltasAfterCheckpoint` is empty, `deltas` should not be) - if (deltas.isEmpty() - || FileNames.deltaVersion(deltas.get(deltas.size() - 1).getPath()) < newVersion) { + logger.info("New version to load: {}", newVersion); + + ///////////////////////////////////////////// + // Step 9: Perform some basic validations. // + ///////////////////////////////////////////// + + // Check that we have found at least one checkpoint or delta file + if (!latestCompleteCheckpointOpt.isPresent() && deltasAfterCheckpoint.isEmpty()) { throw new InvalidTableException( - tablePath.toString(), String.format("Missing delta file for version %s", newVersion)); + tablePath.toString(), "No complete checkpoint found and no delta files found"); } - versionToLoadOpt - .filter(v -> v != newVersion) - .ifPresent( - v -> { - throw DeltaErrors.versionAfterLatestCommit(tablePath.toString(), v, newVersion); - }); + // Check that, for a checkpoint at version N, there's a delta file at N, too. + if (latestCompleteCheckpointOpt.isPresent() + && listedDeltaFileStatuses.stream() + .map(x -> FileNames.deltaVersion(new Path(x.getPath()))) + .noneMatch(v -> v == latestCompleteCheckpointVersion)) { + throw new InvalidTableException( + tablePath.toString(), + String.format("Missing delta file for version %s", latestCompleteCheckpointVersion)); + } + + // Check that the $newVersion we actually loaded is the desired $versionToLoad + versionToLoadOpt.ifPresent( + versionToLoad -> { + if (newVersion < versionToLoad) { + throw DeltaErrors.versionToLoadAfterLatestCommit( + tablePath.toString(), versionToLoad, newVersion); + } else if (newVersion > versionToLoad) { + throw new IllegalStateException( + String.format( + "%s: Expected to load version %s but actually loaded version %s", + tablePath, versionToLoad, newVersion)); + } + }); + + if (!deltasAfterCheckpoint.isEmpty()) { + // Check that the delta versions are contiguous + verifyDeltaVersionsContiguous(deltaVersionsAfterCheckpoint, tablePath); - // We may just be getting a checkpoint file after the filtering - if (!deltaVersionsAfterCheckpoint.isEmpty()) { - // If we have deltas after the checkpoint, the first file should be 1 greater than our - // last checkpoint version. If no checkpoint is present, this means the first delta file - // should be version 0. - if (deltaVersionsAfterCheckpoint.get(0) != newCheckpointVersion + 1) { + // Check that the delta versions start with $latestCompleteCheckpointVersion + 1. If they + // don't, then we have a gap in between the checkpoint and the first delta file. + if (!deltaVersionsAfterCheckpoint.get(0).equals(latestCompleteCheckpointVersion + 1)) { throw new InvalidTableException( tablePath.toString(), String.format( - "Unable to reconstruct table state: missing log file for version %s", - newCheckpointVersion + 1)); + "Cannot compute snapshot. Missing delta file version %d.", + latestCompleteCheckpointVersion + 1)); } - verifyDeltaVersions( - deltaVersionsAfterCheckpoint, - Optional.of(newCheckpointVersion + 1), - versionToLoadOpt, - tablePath); + // Note: We have already asserted above that $versionToLoad equals $newVersion. + // Note: We already know that the last element of deltasAfterCheckpoint is $newVersion IF + // $deltasAfterCheckpoint is not empty. logger.info( "Verified delta files are contiguous from version {} to {}", - newCheckpointVersion + 1, + latestCompleteCheckpointVersion + 1, newVersion); } - final long lastCommitTimestamp = deltas.get(deltas.size() - 1).getModificationTime(); + //////////////////////////////////////////////////////////////////////////////////////////// + // Step 10: Grab the actual checkpoint file statuses for latestCompleteCheckpointVersion. // + //////////////////////////////////////////////////////////////////////////////////////////// - final List newCheckpointFiles = - newCheckpointOpt + final List latestCompleteCheckpointFileStatuses = + latestCompleteCheckpointOpt .map( - newCheckpoint -> { + latestCompleteCheckpoint -> { final Set newCheckpointPaths = - new HashSet<>(newCheckpoint.getCorrespondingFiles(logPath)); - final List newCheckpointFileList = - checkpoints.stream() + new HashSet<>(latestCompleteCheckpoint.getCorrespondingFiles(logPath)); + + final List newCheckpointFileStatuses = + listedCheckpointFileStatuses.stream() .filter(f -> newCheckpointPaths.contains(new Path(f.getPath()))) .collect(Collectors.toList()); - if (newCheckpointFileList.size() != newCheckpointPaths.size()) { - String msg = + logDebugFileStatuses("newCheckpointFileStatuses", newCheckpointFileStatuses); + + if (newCheckpointFileStatuses.size() != newCheckpointPaths.size()) { + final String msg = format( "Seems like the checkpoint is corrupted. Failed in getting the file " + "information for:\n%s\namong\n%s", newCheckpointPaths.stream() .map(Path::toString) - .collect(Collectors.toList()), - checkpoints.stream() + .collect(Collectors.joining("\n - ")), + listedCheckpointFileStatuses.stream() .map(FileStatus::getPath) .collect(Collectors.joining("\n - "))); throw new IllegalStateException(msg); } - return newCheckpointFileList; + + return newCheckpointFileStatuses; }) .orElse(Collections.emptyList()); + /////////////////////////////////////////////////// + // Step 11: Construct the LogSegment and return. // + /////////////////////////////////////////////////// + + logger.info("Successfully constructed LogSegment at version {}", newVersion); + + final long lastCommitTimestamp = + ListUtils.getLast(listedDeltaFileStatuses).getModificationTime(); + return new LogSegment( logPath, newVersion, deltasAfterCheckpoint, - newCheckpointFiles, - newCheckpointOpt.map(x -> x.version), + latestCompleteCheckpointFileStatuses, + latestCompleteCheckpointOpt.map(x -> x.version), lastCommitTimestamp); } @@ -651,23 +611,14 @@ private Optional getStartCheckpointVersion(Engine engine, Optional v }); } - /** - * Returns a [[LogSegment]] for reading `snapshotVersion` such that the segment's checkpoint - * version (if checkpoint present) is LESS THAN `maxExclusiveCheckpointVersion`. This is useful - * when trying to skip a bad checkpoint. Returns `None` when we are not able to construct such - * [[LogSegment]], for example, no checkpoint can be used but we don't have the entire history - * from version 0 to version `snapshotVersion`. - */ - private Optional getLogSegmentWithMaxExclusiveCheckpointVersion( - long snapshotVersion, long maxExclusiveCheckpointVersion) { - // TODO - return Optional.empty(); - } - - // TODO logger interface to support this across kernel-api module - private void logDebug(Supplier message) { + private void logDebugFileStatuses(String varName, List fileStatuses) { if (logger.isDebugEnabled()) { - logger.debug(message.get()); + logger.debug( + String.format( + "%s: %s", + varName, + Arrays.toString( + fileStatuses.stream().map(x -> new Path(x.getPath()).getName()).toArray()))); } } } diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/SnapshotManagerSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/SnapshotManagerSuite.scala index 5fa2cd09106..4da5388d9bd 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/SnapshotManagerSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/SnapshotManagerSuite.scala @@ -23,98 +23,36 @@ import scala.reflect.ClassTag import io.delta.kernel.data.{ColumnVector, ColumnarBatch} import io.delta.kernel.exceptions.{InvalidTableException, TableNotFoundException} import io.delta.kernel.expressions.Predicate -import io.delta.kernel.internal.checkpoints.{CheckpointInstance, SidecarFile} +import io.delta.kernel.internal.checkpoints.{CheckpointInstance, CheckpointMetaData, SidecarFile} import io.delta.kernel.internal.fs.Path import io.delta.kernel.internal.snapshot.{LogSegment, SnapshotManager} import io.delta.kernel.internal.util.{FileNames, Utils} -import io.delta.kernel.test.{BaseMockJsonHandler, BaseMockParquetHandler, MockFileSystemClientUtils, VectorTestUtils} +import io.delta.kernel.test.{BaseMockJsonHandler, BaseMockParquetHandler, MockFileSystemClientUtils, MockListFromFileSystemClient, VectorTestUtils} import io.delta.kernel.types.StructType import io.delta.kernel.utils.{CloseableIterator, FileStatus} import org.scalatest.funsuite.AnyFunSuite class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { - test("verifyDeltaVersions") { + test("verifyDeltaVersionsContiguous") { + val path = new Path("/path/to/table") // empty array - SnapshotManager.verifyDeltaVersions( - Collections.emptyList(), - Optional.empty(), - Optional.empty(), - new Path("/path/to/table")) + SnapshotManager.verifyDeltaVersionsContiguous(Collections.emptyList(), path) + // array of size 1 + SnapshotManager.verifyDeltaVersionsContiguous(Collections.singletonList(1), path) // contiguous versions - SnapshotManager.verifyDeltaVersions( - Arrays.asList(1, 2, 3), - Optional.empty(), - Optional.empty(), - new Path("/path/to/table")) - // contiguous versions with correct `expectedStartVersion` and `expectedStartVersion` - SnapshotManager.verifyDeltaVersions( - Arrays.asList(1, 2, 3), - Optional.empty(), - Optional.of(3), - new Path("/path/to/table")) - SnapshotManager.verifyDeltaVersions( - Arrays.asList(1, 2, 3), - Optional.of(1), - Optional.empty(), - new Path("/path/to/table")) - SnapshotManager.verifyDeltaVersions( - Arrays.asList(1, 2, 3), - Optional.of(1), - Optional.of(3), - new Path("/path/to/table")) - // `expectedStartVersion` or `expectedEndVersion` doesn't match - intercept[IllegalArgumentException] { - SnapshotManager.verifyDeltaVersions( - Arrays.asList(1, 2), - Optional.of(0), - Optional.empty(), - new Path("/path/to/table")) - } - intercept[IllegalArgumentException] { - SnapshotManager.verifyDeltaVersions( - Arrays.asList(1, 2), - Optional.empty(), - Optional.of(3), - new Path("/path/to/table")) - } - intercept[IllegalArgumentException] { - SnapshotManager.verifyDeltaVersions( - Collections.emptyList(), - Optional.of(0), - Optional.empty(), - new Path("/path/to/table")) - } - intercept[IllegalArgumentException] { - SnapshotManager.verifyDeltaVersions( - Collections.emptyList(), - Optional.empty(), - Optional.of(3), - new Path("/path/to/table")) - } - // non contiguous versions + SnapshotManager.verifyDeltaVersionsContiguous(Arrays.asList(1, 2, 3), path) + // non-contiguous versions intercept[InvalidTableException] { - SnapshotManager.verifyDeltaVersions( - Arrays.asList(1, 3), - Optional.empty(), - Optional.empty(), - new Path("/path/to/table")) + SnapshotManager.verifyDeltaVersionsContiguous(Arrays.asList(1, 3), path) } // duplicates in versions intercept[InvalidTableException] { - SnapshotManager.verifyDeltaVersions( - Arrays.asList(1, 2, 2, 3), - Optional.empty(), - Optional.empty(), - new Path("/path/to/table")) + SnapshotManager.verifyDeltaVersionsContiguous(Arrays.asList(1, 2, 2, 3), path) } // unsorted versions intercept[InvalidTableException] { - SnapshotManager.verifyDeltaVersions( - Arrays.asList(3, 2, 1), - Optional.empty(), - Optional.empty(), - new Path("/path/to/table")) + SnapshotManager.verifyDeltaVersionsContiguous(Arrays.asList(3, 2, 1), path) } } @@ -219,7 +157,6 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { createMockFSListFromEngine(listFromProvider(deltas ++ checkpointFiles)("/"), new MockSidecarParquetHandler(expectedSidecars), new MockSidecarJsonHandler(expectedSidecars)), - Optional.empty(), versionToLoad ) @@ -308,13 +245,12 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { */ def testExpectedError[T <: Throwable]( files: Seq[FileStatus], - startCheckpoint: Optional[java.lang.Long] = Optional.empty(), + lastCheckpointVersion: Optional[java.lang.Long] = Optional.empty(), versionToLoad: Optional[java.lang.Long] = Optional.empty(), expectedErrorMessageContains: String = "")(implicit classTag: ClassTag[T]): Unit = { val e = intercept[T] { snapshotManager.getLogSegmentForVersion( - createMockFSListFromEngine(files), - startCheckpoint, + createMockFSAndJsonEngineForLastCheckpoint(files, lastCheckpointVersion), versionToLoad ) } @@ -438,8 +374,7 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { val exMsg = intercept[TableNotFoundException] { snapshotManager.getLogSegmentForVersion( createMockFSListFromEngine(Seq.empty), - Optional.empty(), - Optional.empty() + Optional.empty() /* versionToLoad */ ) }.getMessage @@ -479,21 +414,31 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { } test("getLogSegmentForVersion: start listing from _last_checkpoint when it is provided") { - val deltas = deltaFileStatuses(0L until 25) - val checkpoints = singularCheckpointFileStatuses(Seq(10L, 20L)) - val files = deltas ++ checkpoints - def listFrom(minVersion: Long)(filePath: String): Seq[FileStatus] = { - if (filePath < FileNames.listingPrefix(logPath, minVersion)) { - throw new RuntimeException("Listing from before provided _last_checkpoint") + val deltas = deltaFileStatuses(0L to 24) + val checkpoints = singularCheckpointFileStatuses(Seq(10, 20)) + + for (lastCheckpointVersion <- Seq(10, 20)) { + val lastCheckpointFileStatus = FileStatus.of(s"$logPath/_last_checkpoint", 2, 2) + val files = deltas ++ checkpoints ++ Seq(lastCheckpointFileStatus) + + def listFrom(filePath: String): Seq[FileStatus] = { + if (filePath < FileNames.listingPrefix(logPath, lastCheckpointVersion)) { + throw new RuntimeException( + s"Listing from before the checkpoint version referenced by _last_checkpoint. " + + s"Last checkpoint version: $lastCheckpointVersion. Listing from: $filePath") + } + listFromProvider(files)(filePath) } - listFromProvider(files)(filePath) - } - for (checkpointV <- Seq(10, 20)) { + val logSegment = snapshotManager.getLogSegmentForVersion( - createMockFSListFromEngine(listFrom(checkpointV)(_)), - Optional.of(checkpointV), - Optional.empty() + mockEngine( + jsonHandler = new MockReadLastCheckpointFileJsonHandler( + lastCheckpointFileStatus.getPath, lastCheckpointVersion), + fileSystemClient = new MockListFromFileSystemClient(listFrom) + ), + Optional.empty() /* versionToLoad */ ) + checkLogSegment( logSegment, expectedVersion = 24, @@ -634,14 +579,12 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { /* ------------------- CORRUPT DELTA LOG FILE LISTINGS ------------------ */ test("getLogSegmentForVersion: corrupt listing with only checkpoint file") { - for (versionToLoad <- - Seq(Optional.empty(), Optional.of(10L)): Seq[Optional[java.lang.Long]]) { - for (startCheckpoint <- - Seq(Optional.empty(), Optional.of(10L)): Seq[Optional[java.lang.Long]]) { + Seq(Optional.empty(), Optional.of(10L)).foreach { lastCheckpointVersion => + Seq(Optional.empty(), Optional.of(10L)).foreach { versionToLoad => testExpectedError[InvalidTableException]( files = singularCheckpointFileStatuses(Seq(10L)), - startCheckpoint = startCheckpoint, - versionToLoad = versionToLoad, + lastCheckpointVersion.map(Long.box), + versionToLoad.map(Long.box), expectedErrorMessageContains = "Missing delta file for version 10" ) } @@ -652,88 +595,54 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { // checkpoint(10), 010.json, 011.json, 013.json val fileList = deltaFileStatuses(Seq(10L, 11L)) ++ deltaFileStatuses(Seq(13L)) ++ singularCheckpointFileStatuses(Seq(10L)) - testExpectedError[InvalidTableException]( - fileList, - expectedErrorMessageContains = "versions are not contiguous: ([11, 13])" - ) - testExpectedError[InvalidTableException]( - fileList, - startCheckpoint = Optional.of(10), - expectedErrorMessageContains = "versions are not contiguous: ([11, 13])" - ) - testExpectedError[InvalidTableException]( - fileList, - versionToLoad = Optional.of(13), - expectedErrorMessageContains = "versions are not contiguous: ([11, 13])" - ) + Seq(Optional.empty(), Optional.of(10L)).foreach { lastCheckpointVersion => + Seq(Optional.empty(), Optional.of(13L)).foreach { versionToLoad => + testExpectedError[InvalidTableException]( + fileList, + lastCheckpointVersion.map(Long.box), + versionToLoad.map(Long.box), + expectedErrorMessageContains = "versions are not contiguous: ([11, 13])" + ) + } + } } test("getLogSegmentForVersion: corrupt listing 000.json...009.json + checkpoint(10)") { val fileList = deltaFileStatuses((0L until 10L)) ++ singularCheckpointFileStatuses(Seq(10L)) - - /* ---------- version to load is 15 (greater than latest checkpoint/delta file) ---------- */ - testExpectedError[InvalidTableException]( - fileList, - versionToLoad = Optional.of(15), - expectedErrorMessageContains = "Missing delta file for version 10" - ) - testExpectedError[InvalidTableException]( - fileList, - startCheckpoint = Optional.of(10), - versionToLoad = Optional.of(15), - expectedErrorMessageContains = "Missing delta file for version 10" - ) - - /* ---------- versionToLoad is latest (10) ---------- */ - testExpectedError[InvalidTableException]( - fileList, - startCheckpoint = Optional.of(10), - expectedErrorMessageContains = "Missing delta file for version 10" - ) - testExpectedError[InvalidTableException]( - fileList, - expectedErrorMessageContains = "Missing delta file for version 10" - ) + Seq(Optional.empty(), Optional.of(10L)).foreach { lastCheckpointVersion => + Seq(Optional.empty(), Optional.of(15L)).foreach { versionToLoad => + testExpectedError[InvalidTableException]( + fileList, + lastCheckpointVersion.map(Long.box), + versionToLoad.map(Long.box), + expectedErrorMessageContains = "Missing delta file for version 10" + ) + } + } } - // it's weird that checkpoint(10) fails but 011.json...014.json + checkpoint(10) does not - test("getLogSegmentForVersion: corrupt listing 011.json...014.json + checkpoint(10)") { + test("getLogSegmentForVersion: corrupt listing: checkpoint(10); 11 to 14.json; no 10.json") { val fileList = singularCheckpointFileStatuses(Seq(10L)) ++ deltaFileStatuses((11L until 15L)) - /* ---------- versionToLoad is latest (14) ---------- */ - // no error - testWithSingularAndMultipartCheckpoint( - deltaVersions = (11L until 15L), - checkpointVersions = Seq(10) - ) - testWithSingularAndMultipartCheckpoint( - deltaVersions = (11L until 15L), - checkpointVersions = Seq(10), - startCheckpoint = Optional.of(10) - ) - /* ---------- versionToLoad is 10 ---------- */ - // (?) throws an error - testExpectedError[InvalidTableException]( - fileList, - versionToLoad = Optional.of(10), - expectedErrorMessageContains = "Missing delta file for version 10" - ) - testExpectedError[InvalidTableException]( - fileList, - startCheckpoint = Optional.of(10), - versionToLoad = Optional.of(10), - expectedErrorMessageContains = "Missing delta file for version 10" - ) + Seq(Optional.empty(), Optional.of(10L)).foreach { lastCheckpointVersion => + Seq(Optional.empty(), Optional.of(10L)).foreach { versionToLoad => + testExpectedError[InvalidTableException]( + fileList, + lastCheckpointVersion.map(Long.box), + versionToLoad.map(Long.box), + expectedErrorMessageContains = "Missing delta file for version 10") + } + } } test("getLogSegmentForVersion: corrupted log missing json files / no way to construct history") { testExpectedError[InvalidTableException]( deltaFileStatuses(1L until 10L), - expectedErrorMessageContains = "missing log file for version 0" + expectedErrorMessageContains = "Cannot compute snapshot. Missing delta file version 0." ) testExpectedError[InvalidTableException]( deltaFileStatuses(15L until 25L) ++ singularCheckpointFileStatuses(Seq(20L)), versionToLoad = Optional.of(17), - expectedErrorMessageContains = "missing log file for version 0" + expectedErrorMessageContains = "Cannot compute snapshot. Missing delta file version 0." ) testExpectedError[InvalidTableException]( deltaFileStatuses((0L until 5L) ++ (6L until 9L)), @@ -746,9 +655,7 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { val deltas = deltaFileStatuses(10L to 13L) testExpectedError[InvalidTableException]( corruptedCheckpointStatuses ++ deltas, - Optional.empty(), - Optional.empty(), - expectedErrorMessageContains = "missing log file for version 0" + expectedErrorMessageContains = "Cannot compute snapshot. Missing delta file version 0." ) } @@ -780,44 +687,44 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { ) } - // TODO recover from missing checkpoint (getLogSegmentWithMaxExclusiveCheckpointVersion) test("getLogSegmentForVersion: corrupt _last_checkpoint refers to in range version " + "but no valid checkpoint") { + // _last_checkpoint refers to a v1 checkpoint at version 20 that is missing testExpectedError[RuntimeException]( deltaFileStatuses(0L until 25L) ++ singularCheckpointFileStatuses(Seq(10L)), - startCheckpoint = Optional.of(20), - expectedErrorMessageContains = "Checkpoint file to load version: 20 is missing." + lastCheckpointVersion = Optional.of(20), + expectedErrorMessageContains = "Missing checkpoint at version 20" ) - // _last_checkpoint refers to incomplete multi-part checkpoint + // _last_checkpoint refers to incomplete multi-part checkpoint at version 20 that is missing val corruptedCheckpointStatuses = FileNames.checkpointFileWithParts(logPath, 20, 5).asScala .map(p => FileStatus.of(p.toString, 10, 10)) .take(4) testExpectedError[RuntimeException]( files = corruptedCheckpointStatuses ++ deltaFileStatuses(10L to 20L) ++ singularCheckpointFileStatuses(Seq(10L)), - startCheckpoint = Optional.of(20), - expectedErrorMessageContains = "Checkpoint file to load version: 20 is missing." + lastCheckpointVersion = Optional.of(20), + expectedErrorMessageContains = "Missing checkpoint at version 20" ) } test("getLogSegmentForVersion: corrupted incomplete multi-part checkpoint with no" + "_last_checkpoint or a valid _last_checkpoint provided") { val cases: Seq[(Long, Seq[Long], Seq[Long], Optional[java.lang.Long])] = Seq( - /* (corruptedCheckpointVersion, validCheckpointVersions, deltaVersions, startCheckpoint) */ + /* (corruptedCheckpointVersion, validCheckpointVersions, deltaVersions, lastCheckpointV) */ (20, Seq(10), (10L to 20L), Optional.empty()), (20, Seq(10), (10L to 20L), Optional.of(10)), (10, Seq.empty, (0L to 10L), Optional.empty()) ) - cases.foreach { case (corruptedVersion, validVersions, deltaVersions, startCheckpoint) => + cases.foreach { case (corruptedVersion, validVersions, deltaVersions, lastCheckpointVersion) => val corruptedCheckpoint = FileNames.checkpointFileWithParts(logPath, corruptedVersion, 5) .asScala .map(p => FileStatus.of(p.toString, 10, 10)) .take(4) val checkpoints = singularCheckpointFileStatuses(validVersions) val deltas = deltaFileStatuses(deltaVersions) + val allFiles = deltas ++ corruptedCheckpoint ++ checkpoints val logSegment = snapshotManager.getLogSegmentForVersion( - createMockFSListFromEngine(deltas ++ corruptedCheckpoint ++ checkpoints), - Optional.empty(), + createMockFSAndJsonEngineForLastCheckpoint(allFiles, lastCheckpointVersion), Optional.empty() ) val checkpointVersion = validVersions.sorted.lastOption @@ -836,28 +743,13 @@ class SnapshotManagerSuite extends AnyFunSuite with MockFileSystemClientUtils { test("getLogSegmentForVersion: corrupt _last_checkpoint with empty delta log") { val exMsg = intercept[InvalidTableException] { snapshotManager.getLogSegmentForVersion( - createMockFSListFromEngine(Seq.empty), - Optional.of(1), + createMockFSAndJsonEngineForLastCheckpoint(Seq.empty, Optional.of(1)), Optional.empty() ) }.getMessage assert(exMsg.contains("Missing checkpoint at version 1")) } - - test("getLogSegmentForVersion: startCheckpoint is greater than versionToLoad") { - val exMsg = intercept[IllegalArgumentException] { - snapshotManager.getLogSegmentForVersion( - createMockFSListFromEngine( - singularCheckpointFileStatuses(Seq(10)) ++ deltaFileStatuses(10L until 15L) - ), - Optional.of(10), // startCheckpoint - Optional.of(7) // versionToLoad - ) - }.getMessage - - assert(exMsg.contains("endVersion=7 provided is less than startVersion=10")) - } } trait SidecarIteratorProvider extends VectorTestUtils { @@ -897,3 +789,32 @@ class MockSidecarJsonHandler(sidecars: Seq[FileStatus]) predicate: Optional[Predicate]): CloseableIterator[ColumnarBatch] = singletonSidecarIterator(sidecars) } + +class MockReadLastCheckpointFileJsonHandler( + lastCheckpointPath: String, + lastCheckpointVersion: Long) + extends BaseMockJsonHandler with VectorTestUtils { + override def readJsonFiles( + fileIter: CloseableIterator[FileStatus], + physicalSchema: StructType, + predicate: Optional[Predicate]): CloseableIterator[ColumnarBatch] = { + assert(fileIter.hasNext) + assert(fileIter.next.getPath == lastCheckpointPath) + + Utils.singletonCloseableIterator( + new ColumnarBatch { + override def getSchema: StructType = CheckpointMetaData.READ_SCHEMA + + override def getColumnVector(ordinal: Int): ColumnVector = { + ordinal match { + case 0 => longVector(lastCheckpointVersion) /* version */ + case 1 => longVector(100) /* size */ + case 2 => longVector(1) /* parts */ + } + } + + override def getSize: Int = 1 + } + ) + } +} diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/test/MockFileSystemClientUtils.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/test/MockFileSystemClientUtils.scala index 0e8ed2c56b9..a40e55093ae 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/test/MockFileSystemClientUtils.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/test/MockFileSystemClientUtils.scala @@ -15,7 +15,7 @@ */ package io.delta.kernel.test -import java.util.UUID +import java.util.{Optional, UUID} import io.delta.kernel.engine._ import io.delta.kernel.internal.fs.Path @@ -25,6 +25,8 @@ import io.delta.kernel.utils.{CloseableIterator, FileStatus} import scala.collection.JavaConverters._ +import io.delta.kernel.internal.MockReadLastCheckpointFileJsonHandler + /** * This is an extension to [[BaseMockFileSystemClient]] containing specific mock implementations * [[FileSystemClient]] which are shared across multiple test suite. @@ -107,6 +109,21 @@ trait MockFileSystemClientUtils extends MockEngineUtils { jsonHandler = jsonHandler) } + def createMockFSAndJsonEngineForLastCheckpoint( + contents: Seq[FileStatus], lastCheckpointVersion: Optional[java.lang.Long]): Engine = { + mockEngine( + fileSystemClient = new MockListFromFileSystemClient(listFromProvider(contents)), + jsonHandler = if (lastCheckpointVersion.isPresent) { + new MockReadLastCheckpointFileJsonHandler( + s"$logPath/_last_checkpoint", + lastCheckpointVersion.get() + ) + } else { + null + } + ) + } + /** * Create a mock [[Engine]] to mock the [[FileSystemClient.listFrom]] calls using * the given contents. The contents are filtered depending upon the list from path prefix. From ee98a22211537ad663518ed9bd57fbfbd203dd39 Mon Sep 17 00:00:00 2001 From: Scott Sandre Date: Wed, 29 Jan 2025 15:32:35 -0800 Subject: [PATCH 23/28] [Kernel] [Refactor] Remove superfluous engine param from Snapshot APIs (#4101) #### Which Delta project/connector is this regarding? - [ ] Spark - [ ] Standalone - [ ] Flink - [X] Kernel - [ ] Other (fill in here) ## Description Remove superfluous engine param from Snapshot APIs ## How was this patch tested? Refactor only. ## Does this PR introduce _any_ user-facing changes? Removes param from public API. --- .../internal/KernelDeltaLogDelegator.scala | 2 +- .../flink/internal/KernelSnapshotWrapper.java | 4 +-- .../examples/MultiThreadedTableReader.java | 5 ++- .../examples/SingleThreadedTableReader.java | 5 ++- .../main/java/io/delta/kernel/Snapshot.java | 12 +++---- .../kernel/internal/ScanBuilderImpl.java | 5 +-- .../delta/kernel/internal/SnapshotImpl.java | 17 +++++----- .../internal/TransactionBuilderImpl.java | 2 +- .../kernel/internal/TransactionImpl.java | 6 ++-- .../internal/replay/ConflictChecker.java | 10 ++---- .../internal/snapshot/SnapshotManager.java | 8 ++--- .../internal/util/InCommitTimestampUtils.java | 2 +- .../io/delta/kernel/utils/PartitionUtils.java | 6 ++-- .../BenchmarkParallelCheckpointReading.java | 2 +- .../ActiveAddFilesLogReplayMetricsSuite.scala | 2 +- .../defaults/CheckpointV2ReadSuite.scala | 4 +-- .../defaults/DeltaTableReadsSuite.scala | 4 +-- .../defaults/DeltaTableWritesSuite.scala | 18 +++++----- .../kernel/defaults/DomainMetadataSuite.scala | 4 +-- .../LogReplayEngineMetricsSuite.scala | 10 +++--- .../kernel/defaults/LogReplaySuite.scala | 34 +++++++++---------- .../kernel/defaults/RowTrackingSuite.scala | 10 ++---- .../io/delta/kernel/defaults/ScanSuite.scala | 32 +++++++---------- .../delta/kernel/defaults/SnapshotSuite.scala | 2 +- .../kernel/defaults/utils/TestUtils.scala | 19 +++++------ 25 files changed, 97 insertions(+), 128 deletions(-) diff --git a/connectors/flink/src/main/scala/io/delta/flink/internal/KernelDeltaLogDelegator.scala b/connectors/flink/src/main/scala/io/delta/flink/internal/KernelDeltaLogDelegator.scala index 87680e57b48..52d0cb39464 100644 --- a/connectors/flink/src/main/scala/io/delta/flink/internal/KernelDeltaLogDelegator.scala +++ b/connectors/flink/src/main/scala/io/delta/flink/internal/KernelDeltaLogDelegator.scala @@ -83,7 +83,7 @@ class KernelDeltaLogDelegator( kernelSnapshotWrapper, hadoopConf, logPath, - kernelSnapshot.getVersion(engine), // note: engine isn't used + kernelSnapshot.getVersion(), this, standaloneDeltaLog )) diff --git a/connectors/flink/src/main/scala/io/delta/flink/internal/KernelSnapshotWrapper.java b/connectors/flink/src/main/scala/io/delta/flink/internal/KernelSnapshotWrapper.java index ba1342132bd..9cd4ca6bfb2 100644 --- a/connectors/flink/src/main/scala/io/delta/flink/internal/KernelSnapshotWrapper.java +++ b/connectors/flink/src/main/scala/io/delta/flink/internal/KernelSnapshotWrapper.java @@ -73,9 +73,7 @@ public Metadata getMetadata() { */ @Override public long getVersion() { - // WARNING: getVersion in SnapshotImpl currently doesn't use the engine so we can - // pass null, but if this changes this code could break - return kernelSnapshot.getVersion(null); + return kernelSnapshot.getVersion(); } /** diff --git a/kernel/examples/kernel-examples/src/main/java/io/delta/kernel/examples/MultiThreadedTableReader.java b/kernel/examples/kernel-examples/src/main/java/io/delta/kernel/examples/MultiThreadedTableReader.java index ab43d98ff66..e5eefd668b0 100644 --- a/kernel/examples/kernel-examples/src/main/java/io/delta/kernel/examples/MultiThreadedTableReader.java +++ b/kernel/examples/kernel-examples/src/main/java/io/delta/kernel/examples/MultiThreadedTableReader.java @@ -83,10 +83,9 @@ public int show(int limit, Optional> columnsOpt, Optional> columnsOpt, OptionalThe partition column names are returned in the order they are defined in the Delta table * schema. If the table does not define any partition columns, this method returns an empty list. * - * @param engine {@link Engine} instance to use in Delta Kernel. * @return a list of partition column names, or an empty list if the table is not partitioned. */ - List getPartitionColumnNames(Engine engine); + List getPartitionColumnNames(); /** * Get the timestamp (in milliseconds since the Unix epoch) of the latest commit in this snapshot. @@ -59,16 +57,14 @@ public interface Snapshot { /** * Get the schema of the table at this snapshot. * - * @param engine {@link Engine} instance to use in Delta Kernel. * @return Schema of the Delta table at this snapshot. */ - StructType getSchema(Engine engine); + StructType getSchema(); /** * Create a scan builder to construct a {@link Scan} to read data from this snapshot. * - * @param engine {@link Engine} instance to use in Delta Kernel. * @return an instance of {@link ScanBuilder} */ - ScanBuilder getScanBuilder(Engine engine); + ScanBuilder getScanBuilder(); } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/ScanBuilderImpl.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/ScanBuilderImpl.java index 9ea79542462..4aa5b1c1e2b 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/ScanBuilderImpl.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/ScanBuilderImpl.java @@ -35,7 +35,6 @@ public class ScanBuilderImpl implements ScanBuilder { private final Metadata metadata; private final StructType snapshotSchema; private final LogReplay logReplay; - private final Engine engine; private StructType readSchema; private Optional predicate; @@ -45,14 +44,12 @@ public ScanBuilderImpl( Protocol protocol, Metadata metadata, StructType snapshotSchema, - LogReplay logReplay, - Engine engine) { + LogReplay logReplay) { this.dataPath = dataPath; this.protocol = protocol; this.metadata = metadata; this.snapshotSchema = snapshotSchema; this.logReplay = logReplay; - this.engine = engine; this.readSchema = snapshotSchema; this.predicate = Optional.empty(); } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/SnapshotImpl.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/SnapshotImpl.java index 01e3fe45167..56bc6f31160 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/SnapshotImpl.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/SnapshotImpl.java @@ -73,10 +73,15 @@ public SnapshotImpl( ///////////////// @Override - public long getVersion(Engine engine) { + public long getVersion() { return version; } + @Override + public List getPartitionColumnNames() { + return VectorUtils.toJavaList(getMetadata().getPartitionColumns()); + } + /** * Get the timestamp (in milliseconds since the Unix epoch) of the latest commit in this Snapshot. * If the table does not yet exist (i.e. this Snapshot is being used to create the new table), @@ -107,14 +112,14 @@ public long getTimestamp(Engine engine) { } @Override - public StructType getSchema(Engine engine) { + public StructType getSchema() { return getMetadata().getSchema(); } @Override - public ScanBuilder getScanBuilder(Engine engine) { + public ScanBuilder getScanBuilder() { // TODO when we add ScanReport we will pass the SnapshotReport downstream here - return new ScanBuilderImpl(dataPath, protocol, metadata, getSchema(engine), logReplay, engine); + return new ScanBuilderImpl(dataPath, protocol, metadata, getSchema(), logReplay); } /////////////////// @@ -133,10 +138,6 @@ public Protocol getProtocol() { return protocol; } - public List getPartitionColumnNames(Engine engine) { - return VectorUtils.toJavaList(getMetadata().getPartitionColumns()); - } - public SnapshotReport getSnapshotReport() { return snapshotReport; } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionBuilderImpl.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionBuilderImpl.java index 5b2e9739d7f..6639a097381 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionBuilderImpl.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionBuilderImpl.java @@ -128,7 +128,7 @@ public Transaction build(Engine engine) { new InitialSnapshot(table.getDataPath(), logReplay, metadata, protocol, snapshotContext); } - boolean isNewTable = snapshot.getVersion(engine) < 0; + boolean isNewTable = snapshot.getVersion() < 0; validate(engine, snapshot, isNewTable); boolean shouldUpdateMetadata = false; diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionImpl.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionImpl.java index 3f9d7706f81..44b0b49ccd9 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionImpl.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/TransactionImpl.java @@ -114,7 +114,7 @@ public List getPartitionColumns(Engine engine) { @Override public StructType getSchema(Engine engine) { - return readSnapshot.getSchema(engine); + return readSnapshot.getSchema(); } public Optional getSetTxnOpt() { @@ -166,11 +166,11 @@ public TransactionCommitResult commit(Engine engine, CloseableIterable data private TransactionCommitResult commitWithRetry( Engine engine, CloseableIterable dataActions, TransactionMetrics transactionMetrics) { try { - long commitAsVersion = readSnapshot.getVersion(engine) + 1; + long commitAsVersion = readSnapshot.getVersion() + 1; // Generate the commit action with the inCommitTimestamp if ICT is enabled. CommitInfo attemptCommitInfo = generateCommitAction(engine); updateMetadataWithICTIfRequired( - engine, attemptCommitInfo.getInCommitTimestamp(), readSnapshot.getVersion(engine)); + engine, attemptCommitInfo.getInCommitTimestamp(), readSnapshot.getVersion()); // If row tracking is supported, assign base row IDs and default row commit versions to any // AddFile actions that do not yet have them. If the row ID high watermark changes, emit a diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/ConflictChecker.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/ConflictChecker.java index 828da78d77b..42fa784a7ce 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/ConflictChecker.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/replay/ConflictChecker.java @@ -166,8 +166,7 @@ public TransactionRebaseState resolveConflicts(Engine engine) throws ConcurrentW // against the winning transactions return new TransactionRebaseState( lastWinningVersion, - getLastCommitTimestamp( - engine, lastWinningVersion, lastWinningTxn, winningCommitInfoOpt.get()), + getLastCommitTimestamp(lastWinningVersion, lastWinningTxn, winningCommitInfoOpt.get()), updatedDataActions, updatedDomainMetadatas); } @@ -343,8 +342,7 @@ private void handleTxn(ColumnVector txnVector) { } private List getWinningCommitFiles(Engine engine) { - String firstWinningCommitFile = - deltaFile(snapshot.getLogPath(), snapshot.getVersion(engine) + 1); + String firstWinningCommitFile = deltaFile(snapshot.getLogPath(), snapshot.getVersion() + 1); try (CloseableIterator files = wrapEngineExceptionThrowsIO( @@ -374,18 +372,16 @@ private List getWinningCommitFiles(Engine engine) { * latest winning transaction commit file. For non-ICT enabled tables, this is the modification * time of the latest winning transaction commit file. * - * @param engine {@link Engine} instance to use * @param lastWinningVersion last winning version of the table * @param lastWinningTxn the last winning transaction commit file * @param winningCommitInfoOpt winning commit info * @return last commit timestamp of the table */ private long getLastCommitTimestamp( - Engine engine, long lastWinningVersion, FileStatus lastWinningTxn, Optional winningCommitInfoOpt) { - if (snapshot.getVersion(engine) == -1 + if (snapshot.getVersion() == -1 || !IN_COMMIT_TIMESTAMPS_ENABLED.fromMetadata(snapshot.getMetadata())) { return lastWinningTxn.getModificationTime(); } else { diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java index 4f305ad9e0f..ed3f228389d 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java @@ -161,10 +161,7 @@ public void checkpoint(Engine engine, Clock clock, long version) // Check if writing to the given table protocol version/features is supported in Kernel validateWriteSupportedTable( - snapshot.getProtocol(), - snapshot.getMetadata(), - snapshot.getSchema(engine), - tablePath.toString()); + snapshot.getProtocol(), snapshot.getMetadata(), snapshot.getSchema(), tablePath.toString()); Path checkpointPath = FileNames.checkpointFileSingular(logPath, version); @@ -288,8 +285,7 @@ private SnapshotImpl createSnapshot( startingFromStr); final SnapshotHint hint = - new SnapshotHint( - snapshot.getVersion(engine), snapshot.getProtocol(), snapshot.getMetadata()); + new SnapshotHint(snapshot.getVersion(), snapshot.getProtocol(), snapshot.getMetadata()); registerHint(hint); diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/InCommitTimestampUtils.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/InCommitTimestampUtils.java index 1871988eb22..c4d8255acd6 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/InCommitTimestampUtils.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/util/InCommitTimestampUtils.java @@ -71,7 +71,7 @@ private static boolean didCurrentTransactionEnableICT( boolean isICTCurrentlyEnabled = IN_COMMIT_TIMESTAMPS_ENABLED.fromMetadata(currentTransactionMetadata); boolean wasICTEnabledInReadSnapshot = - readSnapshot.getVersion(engine) != -1 + readSnapshot.getVersion() != -1 && IN_COMMIT_TIMESTAMPS_ENABLED.fromMetadata(readSnapshot.getMetadata()); return isICTCurrentlyEnabled && !wasICTEnabledInReadSnapshot; } diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/utils/PartitionUtils.java b/kernel/kernel-api/src/main/java/io/delta/kernel/utils/PartitionUtils.java index 9ad35db2a84..33fa2af6d68 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/utils/PartitionUtils.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/utils/PartitionUtils.java @@ -50,14 +50,12 @@ public static boolean partitionExists( requireNonNull(snapshot, "snapshot is null"); requireNonNull(partitionPredicate, "partitionPredicate is null"); - final Set snapshotPartColNames = - new HashSet<>(snapshot.getPartitionColumnNames(engine)); + final Set snapshotPartColNames = new HashSet<>(snapshot.getPartitionColumnNames()); io.delta.kernel.internal.util.PartitionUtils.validatePredicateOnlyOnPartitionColumns( partitionPredicate, snapshotPartColNames); - final Scan scan = - snapshot.getScanBuilder(engine).withFilter(engine, partitionPredicate).build(); + final Scan scan = snapshot.getScanBuilder().withFilter(engine, partitionPredicate).build(); try (CloseableIterator columnarBatchIter = scan.getScanFiles(engine)) { while (columnarBatchIter.hasNext()) { diff --git a/kernel/kernel-defaults/src/test/java/io/delta/kernel/defaults/benchmarks/BenchmarkParallelCheckpointReading.java b/kernel/kernel-defaults/src/test/java/io/delta/kernel/defaults/benchmarks/BenchmarkParallelCheckpointReading.java index 6d456c5d5e0..f84c2caf3fe 100644 --- a/kernel/kernel-defaults/src/test/java/io/delta/kernel/defaults/benchmarks/BenchmarkParallelCheckpointReading.java +++ b/kernel/kernel-defaults/src/test/java/io/delta/kernel/defaults/benchmarks/BenchmarkParallelCheckpointReading.java @@ -114,7 +114,7 @@ public void benchmark(BenchmarkData benchmarkData, Blackhole blackhole) throws E Table table = Table.forPath(engine, testTablePath); Snapshot snapshot = table.getLatestSnapshot(engine); - ScanBuilder scanBuilder = snapshot.getScanBuilder(engine); + ScanBuilder scanBuilder = snapshot.getScanBuilder(); Scan scan = scanBuilder.build(); // Scan state is not used, but get it so that we simulate the real use case. diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ActiveAddFilesLogReplayMetricsSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ActiveAddFilesLogReplayMetricsSuite.scala index b9f7eb4d1b2..6acc3ae4941 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ActiveAddFilesLogReplayMetricsSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ActiveAddFilesLogReplayMetricsSuite.scala @@ -142,7 +142,7 @@ class ActiveAddFilesLogReplayMetricsSuite extends AnyFunSuite with TestUtils { val scanFileIter = Table.forPath(engine, tablePath) .getLatestSnapshot(engine) - .getScanBuilder(engine) + .getScanBuilder() .build() .getScanFiles(engine) diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/CheckpointV2ReadSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/CheckpointV2ReadSuite.scala index 8804c9ddc3c..a1fa03bd130 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/CheckpointV2ReadSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/CheckpointV2ReadSuite.scala @@ -82,7 +82,7 @@ class CheckpointV2ReadSuite extends AnyFunSuite with TestUtils with ExpressionTe snapshotFromSpark.protocol.readerFeatureNames) assert(snapshotImpl.getProtocol.getWriterFeatures.asScala.toSet == snapshotFromSpark.protocol.writerFeatureNames) - assert(snapshot.getVersion(defaultEngine) == snapshotFromSpark.version) + assert(snapshot.getVersion() == snapshotFromSpark.version) // Validate that snapshot read from most recent checkpoint. For most cases, given a checkpoint // interval of 2, this will be the most recent even version. @@ -97,7 +97,7 @@ class CheckpointV2ReadSuite extends AnyFunSuite with TestUtils with ExpressionTe // Validate AddFiles from sidecars found against Spark connector. - val scan = snapshot.getScanBuilder(defaultEngine).build() + val scan = snapshot.getScanBuilder().build() val foundFiles = collectScanFileRows(scan).map(InternalScanFileUtils.getAddFileStatus).map( _.getPath.split('/').last).toSet diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableReadsSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableReadsSuite.scala index 6c99a8664ef..55b8aa99724 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableReadsSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableReadsSuite.scala @@ -282,7 +282,7 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { // for now we don't support timestamp type partition columns so remove from read columns val readCols = Table.forPath(defaultEngine, path).getLatestSnapshot(defaultEngine) - .getSchema(defaultEngine) + .getSchema() .withoutField("as_timestamp") .fields() .asScala @@ -680,7 +680,7 @@ class DeltaTableReadsSuite extends AnyFunSuite with TestUtils { test("table protocol version greater than reader protocol version") { val e = intercept[Exception] { latestSnapshot(goldenTablePath("deltalog-invalid-protocol-version")) - .getScanBuilder(defaultEngine) + .getScanBuilder() .build() } assert(e.getMessage.contains("Unsupported Delta protocol reader version")) diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWritesSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWritesSuite.scala index 597a66b1b72..5907fb32108 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWritesSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaTableWritesSuite.scala @@ -1060,7 +1060,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa .commit(engine, emptyIterable()) val table = Table.forPath(engine, tablePath) - assert(table.getLatestSnapshot(engine).getSchema(engine).equals(testSchema)) + assert(table.getLatestSnapshot(engine).getSchema().equals(testSchema)) } } @@ -1112,7 +1112,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa tableProperties = Map(TableConfig.COLUMN_MAPPING_MODE.getKey -> "id")) .commit(engine, emptyIterable()) - val structType = table.getLatestSnapshot(engine).getSchema(engine) + val structType = table.getLatestSnapshot(engine).getSchema() assertColumnMapping(structType.get("a"), 1) assertColumnMapping(structType.get("b"), 2) @@ -1140,7 +1140,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa tableProperties = Map(TableConfig.COLUMN_MAPPING_MODE.getKey -> "name")) .commit(engine, emptyIterable()) - val structType = table.getLatestSnapshot(engine).getSchema(engine) + val structType = table.getLatestSnapshot(engine).getSchema() assertColumnMapping(structType.get("a"), 1) assertColumnMapping(structType.get("b"), 2) @@ -1167,7 +1167,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa createTxn(engine, tablePath, isNewTable = true, schema, partCols = Seq.empty) .commit(engine, emptyIterable()) - val structType = table.getLatestSnapshot(engine).getSchema(engine) + val structType = table.getLatestSnapshot(engine).getSchema() assert(structType.equals(schema)) val ex = intercept[IllegalArgumentException] { @@ -1203,7 +1203,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa tableProperties = Map(TableConfig.COLUMN_MAPPING_MODE.getKey -> "name")) .commit(engine, emptyIterable()) - val structType = table.getLatestSnapshot(engine).getSchema(engine) + val structType = table.getLatestSnapshot(engine).getSchema() assertColumnMapping(structType.get("a"), 1) assertColumnMapping(structType.get("b"), 2) } @@ -1220,7 +1220,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa tableProperties = Map(TableConfig.COLUMN_MAPPING_MODE.getKey -> "id")) .commit(engine, emptyIterable()) - val structType = table.getLatestSnapshot(engine).getSchema(engine) + val structType = table.getLatestSnapshot(engine).getSchema() assertColumnMapping(structType.get("a"), 1) assertColumnMapping(structType.get("b"), 2) } @@ -1236,7 +1236,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa createTxn(engine, tablePath, isNewTable = true, schema, partCols = Seq.empty) .commit(engine, emptyIterable()) - val structType = table.getLatestSnapshot(engine).getSchema(engine) + val structType = table.getLatestSnapshot(engine).getSchema() assert(structType.equals(schema)) table.createTransactionBuilder(engine, testEngineInfo, Operation.WRITE) @@ -1246,7 +1246,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa .build(engine) .commit(engine, emptyIterable()) - val updatedSchema = table.getLatestSnapshot(engine).getSchema(engine) + val updatedSchema = table.getLatestSnapshot(engine).getSchema() assertColumnMapping(updatedSchema.get("a"), 1, "a") assertColumnMapping(updatedSchema.get("b"), 2, "b") } @@ -1268,7 +1268,7 @@ class DeltaTableWritesSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBa TableConfig.ICEBERG_COMPAT_V2_ENABLED.getKey -> "true")) .commit(engine, emptyIterable()) - val structType = table.getLatestSnapshot(engine).getSchema(engine) + val structType = table.getLatestSnapshot(engine).getSchema() assertColumnMapping(structType.get("a"), 1) assertColumnMapping(structType.get("b"), 2) val innerStruct = structType.get("b").getDataType.asInstanceOf[StructType] diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DomainMetadataSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DomainMetadataSuite.scala index 9d362b6ba84..49c7b75f2ff 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DomainMetadataSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DomainMetadataSuite.scala @@ -291,7 +291,7 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase } // Checkpoint the table - val latestVersion = table.getLatestSnapshot(engine).getVersion(engine) + val latestVersion = table.getLatestSnapshot(engine).getVersion() table.checkpoint(engine, latestVersion) // Verify that only the latest domain metadata is persisted in the checkpoint @@ -508,7 +508,7 @@ class DomainMetadataSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase // Checkpoint the table so domain metadata is distributed to both checkpoint and log files val table = Table.forPath(engine, tablePath) - val latestVersion = table.getLatestSnapshot(engine).getVersion(engine) + val latestVersion = table.getLatestSnapshot(engine).getVersion() table.checkpoint(engine, latestVersion) // Manually commit two domain metadata actions diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/LogReplayEngineMetricsSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/LogReplayEngineMetricsSuite.scala index cd43dc11838..81ac5202173 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/LogReplayEngineMetricsSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/LogReplayEngineMetricsSuite.scala @@ -76,7 +76,7 @@ class LogReplayEngineMetricsSuite extends QueryTest expParquetVersionsRead: Seq[Long], expParquetReadSetSizes: Seq[Long] = Nil): Unit = { engine.resetMetrics() - table.getLatestSnapshot(engine).getSchema(engine) + table.getLatestSnapshot(engine).getSchema() assertMetrics( engine, @@ -93,7 +93,7 @@ class LogReplayEngineMetricsSuite extends QueryTest expParquetReadSetSizes: Seq[Long], expLastCheckpointReadCalls: Option[Int] = None): Unit = { engine.resetMetrics() - val scan = table.getLatestSnapshot(engine).getScanBuilder(engine).build() + val scan = table.getLatestSnapshot(engine).getScanBuilder().build() // get all scan files and iterate through them to trigger the metrics collection val scanFiles = scan.getScanFiles(engine) while (scanFiles.hasNext) scanFiles.next() @@ -206,7 +206,7 @@ class LogReplayEngineMetricsSuite extends QueryTest val table = Table.forPath(tc, path) - table.getLatestSnapshot(tc).getSchema(tc) + table.getLatestSnapshot(tc).getSchema() // A hint is now saved at v14 @@ -222,7 +222,7 @@ class LogReplayEngineMetricsSuite extends QueryTest val table = Table.forPath(tc, path) - table.getLatestSnapshot(tc).getSchema(tc) + table.getLatestSnapshot(tc).getSchema() // A hint is now saved at v14 @@ -253,7 +253,7 @@ class LogReplayEngineMetricsSuite extends QueryTest val table = Table.forPath(tc, path) - table.getLatestSnapshot(tc).getSchema(tc) + table.getLatestSnapshot(tc).getSchema() // A hint is now saved at v3 diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/LogReplaySuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/LogReplaySuite.scala index 16722c955c8..b602396a141 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/LogReplaySuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/LogReplaySuite.scala @@ -87,7 +87,7 @@ class LogReplaySuite extends AnyFunSuite with TestUtils { test(s"missing $action should fail") { val path = goldenTablePath(s"deltalog-state-reconstruction-without-$action") val e = intercept[IllegalStateException] { - latestSnapshot(path).getSchema(defaultEngine) + latestSnapshot(path).getSchema() } assert(e.getMessage.contains(s"No $action found")) } @@ -100,7 +100,7 @@ class LogReplaySuite extends AnyFunSuite with TestUtils { test(s"missing $action should fail missing from checkpoint") { val path = goldenTablePath(s"deltalog-state-reconstruction-from-checkpoint-missing-$action") val e = intercept[IllegalStateException] { - latestSnapshot(path).getSchema(defaultEngine) + latestSnapshot(path).getSchema() } assert(e.getMessage.contains(s"No $action found")) } @@ -109,7 +109,7 @@ class LogReplaySuite extends AnyFunSuite with TestUtils { test("fetches the latest protocol and metadata") { val path = goldenTablePath("log-replay-latest-metadata-protocol") val snapshot = latestSnapshot(path) - val scanStateRow = snapshot.getScanBuilder(defaultEngine).build() + val scanStateRow = snapshot.getScanBuilder().build() .getScanState(defaultEngine) // schema is updated @@ -126,8 +126,8 @@ class LogReplaySuite extends AnyFunSuite with TestUtils { test("standalone DeltaLogSuite: 'checkpoint'") { val path = goldenTablePath("checkpoint") val snapshot = latestSnapshot(path) - assert(snapshot.getVersion(defaultEngine) == 14) - val scan = snapshot.getScanBuilder(defaultEngine).build() + assert(snapshot.getVersion() == 14) + val scan = snapshot.getScanBuilder().build() assert(collectScanFileRows(scan).length == 1) } @@ -144,9 +144,9 @@ class LogReplaySuite extends AnyFunSuite with TestUtils { expectedFiles: Array[File], expectedVersion: Int): Unit = { val snapshot = latestSnapshot(tablePath) - assert(snapshot.getVersion(defaultEngine) == expectedVersion) + assert(snapshot.getVersion() == expectedVersion) val scanFileRows = collectScanFileRows( - snapshot.getScanBuilder(defaultEngine).build()) + snapshot.getScanBuilder().build()) assert(scanFileRows.length == expectedFiles.length) val scanFilePaths = scanFileRows .map(InternalScanFileUtils.getAddFileStatus) @@ -199,9 +199,9 @@ class LogReplaySuite extends AnyFunSuite with TestUtils { // Repartition into 2 files withGoldenTable("snapshot-repartitioned") { tablePath => val snapshot = latestSnapshot(tablePath) - assert(snapshot.getVersion(defaultEngine) == 5) + assert(snapshot.getVersion() == 5) val scanFileRows = collectScanFileRows( - snapshot.getScanBuilder(defaultEngine).build()) + snapshot.getScanBuilder().build()) assert(scanFileRows.length == 2) } @@ -216,7 +216,7 @@ class LogReplaySuite extends AnyFunSuite with TestUtils { test("DV cases with same path different DV keys") { val snapshot = latestSnapshot(goldenTablePath("log-replay-dv-key-cases")) val scanFileRows = collectScanFileRows( - snapshot.getScanBuilder(defaultEngine).build() + snapshot.getScanBuilder().build() ) assert(scanFileRows.length == 1) // there should only be 1 add file val dv = InternalScanFileUtils.getDeletionVectorDescriptorFromRow(scanFileRows.head) @@ -227,14 +227,14 @@ class LogReplaySuite extends AnyFunSuite with TestUtils { withGoldenTable("log-replay-special-characters-a") { path => val snapshot = latestSnapshot(path) val scanFileRows = collectScanFileRows( - snapshot.getScanBuilder(defaultEngine).build() + snapshot.getScanBuilder().build() ) assert(scanFileRows.isEmpty) } withGoldenTable("log-replay-special-characters-b") { path => val snapshot = latestSnapshot(path) val scanFileRows = collectScanFileRows( - snapshot.getScanBuilder(defaultEngine).build() + snapshot.getScanBuilder().build() ) assert(scanFileRows.length == 1) val addFileStatus = InternalScanFileUtils.getAddFileStatus(scanFileRows.head) @@ -247,8 +247,8 @@ class LogReplaySuite extends AnyFunSuite with TestUtils { ignore("path should be canonicalized - normal characters") { Seq("canonicalized-paths-normal-a", "canonicalized-paths-normal-b").foreach { path => val snapshot = latestSnapshot(goldenTablePath(path)) - assert(snapshot.getVersion(defaultEngine) == 1) - val scanFileRows = collectScanFileRows(snapshot.getScanBuilder(defaultEngine).build()) + assert(snapshot.getVersion() == 1) + val scanFileRows = collectScanFileRows(snapshot.getScanBuilder().build()) assert(scanFileRows.isEmpty) } } @@ -256,8 +256,8 @@ class LogReplaySuite extends AnyFunSuite with TestUtils { ignore("path should be canonicalized - special characters") { Seq("canonicalized-paths-special-a", "canonicalized-paths-special-b").foreach { path => val snapshot = latestSnapshot(goldenTablePath(path)) - assert(snapshot.getVersion(defaultEngine) == 1) - val scanFileRows = collectScanFileRows(snapshot.getScanBuilder(defaultEngine).build()) + assert(snapshot.getVersion() == 1) + val scanFileRows = collectScanFileRows(snapshot.getScanBuilder().build()) assert(scanFileRows.isEmpty) } } @@ -274,7 +274,7 @@ class LogReplaySuite extends AnyFunSuite with TestUtils { test("delete and re-add same file in different transactions") { val path = goldenTablePath("delete-re-add-same-file-different-transactions") val snapshot = latestSnapshot(path) - val scan = snapshot.getScanBuilder(defaultEngine).build() + val scan = snapshot.getScanBuilder().build() val foundFiles = collectScanFileRows(scan).map(InternalScanFileUtils.getAddFileStatus) diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/RowTrackingSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/RowTrackingSuite.scala index f5406ce8869..ba6d9d57049 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/RowTrackingSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/RowTrackingSuite.scala @@ -75,9 +75,7 @@ class RowTrackingSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase { val table = TableImpl.forPath(engine, tablePath) val snapshot = table.getLatestSnapshot(engine).asInstanceOf[SnapshotImpl] - val scanFileRows = collectScanFileRows( - snapshot.getScanBuilder(engine).build() - ) + val scanFileRows = collectScanFileRows(snapshot.getScanBuilder().build()) val sortedBaseRowIds = scanFileRows .map(InternalScanFileUtils.getBaseRowId) .map(_.orElse(-1)) @@ -93,9 +91,7 @@ class RowTrackingSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase { val table = TableImpl.forPath(engine, tablePath) val snapshot = table.getLatestSnapshot(engine).asInstanceOf[SnapshotImpl] - val scanFileRows = collectScanFileRows( - snapshot.getScanBuilder(engine).build() - ) + val scanFileRows = collectScanFileRows(snapshot.getScanBuilder().build()) val sortedAddFileDefaultRowCommitVersions = scanFileRows .map(InternalScanFileUtils.getDefaultRowCommitVersion) .map(_.orElse(-1)) @@ -189,7 +185,7 @@ class RowTrackingSuite extends DeltaTableWriteSuiteBase with ParquetSuiteBase { // Checkpoint the table val table = TableImpl.forPath(engine, tablePath) - val latestVersion = table.getLatestSnapshot(engine).getVersion(engine) + val latestVersion = table.getLatestSnapshot(engine).getVersion() table.checkpoint(engine, latestVersion) val commitVersion3 = appendData( diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ScanSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ScanSuite.scala index 2d895472868..5e01230fcf7 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ScanSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ScanSuite.scala @@ -97,14 +97,14 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with val snapshot = latestSnapshot(tablePath) hits.foreach { predicate => val scanFiles = collectScanFileRows( - snapshot.getScanBuilder(defaultEngine) + snapshot.getScanBuilder() .withFilter(defaultEngine, predicate) .build()) assert(scanFiles.nonEmpty, s"Expected hit but got miss for $predicate") } misses.foreach { predicate => val scanFiles = collectScanFileRows( - snapshot.getScanBuilder(defaultEngine) + snapshot.getScanBuilder() .withFilter(defaultEngine, predicate) .build()) assert(scanFiles.isEmpty, s"Expected miss but got hit for $predicate\n" + @@ -121,7 +121,7 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with val snapshot = latestSnapshot(tablePath) filterToNumExpFiles.foreach { case (filter, numExpFiles) => val scanFiles = collectScanFileRows( - snapshot.getScanBuilder(defaultEngine) + snapshot.getScanBuilder() .withFilter(defaultEngine, filter) .build()) assert(scanFiles.length == numExpFiles, @@ -1010,9 +1010,7 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with predicate: Predicate, expNumPartitions: Int, expNumFiles: Long): Unit = { val snapshot = latestSnapshot(tableDir.getCanonicalPath) val scanFiles = collectScanFileRows( - snapshot.getScanBuilder(defaultEngine) - .withFilter(defaultEngine, predicate) - .build()) + snapshot.getScanBuilder().withFilter(defaultEngine, predicate).build()) assert(scanFiles.length == expNumFiles, s"Expected $expNumFiles but found ${scanFiles.length} for $predicate") @@ -1492,15 +1490,13 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with // no filter --> don't read stats verifyNoStatsColumn( - snapshot(engineDisallowedStatsReads) - .getScanBuilder(engine).build() - .getScanFiles(engine)) + snapshot(engineDisallowedStatsReads).getScanBuilder().build().getScanFiles(engine)) // partition filter only --> don't read stats val partFilter = equals(new Column("part"), ofInt(1)) verifyNoStatsColumn( snapshot(engineDisallowedStatsReads) - .getScanBuilder(engine).withFilter(engine, partFilter).build() + .getScanBuilder().withFilter(engine, partFilter).build() .getScanFiles(engine)) // no eligible data skipping filter --> don't read stats @@ -1509,7 +1505,7 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with ofInt(1)) verifyNoStatsColumn( snapshot(engineDisallowedStatsReads) - .getScanBuilder(engine).withFilter(engine, nonEligibleFilter).build() + .getScanBuilder().withFilter(engine, nonEligibleFilter).build() .getScanFiles(engine)) } @@ -1547,7 +1543,7 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with val engine = engineVerifyJsonParseSchema(verifySchema(expectedCols)) collectScanFileRows( Table.forPath(engine, path).getLatestSnapshot(engine) - .getScanBuilder(engine) + .getScanBuilder() .withFilter(engine, predicate) .build(), engine = engine) @@ -1570,14 +1566,12 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with } // No query filter checkStatsPresent( - latestSnapshot(tempDir.getCanonicalPath) - .getScanBuilder(defaultEngine) - .build() + latestSnapshot(tempDir.getCanonicalPath).getScanBuilder().build() ) // Query filter but no valid data skipping filter checkStatsPresent( latestSnapshot(tempDir.getCanonicalPath) - .getScanBuilder(defaultEngine) + .getScanBuilder() .withFilter( defaultEngine, greaterThan( @@ -1589,7 +1583,7 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with // With valid data skipping filter present checkStatsPresent( latestSnapshot(tempDir.getCanonicalPath) - .getScanBuilder(defaultEngine) + .getScanBuilder() .withFilter( defaultEngine, greaterThan( @@ -1617,7 +1611,7 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with case Some(version) => table.getSnapshotAsOfVersion(defaultEngine, version) case None => table.getLatestSnapshot(defaultEngine) } - val snapshotSchema = snapshot.getSchema(defaultEngine) + val snapshotSchema = snapshot.getSchema() val expectedSchema = new StructType() .add("id", LongType.LONG, true) @@ -1638,7 +1632,7 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with assert(snapshotSchema == expectedSchema) - val scanBuilder = snapshot.getScanBuilder(defaultEngine) + val scanBuilder = snapshot.getScanBuilder() val scan = predicate match { case Some(pred) => scanBuilder.withFilter(defaultEngine, pred).build() case None => scanBuilder.build() diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/SnapshotSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/SnapshotSuite.scala index bbd0e1e7ac2..a7373bc0acf 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/SnapshotSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/SnapshotSuite.scala @@ -55,7 +55,7 @@ class SnapshotSuite extends AnyFunSuite with TestUtils { // Step 2: Check the partition columns val tablePartCols = - table.getLatestSnapshot(defaultEngine).getPartitionColumnNames(defaultEngine) + table.getLatestSnapshot(defaultEngine).getPartitionColumnNames() assert(partCols.asJava === tablePartCols) } diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/utils/TestUtils.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/utils/TestUtils.scala index 30c3bde087e..ee73a0ce38f 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/utils/TestUtils.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/utils/TestUtils.scala @@ -154,12 +154,12 @@ trait TestUtils extends Assertions with SQLHelper { def tableSchema(path: String): StructType = { Table.forPath(defaultEngine, path) .getLatestSnapshot(defaultEngine) - .getSchema(defaultEngine) + .getSchema() } def hasTableProperty(tablePath: String, propertyKey: String, expValue: String): Boolean = { val table = Table.forPath(defaultEngine, tablePath) - val schema = table.getLatestSnapshot(defaultEngine).getSchema(defaultEngine) + val schema = table.getLatestSnapshot(defaultEngine).getSchema() schema.fields().asScala.exists { field => field.getMetadata.getString(propertyKey) == expValue } @@ -194,7 +194,7 @@ trait TestUtils extends Assertions with SQLHelper { val result = ArrayBuffer[Row]() - var scanBuilder = snapshot.getScanBuilder(engine) + var scanBuilder = snapshot.getScanBuilder() if (readSchema != null) { scanBuilder = scanBuilder.withReadSchema(engine, readSchema) @@ -264,7 +264,7 @@ trait TestUtils extends Assertions with SQLHelper { readSchema: StructType): Seq[FilteredColumnarBatch] = { val scan = Table.forPath(engine, tablePath) .getLatestSnapshot(engine) - .getScanBuilder(engine) + .getScanBuilder() .withReadSchema(engine, readSchema) .build() val scanState = scan.getScanState(engine) @@ -372,25 +372,24 @@ trait TestUtils extends Assertions with SQLHelper { val readSchema = if (readCols == null) { null } else { - val schema = snapshot.getSchema(engine) + val schema = snapshot.getSchema() new StructType(readCols.map(schema.get(_)).asJava) } if (expectedSchema != null) { assert( - expectedSchema == snapshot.getSchema(engine), + expectedSchema == snapshot.getSchema(), s""" |Expected schema does not match actual schema: |Expected schema: $expectedSchema - |Actual schema: ${snapshot.getSchema(engine)} + |Actual schema: ${snapshot.getSchema()} |""".stripMargin ) } expectedVersion.foreach { version => - assert(version == snapshot.getVersion(defaultEngine), - s"Expected version $version does not match actual version" + - s" ${snapshot.getVersion(defaultEngine)}") + assert(version == snapshot.getVersion(), + s"Expected version $version does not match actual version ${snapshot.getVersion()}") } val result = readSnapshot(snapshot, readSchema, filter, expectedRemainingFilter, engine) From 75d6b8eed610b8fa05f474df9ca397b6ab5da7c2 Mon Sep 17 00:00:00 2001 From: Scott Sandre Date: Thu, 30 Jan 2025 08:49:41 -0800 Subject: [PATCH 24/28] [Kernel] Minor refactor to DeltaLogActionUtils; add CloseableIterator takeWhile and other helpful methods (#4097) #### Which Delta project/connector is this regarding? - [ ] Spark - [ ] Standalone - [ ] Flink - [X] Kernel - [ ] Other (fill in here) ## Description This PR does the following: - changes our DeltaLogActionUtils `listDeltaLogFiles` method to return an iterator. This will eventually let it be used in a followup PR to further consolidate and clean up code in DeltaHistoryManager (which lists the delta log and expects an iterator) - adds CloseabelIterator::takeWhile, breakableFilter, and toInMemoryList ## How was this patch tested? - New UTs ## Does this PR introduce _any_ user-facing changes? No. --- .../kernel/internal/DeltaLogActionUtils.java | 138 +++++++++--------- .../internal/snapshot/SnapshotManager.java | 16 +- .../delta/kernel/utils/CloseableIterator.java | 112 +++++++++++++- .../delta/kernel/CloseableIteratorSuite.scala | 78 ++++++++++ .../internal/DeltaLogActionUtilsSuite.scala | 18 +-- .../DeltaLogActionUtilsE2ESuite.scala | 10 +- 6 files changed, 276 insertions(+), 96 deletions(-) create mode 100644 kernel/kernel-api/src/test/scala/io/delta/kernel/CloseableIteratorSuite.scala diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaLogActionUtils.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaLogActionUtils.java index 69389a9c806..67f2add01f3 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaLogActionUtils.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/DeltaLogActionUtils.java @@ -35,11 +35,13 @@ import io.delta.kernel.internal.util.FileNames.DeltaLogFileType; import io.delta.kernel.types.*; import io.delta.kernel.utils.CloseableIterator; +import io.delta.kernel.utils.CloseableIterator.BreakableFilterResult; import io.delta.kernel.utils.FileStatus; import java.io.FileNotFoundException; import java.io.IOException; import java.io.UncheckedIOException; import java.util.*; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -103,13 +105,14 @@ public static List getCommitFilesForVersionRange( // Get any available commit files within the version range final List commitFiles = - listDeltaLogFiles( - engine, - Collections.singleton(DeltaLogFileType.COMMIT), - tablePath, - startVersion, - Optional.of(endVersion), - false /* mustBeRecreatable */); + listDeltaLogFilesAsIter( + engine, + Collections.singleton(DeltaLogFileType.COMMIT), + tablePath, + startVersion, + Optional.of(endVersion), + false /* mustBeRecreatable */) + .toInMemoryList(); // There are no available commit files within the version range. // This can be due to (1) an empty directory, (2) no valid delta files in the directory, @@ -181,14 +184,14 @@ public static CloseableIterator readCommitFiles( } /** - * Returns the list of files of type $fileTypes in the _delta_log directory of the given - * $tablePath, in increasing order from $startVersion to the optional $endVersion. + * Returns a {@link CloseableIterator} of files of type $fileTypes in the _delta_log directory of + * the given $tablePath, in increasing order from $startVersion to the optional $endVersion. * * @throws TableNotFoundException if the table or its _delta_log does not exist * @throws KernelException if mustBeRecreatable is true, endVersionOpt is present, and the * _delta_log history has been truncated so that we cannot load the desired end version */ - public static List listDeltaLogFiles( + public static CloseableIterator listDeltaLogFilesAsIter( Engine engine, Set fileTypes, Path tablePath, @@ -215,69 +218,58 @@ public static List listDeltaLogFiles( startVersion, endVersionOpt); - final List output = new ArrayList<>(); - final long startTimeMillis = System.currentTimeMillis(); - - try (CloseableIterator fsIter = listLogDir(engine, tablePath, startVersion)) { - while (fsIter.hasNext()) { - final FileStatus fs = fsIter.next(); - - if (fileTypes.contains(DeltaLogFileType.COMMIT) - && FileNames.isCommitFile(getName(fs.getPath()))) { - // Here, we do nothing (we will consume this file). - } else if (fileTypes.contains(DeltaLogFileType.CHECKPOINT) - && FileNames.isCheckpointFile(getName(fs.getPath())) - && fs.getSize() > 0) { - // Checkpoint files of 0 size are invalid but may be ignored silently when read, hence we - // ignore them so that we never pick up such checkpoints. - // Here, we do nothing (we will consume this file). - } else { - logger.debug("Ignoring file {} as it is not of the desired type", fs.getPath()); - continue; // Here, we continue and skip this file. - } - - final long fileVersion = FileNames.getFileVersion(new Path(fs.getPath())); - - if (fileVersion < startVersion) { - throw new RuntimeException( - String.format( - "Listing files in %s with startVersion %s yet found file %s with version %s", - logPath, startVersion, fs.getPath(), fileVersion)); - } - - if (endVersionOpt.isPresent()) { - final long endVersion = endVersionOpt.get(); - - if (fileVersion > endVersion) { - if (mustBeRecreatable && output.isEmpty()) { - final long earliestVersion = - DeltaHistoryManager.getEarliestRecreatableCommit(engine, logPath); - throw DeltaErrors.versionBeforeFirstAvailableCommit( - tablePath.toString(), endVersion, earliestVersion); - } else { - logger.debug( - "Stopping listing; found file {} with version > {}=endVersion", - fs.getPath(), - endVersion); - break; - } - } - } - - output.add(fs); - } - } catch (IOException e) { - throw new UncheckedIOException("Unable to close resource", e); - } - - logger.info( - "{}: Took {} ms to list the commit files for versions [{}, {}]", - tablePath, - System.currentTimeMillis() - startTimeMillis, - startVersion, - endVersionOpt); - - return output; + // Must be final to be used in lambda + final AtomicBoolean hasReturnedAnElement = new AtomicBoolean(false); + + return listLogDir(engine, tablePath, startVersion) + .breakableFilter( + fs -> { + if (fileTypes.contains(DeltaLogFileType.COMMIT) + && FileNames.isCommitFile(getName(fs.getPath()))) { + // Here, we do nothing (we will consume this file). + } else if (fileTypes.contains(DeltaLogFileType.CHECKPOINT) + && FileNames.isCheckpointFile(getName(fs.getPath())) + && fs.getSize() > 0) { + // Checkpoint files of 0 size are invalid but may be ignored silently when read, + // hence we ignore them so that we never pick up such checkpoints. + // Here, we do nothing (we will consume this file). + } else { + logger.debug("Ignoring file {} as it is not of the desired type", fs.getPath()); + return BreakableFilterResult.EXCLUDE; // Here, we exclude and filter out this file. + } + + final long fileVersion = FileNames.getFileVersion(new Path(fs.getPath())); + + if (fileVersion < startVersion) { + throw new RuntimeException( + String.format( + "Listing files in %s with startVersion %s yet found file %s.", + logPath, startVersion, fs.getPath())); + } + + if (endVersionOpt.isPresent()) { + final long endVersion = endVersionOpt.get(); + + if (fileVersion > endVersion) { + if (mustBeRecreatable && !hasReturnedAnElement.get()) { + final long earliestVersion = + DeltaHistoryManager.getEarliestRecreatableCommit(engine, logPath); + throw DeltaErrors.versionBeforeFirstAvailableCommit( + tablePath.toString(), endVersion, earliestVersion); + } else { + logger.debug( + "Stopping listing; found file {} with version greater than endVersion {}", + fs.getPath(), + endVersion); + return BreakableFilterResult.BREAK; + } + } + } + + hasReturnedAnElement.set(true); + + return BreakableFilterResult.INCLUDE; + }); } ////////////////////// diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java index ed3f228389d..eabac72e5bc 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java @@ -344,13 +344,15 @@ public LogSegment getLogSegmentForVersion(Engine engine, Optional versionT final long startTimeMillis = System.currentTimeMillis(); final List listedFileStatuses = - DeltaLogActionUtils.listDeltaLogFiles( - engine, - new HashSet<>(Arrays.asList(DeltaLogFileType.COMMIT, DeltaLogFileType.CHECKPOINT)), - tablePath, - listFromStartVersion, - versionToLoadOpt, - true /* mustBeRecreatable */); + DeltaLogActionUtils.listDeltaLogFilesAsIter( + engine, + new HashSet<>(Arrays.asList(DeltaLogFileType.COMMIT, DeltaLogFileType.CHECKPOINT)), + tablePath, + listFromStartVersion, + versionToLoadOpt, + true /* mustBeRecreatable */) + .toInMemoryList(); + logger.info( "{}: Took {}ms to list the files after starting checkpoint", tablePath, diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/utils/CloseableIterator.java b/kernel/kernel-api/src/main/java/io/delta/kernel/utils/CloseableIterator.java index dbae0033625..cb735e278fa 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/utils/CloseableIterator.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/utils/CloseableIterator.java @@ -23,7 +23,10 @@ import io.delta.kernel.internal.util.Utils; import java.io.Closeable; import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.ArrayList; import java.util.Iterator; +import java.util.List; import java.util.NoSuchElementException; import java.util.function.Function; @@ -36,6 +39,31 @@ @Evolving public interface CloseableIterator extends Iterator, Closeable { + /** + * Represents the result of applying the filter condition in the {@link + * #breakableFilter(Function)} method of a {@link CloseableIterator}. This enum determines how + * each element in the iterator should be handled. + */ + enum BreakableFilterResult { + /** + * Indicates that the current element should be included in the resulting iterator produced by + * {@link #breakableFilter(Function)}. + */ + INCLUDE, + + /** + * Indicates that the current element should be excluded from the resulting iterator produced by + * {@link #breakableFilter(Function)}. + */ + EXCLUDE, + + /** + * Indicates that the iteration should stop immediately and that no further elements should be + * processed by {@link #breakableFilter(Function)}. + */ + BREAK + } + /** * Returns true if the iteration has more elements. (In other words, returns true if next would * return an element rather than throwing an exception.) @@ -91,23 +119,81 @@ public void close() throws IOException { }; } + /** + * Returns a new {@link CloseableIterator} that includes only the elements of this iterator for + * which the given {@code mapper} function returns {@code true}. + * + * @param mapper A function that determines whether an element should be included in the resulting + * iterator. + * @return A {@link CloseableIterator} that includes only the filtered the elements of this + * iterator. + */ default CloseableIterator filter(Function mapper) { + return breakableFilter( + t -> { + if (mapper.apply(t)) { + return BreakableFilterResult.INCLUDE; + } else { + return BreakableFilterResult.EXCLUDE; + } + }); + } + + /** + * Returns a new {@link CloseableIterator} that includes elements from this iterator as long as + * the given {@code mapper} function returns {@code true}. Once the mapper function returns {@code + * false}, the iteration is terminated. + * + * @param mapper A function that determines whether to include an element in the resulting + * iterator. + * @return A {@link CloseableIterator} that stops iteration when the condition is not met. + */ + default CloseableIterator takeWhile(Function mapper) { + return breakableFilter( + t -> { + if (mapper.apply(t)) { + return BreakableFilterResult.INCLUDE; + } else { + return BreakableFilterResult.BREAK; + } + }); + } + + /** + * Returns a new {@link CloseableIterator} that applies a {@link BreakableFilterResult}-based + * filtering function to determine whether elements of this iterator should be included or + * excluded, or whether the iteration should terminate. + * + * @param mapper A function that determines the filtering action for each element: include, + * exclude, or break. + * @return A {@link CloseableIterator} that applies the specified {@link + * BreakableFilterResult}-based logic. + */ + default CloseableIterator breakableFilter(Function mapper) { CloseableIterator delegate = this; return new CloseableIterator() { T next; boolean hasLoadedNext; + boolean shouldBreak = false; @Override public boolean hasNext() { + if (shouldBreak) { + return false; + } if (hasLoadedNext) { return true; } while (delegate.hasNext()) { - T potentialNext = delegate.next(); - if (mapper.apply(potentialNext)) { + final T potentialNext = delegate.next(); + final BreakableFilterResult result = mapper.apply(potentialNext); + if (result == BreakableFilterResult.INCLUDE) { next = potentialNext; hasLoadedNext = true; return true; + } else if (result == BreakableFilterResult.BREAK) { + shouldBreak = true; + return false; } } return false; @@ -160,4 +246,26 @@ public void close() throws IOException { } }; } + + /** + * Collects all elements from this {@link CloseableIterator} into a {@link List}. + * + *

    This method iterates through all elements of the iterator, storing them in an in-memory + * list. Once iteration is complete, the iterator is automatically closed to release any + * underlying resources. + * + * @return A {@link List} containing all elements from this iterator. + * @throws UncheckedIOException If an {@link IOException} occurs while closing the iterator. + */ + default List toInMemoryList() { + final List result = new ArrayList<>(); + try (CloseableIterator iterator = this) { + while (iterator.hasNext()) { + result.add(iterator.next()); + } + } catch (IOException e) { + throw new UncheckedIOException("Failed to close the CloseableIterator", e); + } + return result; + } } diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/CloseableIteratorSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/CloseableIteratorSuite.scala new file mode 100644 index 00000000000..387112bb25f --- /dev/null +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/CloseableIteratorSuite.scala @@ -0,0 +1,78 @@ +/* + * Copyright (2025) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.kernel + +import scala.collection.JavaConverters._ + +import io.delta.kernel.internal.util.Utils +import io.delta.kernel.utils.CloseableIterator +import io.delta.kernel.utils.CloseableIterator.BreakableFilterResult +import org.scalatest.funsuite.AnyFunSuite + +class CloseableIteratorSuite extends AnyFunSuite { + + private def toCloseableIter[T](elems: Seq[T]): CloseableIterator[T] = { + Utils.toCloseableIterator(elems.iterator.asJava) + } + + private def toList[T](iter: CloseableIterator[T]): List[T] = { + iter.toInMemoryList.asScala.toList + } + + private def normalDataIter = toCloseableIter(Seq(1, 2, 3, 4, 5)) + + private def throwingDataIter = toCloseableIter(Seq(1, 2, 3, 4, 5)).map { x => + if (x > 4) { + throw new RuntimeException("Underlying data evaluated at element > 4") + } + x + } + + test("CloseableIterator::filter -- returns filtered result") { + val result = normalDataIter.filter(x => x <= 3 || x == 5) + assert(toList(result) === List(1, 2, 3, 5)) + } + + test("CloseableIterator::filter -- iterates over all elements") { + intercept[RuntimeException] { + toList(throwingDataIter.filter(x => x <= 3)) + } + } + + test("CloseableIterator::takeWhile -- stops iteration at first false condition") { + // we expect it to evaluate 1, 2, 3, 4; break when it sees x == 4; and only return 1, 2, 3 + val result = throwingDataIter.takeWhile(x => x <= 3) + assert(toList(result) === List(1, 2, 3)) + } + + test("CloseableIterator::breakableFilter -- correctly filters and breaks iteration") { + val result = throwingDataIter.breakableFilter { x => + if (x <= 1 || x == 3) { + BreakableFilterResult.INCLUDE + } else if (x == 2) { + BreakableFilterResult.EXCLUDE + } else if (x == 4) { + BreakableFilterResult.BREAK + } else { + throw new RuntimeException("This should never be reached") + } + } + // we except it to include 1; exclude 2; include 3; and break at 4, thus never seeing 5 + assert(toList(result) === List(1, 3)) + } + +} diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/DeltaLogActionUtilsSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/DeltaLogActionUtilsSuite.scala index 938e025d30c..d6b799bb62b 100644 --- a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/DeltaLogActionUtilsSuite.scala +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/DeltaLogActionUtilsSuite.scala @@ -24,7 +24,7 @@ import scala.reflect.ClassTag import io.delta.kernel.exceptions.{InvalidTableException, KernelException, TableNotFoundException} import io.delta.kernel.internal.util.FileNames import io.delta.kernel.utils.FileStatus -import io.delta.kernel.internal.DeltaLogActionUtils.{getCommitFilesForVersionRange, listDeltaLogFiles, verifyDeltaVersions} +import io.delta.kernel.internal.DeltaLogActionUtils.{getCommitFilesForVersionRange, listDeltaLogFilesAsIter, verifyDeltaVersions} import io.delta.kernel.internal.fs.Path import io.delta.kernel.test.MockFileSystemClientUtils @@ -261,38 +261,38 @@ class DeltaLogActionUtilsSuite extends AnyFunSuite with MockFileSystemClientUtil test("listDeltaLogFiles: no fileTypes provided") { intercept[IllegalArgumentException] { - listDeltaLogFiles( + listDeltaLogFilesAsIter( createMockFSListFromEngine(deltaFileStatuses(Seq(1, 2, 3))), Collections.emptySet(), // No fileTypes provided! dataPath, 1, Optional.empty(), false /* mustBeRecreatable */ - ) + ).toInMemoryList } } test("listDeltaLogFiles: returns requested file type only") { - val commitFiles = listDeltaLogFiles( + val commitFiles = listDeltaLogFilesAsIter( createMockFSListFromEngine(checkpointsAndDeltas), Set(FileNames.DeltaLogFileType.COMMIT).asJava, dataPath, 10, Optional.empty(), false /* mustBeRecreatable */ - ).asScala + ).toInMemoryList.asScala assert(commitFiles.forall(fs => FileNames.isCommitFile(fs.getPath))) assert(extractVersions(commitFiles) == Seq(10, 11, 12, 13, 14, 15, 16, 17)) - val checkpointFiles = listDeltaLogFiles( + val checkpointFiles = listDeltaLogFilesAsIter( createMockFSListFromEngine(checkpointsAndDeltas), Set(FileNames.DeltaLogFileType.CHECKPOINT).asJava, dataPath, 10, Optional.empty(), false /* mustBeRecreatable */ - ).asScala + ).toInMemoryList.asScala assert(checkpointFiles.forall(fs => FileNames.isCheckpointFile(fs.getPath))) assert(extractVersions(checkpointFiles) == Seq(10, 14, 14, 17)) @@ -300,14 +300,14 @@ class DeltaLogActionUtilsSuite extends AnyFunSuite with MockFileSystemClientUtil test("listDeltaLogFiles: mustBeRecreatable") { val exMsg = intercept[KernelException] { - listDeltaLogFiles( + listDeltaLogFilesAsIter( createMockFSListFromEngine(checkpointsAndDeltas), Set(FileNames.DeltaLogFileType.COMMIT, FileNames.DeltaLogFileType.CHECKPOINT).asJava, dataPath, 0, Optional.of(4), true /* mustBeRecreatable */ - ) + ).toInMemoryList }.getMessage assert(exMsg.contains("Cannot load table version 4 as the transaction log has been " + "truncated due to manual deletion or the log/checkpoint retention policy. The earliest " + diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaLogActionUtilsE2ESuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaLogActionUtilsE2ESuite.scala index 84f9aebb3bc..3bb7a96cb08 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaLogActionUtilsE2ESuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/DeltaLogActionUtilsE2ESuite.scala @@ -23,7 +23,7 @@ import scala.collection.JavaConverters._ import io.delta.kernel.defaults.utils.TestUtils import io.delta.kernel.exceptions.TableNotFoundException -import io.delta.kernel.internal.DeltaLogActionUtils.listDeltaLogFiles +import io.delta.kernel.internal.DeltaLogActionUtils.listDeltaLogFilesAsIter import io.delta.kernel.internal.fs.Path import io.delta.kernel.internal.util.FileNames @@ -34,14 +34,14 @@ class DeltaLogActionUtilsE2ESuite extends AnyFunSuite with TestUtils { test("listDeltaLogFiles: throws TableNotFoundException if _delta_log does not exist") { withTempDir { tableDir => intercept[TableNotFoundException] { - listDeltaLogFiles( + listDeltaLogFilesAsIter( defaultEngine, Set(FileNames.DeltaLogFileType.COMMIT, FileNames.DeltaLogFileType.CHECKPOINT).asJava, new Path(tableDir.getAbsolutePath), 0, Optional.empty(), true /* mustBeRecreatable */ - ) + ).toInMemoryList } } } @@ -51,14 +51,14 @@ class DeltaLogActionUtilsE2ESuite extends AnyFunSuite with TestUtils { val logDir = new File(tableDir, "_delta_log") assert(logDir.mkdirs() && logDir.isDirectory && logDir.listFiles().isEmpty) - val result = listDeltaLogFiles( + val result = listDeltaLogFilesAsIter( defaultEngine, Set(FileNames.DeltaLogFileType.COMMIT, FileNames.DeltaLogFileType.CHECKPOINT).asJava, new Path(tableDir.getAbsolutePath), 0, Optional.empty(), true /* mustBeRecreatable */ - ) + ).toInMemoryList assert(result.isEmpty) } From 57029e1d1feb1f0e23f210b405fce0f37eac428f Mon Sep 17 00:00:00 2001 From: Chirag Singh <137233133+chirag-s-db@users.noreply.github.com> Date: Thu, 30 Jan 2025 08:55:12 -0800 Subject: [PATCH 25/28] [Spark] Add configurable expansions to partition-like skipping coverage (#4104) #### Which Delta project/connector is this regarding? - [ ] Spark - [ ] Standalone - [ ] Flink - [ ] Kernel - [ ] Other (fill in here) ## Description Currently, partition-like skipping is limited to a set of whitelisted expressions referencing only Liquid clustering columns. This PR adds configs that can be used to expand these restrictions (allow an arbitrary expression referencing any column). ## How was this patch tested? See test changes. ## Does this PR introduce _any_ user-facing changes? No. --- .../sql/delta/sources/DeltaSQLConf.scala | 18 +++++++ .../sql/delta/stats/DataSkippingReader.scala | 14 ++++- .../PartitionLikeDataSkippingSuite.scala | 51 +++++++++++++++++++ 3 files changed, 81 insertions(+), 2 deletions(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala b/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala index d43dcf0cb34..e90ce982a46 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala @@ -1452,6 +1452,24 @@ trait DeltaSQLConfBase { .intConf .createWithDefault(100) + val DELTA_DATASKIPPING_PARTITION_LIKE_FILTERS_CLUSTERING_COLUMNS_ONLY = + buildConf("skipping.partitionLikeDataSkipping.limitToClusteringColumns") + .internal() + .doc("Limits partition-like data skipping to filters referencing only clustering columns" + + "In general, clustering columns will be most likely to produce files with the same" + + "min-max values, though this restriction might exclude filters on columns highly " + + "correlated with the clustering columns.") + .booleanConf + .createWithDefault(true) + + val DELTA_DATASKIPPING_PARTITION_LIKE_FILTERS_ADDITIONAL_SUPPORTED_EXPRESSIONS = + buildConf("skipping.partitionLikeDataSkipping.additionalSupportedExpressions") + .internal() + .doc("Comma-separated list of the canonical class names of additional expressions for which" + + "partition-like data skipping can be safely applied.") + .stringConf + .createOptional + /** * The below confs have a special prefix `spark.databricks.io` because this is the conf value * already used by Databricks' data skipping implementation. There's no benefit to making OSS diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/stats/DataSkippingReader.scala b/spark/src/main/scala/org/apache/spark/sql/delta/stats/DataSkippingReader.scala index aedc7c77342..d359d8faac7 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/stats/DataSkippingReader.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/stats/DataSkippingReader.scala @@ -220,6 +220,13 @@ trait DataSkippingReaderBase private def useStats = spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_STATS_SKIPPING) + private lazy val limitPartitionLikeFiltersToClusteringColumns = spark.sessionState.conf.getConf( + DeltaSQLConf.DELTA_DATASKIPPING_PARTITION_LIKE_FILTERS_CLUSTERING_COLUMNS_ONLY) + private lazy val additionalPartitionLikeFilterSupportedExpressions = + spark.sessionState.conf.getConf( + DeltaSQLConf.DELTA_DATASKIPPING_PARTITION_LIKE_FILTERS_ADDITIONAL_SUPPORTED_EXPRESSIONS) + .toSet.flatMap((exprs: String) => exprs.split(",")) + /** Returns a DataFrame expression to obtain a list of files with parsed statistics. */ private def withStatsInternal0: DataFrame = { allFiles.withColumn("stats", from_json(col("stats"), statsSchema)) @@ -681,7 +688,8 @@ trait DataSkippingReaderBase // Don't attempt partition-like skipping on any unknown expressions: there's no way to // guarantee it's safe to do so. - case _ => false + case _ => additionalPartitionLikeFilterSupportedExpressions.contains( + expr.getClass.getCanonicalName) } /** @@ -700,6 +708,7 @@ trait DataSkippingReaderBase * CAST(a AS DATE) = '2024-09-11' -> CAST(parsed_stats[minValues][a] AS DATE) = '2024-09-11' * * @param expr The expression to rewrite. + * @param clusteringColumnPaths The logical paths to the clustering columns in the table. * @return If the expression is safe to rewrite, return the rewritten expression and a * set of referenced attributes (with both the logical path to the column and the * column type). @@ -718,7 +727,8 @@ trait DataSkippingReaderBase // to have the same min-max values). case SkippingEligibleColumn(c, SkippingEligibleDataType(dt)) if dt != TimestampType && dt != TimestampNTZType && - clusteringColumnPaths.exists(SchemaUtils.areLogicalNamesEqual(_, c.reverse)) => + (!limitPartitionLikeFiltersToClusteringColumns || + clusteringColumnPaths.exists(SchemaUtils.areLogicalNamesEqual(_, c.reverse))) => // Only rewrite the expression if all stats are collected for this column. val minStatsCol = StatsColumn(MIN, c, dt) val maxStatsCol = StatsColumn(MAX, c, dt) diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/stats/PartitionLikeDataSkippingSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/stats/PartitionLikeDataSkippingSuite.scala index 4b2bfa33688..d9120d53931 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/stats/PartitionLikeDataSkippingSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/stats/PartitionLikeDataSkippingSuite.scala @@ -375,6 +375,57 @@ trait PartitionLikeDataSkippingSuiteBase minNumFilesToApply = 1) } } + + test("partition-like skipping can reference non-clustering columns via config") { + withSQLConf( + DeltaSQLConf.DELTA_DATASKIPPING_PARTITION_LIKE_FILTERS_CLUSTERING_COLUMNS_ONLY.key -> + "false") { + validateExpectedScanMetrics( + tableName = testTableName, + query = s"SELECT * FROM $testTableName WHERE CAST(e AS STRING) = '1'", + expectedNumFiles = 12, + expectedNumPartitionLikeDataFilters = 1, + allPredicatesUsed = true, + minNumFilesToApply = 1L) + } + } + + test("partition-like skipping whitelist can be expanded via config") { + // Single additional supported expression. + withSQLConf( + DeltaSQLConf.DELTA_DATASKIPPING_PARTITION_LIKE_FILTERS_ADDITIONAL_SUPPORTED_EXPRESSIONS.key -> + "org.apache.spark.sql.catalyst.expressions.RegExpExtract") { + val query = s"SELECT * FROM $testTableName " + + "WHERE REGEXP_EXTRACT(s.b, '([0-9][0-9][0-9][0-9]).*') = '1971'" + validateExpectedScanMetrics( + tableName = testTableName, + query = query, + expectedNumFiles = 12, + expectedNumPartitionLikeDataFilters = 1, + allPredicatesUsed = true, + minNumFilesToApply = 1L) + } + + // Multiple additional supported expressions. + DeltaLog.clearCache() // Clear cache to avoid stale config reads. + withSQLConf( + DeltaSQLConf.DELTA_DATASKIPPING_PARTITION_LIKE_FILTERS_ADDITIONAL_SUPPORTED_EXPRESSIONS.key -> + ("org.apache.spark.sql.catalyst.expressions.RegExpExtract," + + "org.apache.spark.sql.catalyst.expressions.JsonToStructs")) { + val query = s""" + |SELECT * FROM $testTableName + |WHERE (REGEXP_EXTRACT(s.b, '([0-9][0-9][0-9][0-9]).*') = '1971' OR + |FROM_JSON(CONCAT('{"date":"', STRING(c), '"}'), 'date STRING')['date'] = '1972-03-02') + |""".stripMargin + validateExpectedScanMetrics( + tableName = testTableName, + query = query, + expectedNumFiles = 13, + expectedNumPartitionLikeDataFilters = 1, + allPredicatesUsed = true, + minNumFilesToApply = 1L) + } + } } class PartitionLikeDataSkippingSuite extends PartitionLikeDataSkippingSuiteBase From 785ec3c5565afb6d16de21e460228c56a8fd16f5 Mon Sep 17 00:00:00 2001 From: Cuong Nguyen Date: Thu, 30 Jan 2025 12:34:56 -0800 Subject: [PATCH 26/28] [Spark] Introduce the DeltaTransaction trait (#4100) #### Which Delta project/connector is this regarding? - [x] Spark - [ ] Standalone - [ ] Flink - [ ] Kernel - [ ] Other (fill in here) ## Description The `OptimisticTransactionImpl` trait is bloating as more features are added, making it difficult to extend from. This PR introduces `DeltaTransaction`, which is conceptually more general to support other implementations of the transaction, containing minimal number of fields that are essential for a transaction and necessary for the post-commit hooks. ## How was this patch tested? unit test ## Does this PR introduce _any_ user-facing changes? no --- .../spark/sql/delta/hudi/HudiConverter.scala | 12 ++-- .../icebergShaded/IcebergConverter.scala | 18 +++--- .../spark/sql/delta/DeltaTransaction.scala | 61 +++++++++++++++++++ .../sql/delta/OptimisticTransaction.scala | 19 ++---- .../spark/sql/delta/UniversalFormat.scala | 4 +- .../sql/delta/files/TransactionalWrite.scala | 8 --- .../spark/sql/delta/hooks/AutoCompact.scala | 6 +- .../sql/delta/hooks/AutoCompactUtils.scala | 6 +- .../sql/delta/hooks/CheckpointHook.scala | 4 +- .../spark/sql/delta/hooks/ChecksumHook.scala | 6 +- .../delta/hooks/GenerateSymlinkManifest.scala | 4 +- .../sql/delta/hooks/HudiConverterHook.scala | 4 +- .../delta/hooks/IcebergConverterHook.scala | 7 +-- .../sql/delta/hooks/PostCommitHook.scala | 2 +- .../spark/sql/delta/hooks/UpdateCatalog.scala | 4 +- .../spark/sql/delta/DeltaErrorsSuite.scala | 4 +- 16 files changed, 105 insertions(+), 64 deletions(-) create mode 100644 spark/src/main/scala/org/apache/spark/sql/delta/DeltaTransaction.scala diff --git a/hudi/src/main/scala/org/apache/spark/sql/delta/hudi/HudiConverter.scala b/hudi/src/main/scala/org/apache/spark/sql/delta/hudi/HudiConverter.scala index 03e06b9fae8..1efca0dc2d2 100644 --- a/hudi/src/main/scala/org/apache/spark/sql/delta/hudi/HudiConverter.scala +++ b/hudi/src/main/scala/org/apache/spark/sql/delta/hudi/HudiConverter.scala @@ -67,9 +67,9 @@ class HudiConverter(spark: SparkSession) // Save an atomic reference of the snapshot being converted, and the txn that triggered // resulted in the specified snapshot protected val currentConversion = - new AtomicReference[(Snapshot, OptimisticTransactionImpl)]() + new AtomicReference[(Snapshot, DeltaTransaction)]() protected val standbyConversion = - new AtomicReference[(Snapshot, OptimisticTransactionImpl)]() + new AtomicReference[(Snapshot, DeltaTransaction)]() // Whether our async converter thread is active. We may already have an alive thread that is // about to shutdown, but in such cases this value should return false. @@ -88,7 +88,7 @@ class HudiConverter(spark: SparkSession) */ override def enqueueSnapshotForConversion( snapshotToConvert: Snapshot, - txn: OptimisticTransactionImpl): Unit = { + txn: DeltaTransaction): Unit = { if (!UniversalFormat.hudiEnabled(snapshotToConvert.metadata)) { return } @@ -138,7 +138,7 @@ class HudiConverter(spark: SparkSession) } // Get a snapshot to convert from the hudiQueue. Sets the queue to null after. - private def getNextSnapshot: (Snapshot, OptimisticTransactionImpl) = + private def getNextSnapshot: (Snapshot, DeltaTransaction) = asyncThreadLock.synchronized { val potentialSnapshotAndTxn = standbyConversion.get() currentConversion.set(potentialSnapshotAndTxn) @@ -189,7 +189,7 @@ class HudiConverter(spark: SparkSession) * @return Converted Delta version and commit timestamp */ override def convertSnapshot( - snapshotToConvert: Snapshot, txn: OptimisticTransactionImpl): Option[(Long, Long)] = { + snapshotToConvert: Snapshot, txn: DeltaTransaction): Option[(Long, Long)] = { if (!UniversalFormat.hudiEnabled(snapshotToConvert.metadata)) { return None } @@ -207,7 +207,7 @@ class HudiConverter(spark: SparkSession) */ private def convertSnapshot( snapshotToConvert: Snapshot, - txnOpt: Option[OptimisticTransactionImpl], + txnOpt: Option[DeltaTransaction], catalogTable: Option[CatalogTable]): Option[(Long, Long)] = recordFrameProfile("Delta", "HudiConverter.convertSnapshot") { val log = snapshotToConvert.deltaLog diff --git a/iceberg/src/main/scala/org/apache/spark/sql/delta/icebergShaded/IcebergConverter.scala b/iceberg/src/main/scala/org/apache/spark/sql/delta/icebergShaded/IcebergConverter.scala index 075e3ddcb1a..a06836c193f 100644 --- a/iceberg/src/main/scala/org/apache/spark/sql/delta/icebergShaded/IcebergConverter.scala +++ b/iceberg/src/main/scala/org/apache/spark/sql/delta/icebergShaded/IcebergConverter.scala @@ -23,7 +23,7 @@ import scala.collection.JavaConverters._ import scala.util.control.Breaks._ import scala.util.control.NonFatal -import org.apache.spark.sql.delta.{DeltaErrors, DeltaFileNotFoundException, DeltaFileProviderUtils, IcebergConstants, OptimisticTransactionImpl, Snapshot, UniversalFormat, UniversalFormatConverter} +import org.apache.spark.sql.delta.{DeltaErrors, DeltaFileNotFoundException, DeltaFileProviderUtils, DeltaTransaction, IcebergConstants, Snapshot, UniversalFormat, UniversalFormatConverter} import org.apache.spark.sql.delta.DeltaOperations.OPTIMIZE_OPERATION_NAME import org.apache.spark.sql.delta.actions.{Action, AddFile, CommitInfo, RemoveFile} import org.apache.spark.sql.delta.hooks.IcebergConverterHook @@ -78,9 +78,9 @@ class IcebergConverter(spark: SparkSession) // Save an atomic reference of the snapshot being converted, and the txn that triggered // resulted in the specified snapshot protected val currentConversion = - new AtomicReference[(Snapshot, OptimisticTransactionImpl)]() + new AtomicReference[(Snapshot, DeltaTransaction)]() protected val standbyConversion = - new AtomicReference[(Snapshot, OptimisticTransactionImpl)]() + new AtomicReference[(Snapshot, DeltaTransaction)]() // Whether our async converter thread is active. We may already have an alive thread that is // about to shutdown, but in such cases this value should return false. @@ -99,7 +99,7 @@ class IcebergConverter(spark: SparkSession) */ override def enqueueSnapshotForConversion( snapshotToConvert: Snapshot, - txn: OptimisticTransactionImpl): Unit = { + txn: DeltaTransaction): Unit = { if (!UniversalFormat.icebergEnabled(snapshotToConvert.metadata)) { return } @@ -149,7 +149,7 @@ class IcebergConverter(spark: SparkSession) } // Get a snapshot to convert from the icebergQueue. Sets the queue to null after. - private def getNextSnapshot: (Snapshot, OptimisticTransactionImpl) = + private def getNextSnapshot: (Snapshot, DeltaTransaction) = asyncThreadLock.synchronized { val potentialSnapshotAndTxn = standbyConversion.get() currentConversion.set(potentialSnapshotAndTxn) @@ -212,7 +212,7 @@ class IcebergConverter(spark: SparkSession) * @return Converted Delta version and commit timestamp */ override def convertSnapshot( - snapshotToConvert: Snapshot, txn: OptimisticTransactionImpl): Option[(Long, Long)] = { + snapshotToConvert: Snapshot, txn: DeltaTransaction): Option[(Long, Long)] = { try { txn.catalogTable match { case Some(table) => convertSnapshotWithRetry(snapshotToConvert, Some(txn), table) @@ -242,7 +242,7 @@ class IcebergConverter(spark: SparkSession) */ private def convertSnapshotWithRetry( snapshotToConvert: Snapshot, - txnOpt: Option[OptimisticTransactionImpl], + txnOpt: Option[DeltaTransaction], catalogTable: CatalogTable, maxRetry: Int = spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_UNIFORM_ICEBERG_RETRY_TIMES) @@ -283,7 +283,7 @@ class IcebergConverter(spark: SparkSession) */ private def convertSnapshot( snapshotToConvert: Snapshot, - txnOpt: Option[OptimisticTransactionImpl], + txnOpt: Option[DeltaTransaction], catalogTable: CatalogTable): Option[(Long, Long)] = recordFrameProfile("Delta", "IcebergConverter.convertSnapshot") { val cleanedCatalogTable = @@ -433,7 +433,7 @@ class IcebergConverter(spark: SparkSession) private def cleanCatalogTableIfEnablingUniform( table: CatalogTable, snapshotToConvert: Snapshot, - txnOpt: Option[OptimisticTransactionImpl]): CatalogTable = { + txnOpt: Option[DeltaTransaction]): CatalogTable = { val disabledIceberg = txnOpt.map(txn => !UniversalFormat.icebergEnabled(txn.snapshot.metadata) ).getOrElse(!UniversalFormat.icebergEnabled(table.properties)) diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaTransaction.scala b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaTransaction.scala new file mode 100644 index 00000000000..cb7c5835fec --- /dev/null +++ b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaTransaction.scala @@ -0,0 +1,61 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.delta + +import scala.collection.mutable + +import org.apache.spark.sql.delta.actions.{AddFile, CommitInfo, Metadata, Protocol} +import org.apache.spark.sql.delta.hooks.PostCommitHook + +import org.apache.spark.sql.catalyst.catalog.{CatalogStatistics, CatalogTable} + +/** + * Represents a transaction that maps to a delta table commit. + * + * An instance of this trait tracks the reads and writes as well as accumulates additional + * information such as statistics of a single table throughout the life of a transaction. + */ +trait DeltaTransaction { + val deltaLog: DeltaLog + val catalogTable: Option[CatalogTable] + val snapshot: Snapshot + + /** Unique identifier for the transaction */ + val txnId: String + + /** + * Returns the metadata for this transaction. The metadata refers to the metadata of the snapshot + * at the transaction's read version unless updated during the transaction. + */ + def metadata: Metadata + + /** The protocol of the snapshot that this transaction is reading at. */ + def protocol: Protocol + + /** The end to end execution time of this transaction. */ + def txnExecutionTimeMs: Option[Long] + + /** Whether the txn should trigger a checkpoint after the commit */ + private[delta] var needsCheckpoint = false + + /** The set of distinct partitions that contain added files by current transaction. */ + protected[delta] var partitionsAddedToOpt: Option[mutable.HashSet[Map[String, String]]] = None + + /** True if this transaction is a blind append. This is only valid after commit. */ + protected[delta] var isBlindAppend: Boolean = false + +} diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala b/spark/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala index f7fccc37f0e..af57cbaac90 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala @@ -251,7 +251,8 @@ object OptimisticTransaction { * * This trait is not thread-safe. */ -trait OptimisticTransactionImpl extends TransactionalWrite +trait OptimisticTransactionImpl extends DeltaTransaction + with TransactionalWrite with SQLMetricsReporting with DeltaScanGenerator with RecordChecksum @@ -264,9 +265,6 @@ trait OptimisticTransactionImpl extends TransactionalWrite protected val incrementalCommitEnabled = deltaLog.incrementalCommitEnabled protected val shouldVerifyIncrementalCommit = deltaLog.shouldVerifyIncrementalCommit - val deltaLog: DeltaLog - val catalogTable: Option[CatalogTable] - val snapshot: Snapshot def clock: Clock = deltaLog.clock // This would be a quick operation if we already validated the checksum @@ -316,7 +314,7 @@ trait OptimisticTransactionImpl extends TransactionalWrite protected var newProtocol: Option[Protocol] = None /** The transaction start time. */ - protected val txnStartNano = System.nanoTime() + private val txnStartNano = System.nanoTime() override val snapshotToScan: Snapshot = snapshot @@ -334,9 +332,6 @@ trait OptimisticTransactionImpl extends TransactionalWrite protected var commitInfo: CommitInfo = _ - /** Whether the txn should trigger a checkpoint after the commit */ - private[delta] var needsCheckpoint = false - // Whether this transaction is creating a new table. private var isCreatingNewTable: Boolean = false @@ -396,7 +391,7 @@ trait OptimisticTransactionImpl extends TransactionalWrite def txnStartTimeNs: Long = txnStartNano /** Unique identifier for the transaction */ - val txnId = UUID.randomUUID().toString + val txnId: String = UUID.randomUUID().toString /** Whether to check unsupported data type when updating the table schema */ protected var checkUnsupportedDataType: Boolean = @@ -465,12 +460,6 @@ trait OptimisticTransactionImpl extends TransactionalWrite } } - /** The set of distinct partitions that contain added files by current transaction. */ - protected[delta] var partitionsAddedToOpt: Option[mutable.HashSet[Map[String, String]]] = None - - /** True if this transaction is a blind append. This is only valid after commit. */ - protected[delta] var isBlindAppend: Boolean = false - /** * The logSegment of the snapshot prior to the commit. * Will be updated only when retrying due to a conflict. diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/UniversalFormat.scala b/spark/src/main/scala/org/apache/spark/sql/delta/UniversalFormat.scala index 80b9294be8d..972e454e5a1 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/UniversalFormat.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/UniversalFormat.scala @@ -304,7 +304,7 @@ abstract class UniversalFormatConverter(spark: SparkSession) { */ def enqueueSnapshotForConversion( snapshotToConvert: Snapshot, - txn: OptimisticTransactionImpl): Unit + txn: DeltaTransaction): Unit /** * Perform a blocking conversion when performing an OptimisticTransaction @@ -317,7 +317,7 @@ abstract class UniversalFormatConverter(spark: SparkSession) { * @return Converted Delta version and commit timestamp */ def convertSnapshot( - snapshotToConvert: Snapshot, txn: OptimisticTransactionImpl): Option[(Long, Long)] + snapshotToConvert: Snapshot, txn: DeltaTransaction): Option[(Long, Long)] /** * Perform a blocking conversion for the given catalogTable diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/files/TransactionalWrite.scala b/spark/src/main/scala/org/apache/spark/sql/delta/files/TransactionalWrite.scala index a636ea79a11..f656370964d 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/files/TransactionalWrite.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/files/TransactionalWrite.scala @@ -54,14 +54,6 @@ import org.apache.spark.util.SerializableConfiguration */ trait TransactionalWrite extends DeltaLogging { self: OptimisticTransactionImpl => - def deltaLog: DeltaLog - - def protocol: Protocol - - protected def snapshot: Snapshot - - protected def metadata: Metadata - protected var hasWritten = false private[delta] val deltaDataSubdir = diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/hooks/AutoCompact.scala b/spark/src/main/scala/org/apache/spark/sql/delta/hooks/AutoCompact.scala index dcee76d0da9..5c3b9ff95c4 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/hooks/AutoCompact.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/hooks/AutoCompact.scala @@ -86,7 +86,7 @@ trait AutoCompactBase extends PostCommitHook with DeltaLogging { private[hooks] def shouldSkipAutoCompact( autoCompactTypeOpt: Option[AutoCompactType], spark: SparkSession, - txn: OptimisticTransactionImpl): Boolean = { + txn: DeltaTransaction): Boolean = { // If auto compact type is empty, then skip compaction if (autoCompactTypeOpt.isEmpty) return true @@ -99,7 +99,7 @@ trait AutoCompactBase extends PostCommitHook with DeltaLogging { override def run( spark: SparkSession, - txn: OptimisticTransactionImpl, + txn: DeltaTransaction, committedVersion: Long, postCommitSnapshot: Snapshot, actions: Seq[Action]): Unit = { @@ -122,7 +122,7 @@ trait AutoCompactBase extends PostCommitHook with DeltaLogging { */ private[delta] def compactIfNecessary( spark: SparkSession, - txn: OptimisticTransactionImpl, + txn: DeltaTransaction, postCommitSnapshot: Snapshot, opType: String, maxDeletedRowsRatio: Option[Double] diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/hooks/AutoCompactUtils.scala b/spark/src/main/scala/org/apache/spark/sql/delta/hooks/AutoCompactUtils.scala index c3a34969754..1a2eff54f70 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/hooks/AutoCompactUtils.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/hooks/AutoCompactUtils.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.delta.hooks import scala.collection.mutable // scalastyle:off import.ordering.noEmptyLine -import org.apache.spark.sql.delta.{DeltaLog, OptimisticTransactionImpl, Snapshot} +import org.apache.spark.sql.delta.{DeltaLog, DeltaTransaction, Snapshot} import org.apache.spark.sql.delta.metering.DeltaLogging import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.delta.sources.DeltaSQLConf._ @@ -311,7 +311,7 @@ object AutoCompactUtils extends DeltaLogging { */ def prepareAutoCompactRequest( spark: SparkSession, - txn: OptimisticTransactionImpl, + txn: DeltaTransaction, postCommitSnapshot: Snapshot, partitionsAddedToOpt: Option[PartitionKeySet], opType: String, @@ -337,7 +337,7 @@ object AutoCompactUtils extends DeltaLogging { */ def isQualifiedForAutoCompact( spark: SparkSession, - txn: OptimisticTransactionImpl): Boolean = { + txn: DeltaTransaction): Boolean = { // If txnExecutionTimeMs is empty, there is no transaction commit. if (txn.txnExecutionTimeMs.isEmpty) return false // If modified partitions only mode is not enabled, return true to avoid subsequent checking. diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/hooks/CheckpointHook.scala b/spark/src/main/scala/org/apache/spark/sql/delta/hooks/CheckpointHook.scala index 51993e230ed..a0279e44c38 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/hooks/CheckpointHook.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/hooks/CheckpointHook.scala @@ -16,7 +16,7 @@ package org.apache.spark.sql.delta.hooks -import org.apache.spark.sql.delta.{CheckpointInstance, OptimisticTransactionImpl, Snapshot} +import org.apache.spark.sql.delta.{DeltaTransaction, Snapshot} import org.apache.spark.sql.delta.actions.Action import org.apache.spark.sql.SparkSession @@ -27,7 +27,7 @@ object CheckpointHook extends PostCommitHook { override def run( spark: SparkSession, - txn: OptimisticTransactionImpl, + txn: DeltaTransaction, committedVersion: Long, postCommitSnapshot: Snapshot, committedActions: Seq[Action]): Unit = { diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/hooks/ChecksumHook.scala b/spark/src/main/scala/org/apache/spark/sql/delta/hooks/ChecksumHook.scala index d8230e74473..8c9d57dd6e7 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/hooks/ChecksumHook.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/hooks/ChecksumHook.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.delta.hooks // scalastyle:off import.ordering.noEmptyLine -import org.apache.spark.sql.delta.{DeltaLog, OptimisticTransactionImpl, RecordChecksum, Snapshot} +import org.apache.spark.sql.delta.{DeltaLog, DeltaTransaction, RecordChecksum, Snapshot} import org.apache.spark.sql.delta.actions.Action import org.apache.spark.sql.delta.logging.DeltaLogKeys import org.apache.spark.sql.delta.metering.DeltaLogging @@ -40,7 +40,7 @@ object ChecksumHook extends PostCommitHook with DeltaLogging { override def run( spark: SparkSession, - txn: OptimisticTransactionImpl, + txn: DeltaTransaction, committedVersion: Long, postCommitSnapshot: Snapshot, committedActions: Seq[Action]): Unit = { @@ -55,7 +55,7 @@ object ChecksumHook extends PostCommitHook with DeltaLogging { private def writeChecksum( spark: SparkSession, - txn: OptimisticTransactionImpl, + txn: DeltaTransaction, postCommitSnapshot: Snapshot): Unit = { WriteChecksum(spark, txn.deltaLog, txn.txnId, postCommitSnapshot) } diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/hooks/GenerateSymlinkManifest.scala b/spark/src/main/scala/org/apache/spark/sql/delta/hooks/GenerateSymlinkManifest.scala index 127dd328e2e..09d00a54e15 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/hooks/GenerateSymlinkManifest.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/hooks/GenerateSymlinkManifest.scala @@ -71,7 +71,7 @@ trait GenerateSymlinkManifestImpl extends PostCommitHook with DeltaLogging with override def run( spark: SparkSession, - txn: OptimisticTransactionImpl, + txn: DeltaTransaction, committedVersion: Long, postCommitSnapshot: Snapshot, committedActions: Seq[Action]): Unit = { @@ -93,7 +93,7 @@ trait GenerateSymlinkManifestImpl extends PostCommitHook with DeltaLogging with */ protected def generateIncrementalManifest( spark: SparkSession, - txn: OptimisticTransactionImpl, + txn: DeltaTransaction, currentSnapshot: Snapshot, actions: Seq[Action]): Unit = recordManifestGeneration(txn.deltaLog, full = false) { diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/hooks/HudiConverterHook.scala b/spark/src/main/scala/org/apache/spark/sql/delta/hooks/HudiConverterHook.scala index a35e6c8afd6..435c1609bc0 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/hooks/HudiConverterHook.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/hooks/HudiConverterHook.scala @@ -16,7 +16,7 @@ package org.apache.spark.sql.delta.hooks -import org.apache.spark.sql.delta.{OptimisticTransactionImpl, Snapshot, UniversalFormat} +import org.apache.spark.sql.delta.{DeltaTransaction, Snapshot, UniversalFormat} import org.apache.spark.sql.delta.actions.Action import org.apache.spark.sql.delta.metering.DeltaLogging import org.apache.spark.sql.delta.sources.DeltaSQLConf.DELTA_UNIFORM_HUDI_SYNC_CONVERT_ENABLED @@ -31,7 +31,7 @@ object HudiConverterHook extends PostCommitHook with DeltaLogging { override def run( spark: SparkSession, - txn: OptimisticTransactionImpl, + txn: DeltaTransaction, committedVersion: Long, postCommitSnapshot: Snapshot, committedActions: Seq[Action]): Unit = { diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/hooks/IcebergConverterHook.scala b/spark/src/main/scala/org/apache/spark/sql/delta/hooks/IcebergConverterHook.scala index 261f562e577..f716e817917 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/hooks/IcebergConverterHook.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/hooks/IcebergConverterHook.scala @@ -16,9 +16,8 @@ package org.apache.spark.sql.delta.hooks -import org.apache.spark.sql.delta.{OptimisticTransactionImpl, Snapshot, UniversalFormat} -import org.apache.spark.sql.delta.DeltaErrors -import org.apache.spark.sql.delta.actions.{Action, Metadata} +import org.apache.spark.sql.delta.{DeltaErrors, DeltaTransaction, Snapshot, UniversalFormat} +import org.apache.spark.sql.delta.actions.Action import org.apache.spark.sql.delta.metering.DeltaLogging import org.apache.spark.sql.delta.sources.DeltaSQLConf.DELTA_UNIFORM_ICEBERG_SYNC_CONVERT_ENABLED import org.apache.commons.lang3.exception.ExceptionUtils @@ -33,7 +32,7 @@ object IcebergConverterHook extends PostCommitHook with DeltaLogging { override def run( spark: SparkSession, - txn: OptimisticTransactionImpl, + txn: DeltaTransaction, committedVersion: Long, postCommitSnapshot: Snapshot, committedActions: Seq[Action]): Unit = { diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/hooks/PostCommitHook.scala b/spark/src/main/scala/org/apache/spark/sql/delta/hooks/PostCommitHook.scala index 86d526e11a6..d4676bbdc3a 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/hooks/PostCommitHook.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/hooks/PostCommitHook.scala @@ -43,7 +43,7 @@ trait PostCommitHook { */ def run( spark: SparkSession, - txn: OptimisticTransactionImpl, + txn: DeltaTransaction, committedVersion: Long, postCommitSnapshot: Snapshot, committedActions: Seq[Action]): Unit diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/hooks/UpdateCatalog.scala b/spark/src/main/scala/org/apache/spark/sql/delta/hooks/UpdateCatalog.scala index 6cccaf44a74..77f607769f5 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/hooks/UpdateCatalog.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/hooks/UpdateCatalog.scala @@ -26,7 +26,7 @@ import scala.util.control.NonFatal import org.apache.spark.sql.delta.skipping.clustering.{ClusteredTableUtils, ClusteringColumnInfo} import org.apache.spark.sql.delta.skipping.clustering.temp.ClusterBySpec -import org.apache.spark.sql.delta.{DeltaConfigs, DeltaTableIdentifier, OptimisticTransactionImpl, Snapshot} +import org.apache.spark.sql.delta.{DeltaConfigs, DeltaTableIdentifier, DeltaTransaction, Snapshot} import org.apache.spark.sql.delta.actions.{Action, Metadata} import org.apache.spark.sql.delta.logging.DeltaLogKeys import org.apache.spark.sql.delta.metering.DeltaLogging @@ -64,7 +64,7 @@ trait UpdateCatalogBase extends PostCommitHook with DeltaLogging { override def run( spark: SparkSession, - txn: OptimisticTransactionImpl, + txn: DeltaTransaction, committedVersion: Long, postCommitSnapshot: Snapshot, actions: Seq[Action]): Unit = { diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaErrorsSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaErrorsSuite.scala index a274172bbe3..7c570a12c8c 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaErrorsSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaErrorsSuite.scala @@ -902,7 +902,7 @@ trait DeltaErrorsSuiteBase throw DeltaErrors.postCommitHookFailedException(new PostCommitHook() { override val name: String = "DummyPostCommitHook" override def run( - spark: SparkSession, txn: OptimisticTransactionImpl, committedVersion: Long, + spark: SparkSession, txn: DeltaTransaction, committedVersion: Long, postCommitSnapshot: Snapshot, committedActions: Seq[Action]): Unit = {} }, 0, "msg", null) } @@ -915,7 +915,7 @@ trait DeltaErrorsSuiteBase throw DeltaErrors.postCommitHookFailedException(new PostCommitHook() { override val name: String = "DummyPostCommitHook" override def run( - spark: SparkSession, txn: OptimisticTransactionImpl, committedVersion: Long, + spark: SparkSession, txn: DeltaTransaction, committedVersion: Long, postCommitSnapshot: Snapshot, committedActions: Seq[Action]): Unit = {} }, 0, null, null) } From 6ba5c7f01d81998cc79b52bbec88d1f3e62470fa Mon Sep 17 00:00:00 2001 From: Scott Sandre Date: Thu, 30 Jan 2025 14:44:28 -0800 Subject: [PATCH 27/28] [Kernel] [Refactor] Remove superfluous engine param from ScanBuilder APIs (#4106) #### Which Delta project/connector is this regarding? - [ ] Spark - [ ] Standalone - [ ] Flink - [X] Kernel - [ ] Other (fill in here) ## Description Remove superfluous engine param from ScanBuilder APIs ## How was this patch tested? Refactor only. ## Does this PR introduce _any_ user-facing changes? Removes param from public API. --- docs/source/delta-kernel-java.md | 15 +++------ docs/source/delta-kernel.md | 4 +-- kernel/USER_GUIDE.md | 13 +++----- .../examples/MultiThreadedTableReader.java | 4 +-- .../examples/SingleThreadedTableReader.java | 4 +-- .../java/io/delta/kernel/ScanBuilder.java | 6 ++-- .../kernel/internal/ScanBuilderImpl.java | 5 ++- .../io/delta/kernel/utils/PartitionUtils.java | 2 +- .../io/delta/kernel/defaults/ScanSuite.scala | 32 ++++++------------- .../kernel/defaults/utils/TestUtils.scala | 6 ++-- 10 files changed, 33 insertions(+), 58 deletions(-) diff --git a/docs/source/delta-kernel-java.md b/docs/source/delta-kernel-java.md index a5b3494129f..eb1bf84d442 100644 --- a/docs/source/delta-kernel-java.md +++ b/docs/source/delta-kernel-java.md @@ -99,14 +99,14 @@ Snapshot mySnapshot = myTable.getLatestSnapshot(myEngine); Now that we have a consistent snapshot view of the table, we can query more details about the table. For example, you can get the version and schema of this snapshot. ```java -long version = mySnapshot.getVersion(myEngine); -StructType tableSchema = mySnapshot.getSchema(myEngine); +long version = mySnapshot.getVersion(); +StructType tableSchema = mySnapshot.getSchema(); ``` Next, to read the table data, we have to *build* a [`Scan`](https://delta-io.github.io/delta/snapshot/kernel-api/java/io/delta/kernel/Scan.html) object. In order to build a `Scan` object, create a [`ScanBuilder`](https://delta-io.github.io/delta/snapshot/kernel-api/java/io/delta/kernel/ScanBuilder.html) object which optionally allows selecting a subset of columns to read or setting a query filter. For now, ignore these optional settings. ```java -Scan myScan = mySnapshot.getScanBuilder(myEngine).build() +Scan myScan = mySnapshot.getScanBuilder().build() // Common information about scanning for all data files to read. Row scanState = myScan.getScanState(myEngine) @@ -224,9 +224,7 @@ Predicate filter = new Predicate( "=", Arrays.asList(new Column("columnX"), Literal.ofInt(1))); -Scan myFilteredScan = mySnapshot.buildScan(engine) - .withFilter(myEngine, filter) - .build() +Scan myFilteredScan = mySnapshot.getScanBuilder().withFilter(filter).build() // Subset of the given filter that is not guaranteed to be satisfied by // Delta Kernel when it returns data. This filter is used by Delta Kernel @@ -845,10 +843,7 @@ import io.delta.kernel.types.*; StructType readSchema = ... ; // convert engine schema Predicate filterExpr = ... ; // convert engine filter expression -Scan myScan = mySnapshot.buildScan(engine) - .withFilter(myEngine, filterExpr) - .withReadSchema(myEngine, readSchema) - .build() +Scan myScan = mySnapshot.getScanBuilder().withFilter(filterExpr).withReadSchema(readSchema).build(); ``` diff --git a/docs/source/delta-kernel.md b/docs/source/delta-kernel.md index 25738edc3ec..cdbca5807bf 100644 --- a/docs/source/delta-kernel.md +++ b/docs/source/delta-kernel.md @@ -22,8 +22,8 @@ Here is an example of a simple table scan with a filter: Engine myEngine = DefaultEngine.create() ; // define a engine (more details below) Table myTable = Table.forPath("/delta/table/path"); // define what table to scan Snapshot mySnapshot = myTable.getLatestSnapshot(myEngine); // define which version of table to scan -Scan myScan = mySnapshot.getScanBuilder(myEngine) // specify the scan details - .withFilters(myEngine, scanFilter) +Scan myScan = mySnapshot.getScanBuilder() // specify the scan details + .withFilters(scanFilter) .build(); CloseableIterator physicalData = // read the Parquet data files .. read from Parquet data files ... diff --git a/kernel/USER_GUIDE.md b/kernel/USER_GUIDE.md index 41231ab7a14..f3a635af41b 100644 --- a/kernel/USER_GUIDE.md +++ b/kernel/USER_GUIDE.md @@ -95,8 +95,8 @@ Snapshot mySnapshot = myTable.getLatestSnapshot(myEngine); Now that we have a consistent snapshot view of the table, we can query more details about the table. For example, you can get the version and schema of this snapshot. ```java -long version = mySnapshot.getVersion(myEngine); -StructType tableSchema = mySnapshot.getSchema(myEngine); +long version = mySnapshot.getVersion(); +StructType tableSchema = mySnapshot.getSchema(); ``` Next, to read the table data, we have to *build* a [`Scan`](https://delta-io.github.io/delta/snapshot/kernel-api/java/io/delta/kernel/Scan.html) object. In order to build a `Scan` object, create a [`ScanBuilder`](https://delta-io.github.io/delta/snapshot/kernel-api/java/io/delta/kernel/ScanBuilder.html) object which optionally allows selecting a subset of columns to read or setting a query filter. For now, ignore these optional settings. @@ -220,9 +220,7 @@ Predicate filter = new Predicate( "=", Arrays.asList(new Column("columnX"), Literal.ofInt(1))); -Scan myFilteredScan = mySnapshot.buildScan(engine) - .withFilter(myEngine, filter) - .build() +Scan myFilteredScan = mySnapshot.getScanBuilder().withFilter(filter).build() // Subset of the given filter that is not guaranteed to be satisfied by // Delta Kernel when it returns data. This filter is used by Delta Kernel @@ -865,10 +863,7 @@ import io.delta.kernel.types.*; StructType readSchema = ... ; // convert engine schema Predicate filterExpr = ... ; // convert engine filter expression -Scan myScan = mySnapshot.buildScan(engine) - .withFilter(myEngine, filterExpr) - .withReadSchema(myEngine, readSchema) - .build() +Scan myScan = mySnapshot.getScanBuilder().withFilter(filterExpr).withReadSchema(readSchema).build(); ``` diff --git a/kernel/examples/kernel-examples/src/main/java/io/delta/kernel/examples/MultiThreadedTableReader.java b/kernel/examples/kernel-examples/src/main/java/io/delta/kernel/examples/MultiThreadedTableReader.java index e5eefd668b0..7d450bab83e 100644 --- a/kernel/examples/kernel-examples/src/main/java/io/delta/kernel/examples/MultiThreadedTableReader.java +++ b/kernel/examples/kernel-examples/src/main/java/io/delta/kernel/examples/MultiThreadedTableReader.java @@ -85,10 +85,10 @@ public int show(int limit, Optional> columnsOpt, Optional> columnsOpt, OptionalreadSchema. If the builder already has a projection applied, calling * this again replaces the existing projection. * - * @param engine {@link Engine} instance to use in Delta Kernel. * @param readSchema Subset of columns to read from the Delta table. * @return A {@link ScanBuilder} with projection pruning. */ - ScanBuilder withReadSchema(Engine engine, StructType readSchema); + ScanBuilder withReadSchema(StructType readSchema); /** @return Build the {@link Scan instance} */ Scan build(); diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/ScanBuilderImpl.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/ScanBuilderImpl.java index 4aa5b1c1e2b..d725f8dbf4c 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/ScanBuilderImpl.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/ScanBuilderImpl.java @@ -18,7 +18,6 @@ import io.delta.kernel.Scan; import io.delta.kernel.ScanBuilder; -import io.delta.kernel.engine.Engine; import io.delta.kernel.expressions.Predicate; import io.delta.kernel.internal.actions.Metadata; import io.delta.kernel.internal.actions.Protocol; @@ -55,7 +54,7 @@ public ScanBuilderImpl( } @Override - public ScanBuilder withFilter(Engine engine, Predicate predicate) { + public ScanBuilder withFilter(Predicate predicate) { if (this.predicate.isPresent()) { throw new IllegalArgumentException("There already exists a filter in current builder"); } @@ -64,7 +63,7 @@ public ScanBuilder withFilter(Engine engine, Predicate predicate) { } @Override - public ScanBuilder withReadSchema(Engine engine, StructType readSchema) { + public ScanBuilder withReadSchema(StructType readSchema) { // TODO: validate the readSchema is a subset of the table schema this.readSchema = readSchema; return this; diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/utils/PartitionUtils.java b/kernel/kernel-api/src/main/java/io/delta/kernel/utils/PartitionUtils.java index 33fa2af6d68..1877100dc19 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/utils/PartitionUtils.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/utils/PartitionUtils.java @@ -55,7 +55,7 @@ public static boolean partitionExists( io.delta.kernel.internal.util.PartitionUtils.validatePredicateOnlyOnPartitionColumns( partitionPredicate, snapshotPartColNames); - final Scan scan = snapshot.getScanBuilder().withFilter(engine, partitionPredicate).build(); + final Scan scan = snapshot.getScanBuilder().withFilter(partitionPredicate).build(); try (CloseableIterator columnarBatchIter = scan.getScanFiles(engine)) { while (columnarBatchIter.hasNext()) { diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ScanSuite.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ScanSuite.scala index 5e01230fcf7..49992ce6ecc 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ScanSuite.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/ScanSuite.scala @@ -97,15 +97,13 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with val snapshot = latestSnapshot(tablePath) hits.foreach { predicate => val scanFiles = collectScanFileRows( - snapshot.getScanBuilder() - .withFilter(defaultEngine, predicate) - .build()) + snapshot.getScanBuilder().withFilter(predicate).build()) assert(scanFiles.nonEmpty, s"Expected hit but got miss for $predicate") } misses.foreach { predicate => val scanFiles = collectScanFileRows( snapshot.getScanBuilder() - .withFilter(defaultEngine, predicate) + .withFilter(predicate) .build()) assert(scanFiles.isEmpty, s"Expected miss but got hit for $predicate\n" + s"Returned scan files have stats: ${getScanFileStats(scanFiles)}" @@ -121,9 +119,7 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with val snapshot = latestSnapshot(tablePath) filterToNumExpFiles.foreach { case (filter, numExpFiles) => val scanFiles = collectScanFileRows( - snapshot.getScanBuilder() - .withFilter(defaultEngine, filter) - .build()) + snapshot.getScanBuilder().withFilter(filter).build()) assert(scanFiles.length == numExpFiles, s"Expected $numExpFiles but found ${scanFiles.length} for $filter") } @@ -1010,7 +1006,7 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with predicate: Predicate, expNumPartitions: Int, expNumFiles: Long): Unit = { val snapshot = latestSnapshot(tableDir.getCanonicalPath) val scanFiles = collectScanFileRows( - snapshot.getScanBuilder().withFilter(defaultEngine, predicate).build()) + snapshot.getScanBuilder().withFilter(predicate).build()) assert(scanFiles.length == expNumFiles, s"Expected $expNumFiles but found ${scanFiles.length} for $predicate") @@ -1496,7 +1492,7 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with val partFilter = equals(new Column("part"), ofInt(1)) verifyNoStatsColumn( snapshot(engineDisallowedStatsReads) - .getScanBuilder().withFilter(engine, partFilter).build() + .getScanBuilder().withFilter(partFilter).build() .getScanFiles(engine)) // no eligible data skipping filter --> don't read stats @@ -1505,7 +1501,7 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with ofInt(1)) verifyNoStatsColumn( snapshot(engineDisallowedStatsReads) - .getScanBuilder().withFilter(engine, nonEligibleFilter).build() + .getScanBuilder().withFilter(nonEligibleFilter).build() .getScanFiles(engine)) } @@ -1543,9 +1539,7 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with val engine = engineVerifyJsonParseSchema(verifySchema(expectedCols)) collectScanFileRows( Table.forPath(engine, path).getLatestSnapshot(engine) - .getScanBuilder() - .withFilter(engine, predicate) - .build(), + .getScanBuilder().withFilter(predicate).build(), engine = engine) } } @@ -1573,7 +1567,6 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with latestSnapshot(tempDir.getCanonicalPath) .getScanBuilder() .withFilter( - defaultEngine, greaterThan( new ScalarExpression("+", Seq(col("id"), ofInt(10)).asJava), ofInt(100) @@ -1584,13 +1577,8 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with checkStatsPresent( latestSnapshot(tempDir.getCanonicalPath) .getScanBuilder() - .withFilter( - defaultEngine, - greaterThan( - col("id"), - ofInt(0) - ) - ).build() + .withFilter(greaterThan(col("id"), ofInt(0))) + .build() ) } } @@ -1634,7 +1622,7 @@ class ScanSuite extends AnyFunSuite with TestUtils with ExpressionTestUtils with val scanBuilder = snapshot.getScanBuilder() val scan = predicate match { - case Some(pred) => scanBuilder.withFilter(defaultEngine, pred).build() + case Some(pred) => scanBuilder.withFilter(pred).build() case None => scanBuilder.build() } val scanFiles = scan.asInstanceOf[ScanImpl].getScanFiles(defaultEngine, true) diff --git a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/utils/TestUtils.scala b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/utils/TestUtils.scala index ee73a0ce38f..8b32c392cd7 100644 --- a/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/utils/TestUtils.scala +++ b/kernel/kernel-defaults/src/test/scala/io/delta/kernel/defaults/utils/TestUtils.scala @@ -197,11 +197,11 @@ trait TestUtils extends Assertions with SQLHelper { var scanBuilder = snapshot.getScanBuilder() if (readSchema != null) { - scanBuilder = scanBuilder.withReadSchema(engine, readSchema) + scanBuilder = scanBuilder.withReadSchema(readSchema) } if (filter != null) { - scanBuilder = scanBuilder.withFilter(engine, filter) + scanBuilder = scanBuilder.withFilter(filter) } val scan = scanBuilder.build() @@ -265,7 +265,7 @@ trait TestUtils extends Assertions with SQLHelper { val scan = Table.forPath(engine, tablePath) .getLatestSnapshot(engine) .getScanBuilder() - .withReadSchema(engine, readSchema) + .withReadSchema(readSchema) .build() val scanState = scan.getScanState(engine) From f32539d9ba0cff3118ed74e130cd0119365d85d0 Mon Sep 17 00:00:00 2001 From: Scott Sandre Date: Thu, 30 Jan 2025 14:44:49 -0800 Subject: [PATCH 28/28] [Kernel] Add better param validations to LogSegment constructor (#4105) #### Which Delta project/connector is this regarding? - [ ] Spark - [ ] Standalone - [ ] Flink - [X] Kernel - [ ] Other (fill in here) ## Description Add better validations to LogSegment ## How was this patch tested? New UTs. ## Does this PR introduce _any_ user-facing changes? No. --- .../kernel/internal/snapshot/LogSegment.java | 66 +++++++++- .../internal/snapshot/SnapshotManager.java | 1 - .../internal/snapshot/LogSegmentSuite.scala | 117 ++++++++++++++++++ 3 files changed, 178 insertions(+), 6 deletions(-) create mode 100644 kernel/kernel-api/src/test/scala/io/delta/kernel/internal/snapshot/LogSegmentSuite.scala diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/LogSegment.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/LogSegment.java index 7abde22225e..1e710ec2281 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/LogSegment.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/LogSegment.java @@ -16,8 +16,13 @@ package io.delta.kernel.internal.snapshot; +import static io.delta.kernel.internal.util.Preconditions.checkArgument; +import static java.util.Objects.requireNonNull; + import io.delta.kernel.internal.fs.Path; import io.delta.kernel.internal.lang.Lazy; +import io.delta.kernel.internal.lang.ListUtils; +import io.delta.kernel.internal.util.FileNames; import io.delta.kernel.utils.FileStatus; import java.util.Collections; import java.util.Comparator; @@ -38,8 +43,7 @@ public class LogSegment { private final Lazy> allFilesReversed; public static LogSegment empty(Path logPath) { - return new LogSegment( - logPath, -1, Collections.emptyList(), Collections.emptyList(), Optional.empty(), -1); + return new LogSegment(logPath, -1, Collections.emptyList(), Collections.emptyList(), -1); } /** @@ -50,7 +54,6 @@ public static LogSegment empty(Path logPath) { * @param version The Snapshot version to generate * @param deltas The delta commit files (.json) to read * @param checkpoints The checkpoint file(s) to read - * @param checkpointVersionOpt The checkpoint version used to start replay * @param lastCommitTimestamp The "unadjusted" timestamp of the last commit within this segment. * By unadjusted, we mean that the commit timestamps may not necessarily be monotonically * increasing for the commits within this segment. @@ -60,13 +63,66 @@ public LogSegment( long version, List deltas, List checkpoints, - Optional checkpointVersionOpt, long lastCommitTimestamp) { + + /////////////////////// + // Input validations // + /////////////////////// + + requireNonNull(logPath, "logPath is null"); + requireNonNull(deltas, "deltas is null"); + requireNonNull(checkpoints, "checkpoints is null"); + checkArgument( + deltas.stream().allMatch(fs -> FileNames.isCommitFile(fs.getPath())), + "deltas must all be actual delta (commit) files"); + checkArgument( + checkpoints.stream().allMatch(fs -> FileNames.isCheckpointFile(fs.getPath())), + "checkpoints must all be actual checkpoint files"); + + this.checkpointVersionOpt = + checkpoints.isEmpty() + ? Optional.empty() + : Optional.of(FileNames.checkpointVersion(new Path(checkpoints.get(0).getPath()))); + + checkArgument( + checkpoints.stream() + .map(fs -> FileNames.checkpointVersion(new Path(fs.getPath()))) + .allMatch(v -> checkpointVersionOpt.get().equals(v)), + "All checkpoint files must have the same version"); + + if (version != -1) { + checkArgument(!deltas.isEmpty() || !checkpoints.isEmpty(), "No files to read"); + + if (!deltas.isEmpty()) { + this.checkpointVersionOpt.ifPresent( + checkpointVersion -> { + checkArgument( + FileNames.deltaVersion(deltas.get(0).getPath()) == checkpointVersion + 1, + "First delta file version must equal checkpointVersion + 1"); + }); + + checkArgument( + FileNames.deltaVersion(ListUtils.getLast(deltas).getPath()) == version, + "Last delta file version must equal the version of this LogSegment"); + } else { + this.checkpointVersionOpt.ifPresent( + checkpointVersion -> { + checkArgument( + checkpointVersion == version, + "If there are no deltas, then checkpointVersion must equal the version " + + "of this LogSegment"); + }); + } + } + + //////////////////////////////// + // Member variable assignment // + //////////////////////////////// + this.logPath = logPath; this.version = version; this.deltas = deltas; this.checkpoints = checkpoints; - this.checkpointVersionOpt = checkpointVersionOpt; this.lastCommitTimestamp = lastCommitTimestamp; this.allFiles = diff --git a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java index eabac72e5bc..fa5433bb39e 100644 --- a/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java +++ b/kernel/kernel-api/src/main/java/io/delta/kernel/internal/snapshot/SnapshotManager.java @@ -562,7 +562,6 @@ public LogSegment getLogSegmentForVersion(Engine engine, Optional versionT newVersion, deltasAfterCheckpoint, latestCompleteCheckpointFileStatuses, - latestCompleteCheckpointOpt.map(x -> x.version), lastCommitTimestamp); } diff --git a/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/snapshot/LogSegmentSuite.scala b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/snapshot/LogSegmentSuite.scala new file mode 100644 index 00000000000..a3134d60360 --- /dev/null +++ b/kernel/kernel-api/src/test/scala/io/delta/kernel/internal/snapshot/LogSegmentSuite.scala @@ -0,0 +1,117 @@ +/* + * Copyright (2025) The Delta Lake Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.delta.kernel.internal.snapshot + +import java.util.Arrays +import java.util.{Collections, Optional} + +import io.delta.kernel.internal.fs.Path +import io.delta.kernel.internal.util.FileNames +import io.delta.kernel.utils.FileStatus +import org.scalatest.funsuite.AnyFunSuite + +class LogSegmentSuite extends AnyFunSuite { + private val logPath = new Path("/a/_delta_log") + private val checkpointFs10 = + FileStatus.of(FileNames.checkpointFileSingular(logPath, 10).toString, 1, 1) + private val checkpointFs10List = Collections.singletonList(checkpointFs10) + private val deltaFs11 = FileStatus.of(FileNames.deltaFile(logPath, 11), 1, 1) + private val deltaFs11List = Collections.singletonList(deltaFs11) + private val deltaFs12 = FileStatus.of(FileNames.deltaFile(logPath, 12), 1, 1) + private val deltaFs12List = Collections.singletonList(deltaFs12) + private val deltasFs11To12List = Arrays.asList(deltaFs11, deltaFs12) + private val badJsonsList = Collections.singletonList( + FileStatus.of(s"${logPath.toString}/gibberish.json", 1, 1)) + private val badCheckpointsList = Collections.singletonList( + FileStatus.of(s"${logPath.toString}/gibberish.checkpoint.parquet", 1, 1)) + + test("constructor -- valid case (empty)") { + LogSegment.empty(new Path("/a/_delta_log")) + } + + test("constructor -- valid case (non-empty)") { + val logPath = new Path("/a/_delta_log") + new LogSegment(logPath, 12, deltasFs11To12List, checkpointFs10List, 1) + } + + test("constructor -- null arguments => throw") { + // logPath is null + intercept[NullPointerException] { + new LogSegment( + null, 1, Collections.emptyList(), Collections.emptyList(), -1) + } + // deltas is null + intercept[NullPointerException] { + new LogSegment( + new Path("/a/_delta_log"), 1, null, Collections.emptyList(), -1) + } + // checkpoints is null + intercept[NullPointerException] { + new LogSegment( + new Path("/a/_delta_log"), 1, Collections.emptyList(), null, -1) + } + } + + test("constructor -- all deltas must be actual delta files") { + val exMsg = intercept[IllegalArgumentException] { + new LogSegment( + logPath, 12, badJsonsList, checkpointFs10List, 1) + }.getMessage + assert(exMsg === "deltas must all be actual delta (commit) files") + } + + test("constructor -- all checkpoints must be actual checkpoint files") { + val exMsg = intercept[IllegalArgumentException] { + new LogSegment( + logPath, 12, deltasFs11To12List, badCheckpointsList, 1) + }.getMessage + assert(exMsg === "checkpoints must all be actual checkpoint files") + } + + test("constructor -- if version >= 0 then both deltas and checkpoints cannot be empty") { + val exMsg = intercept[IllegalArgumentException] { + new LogSegment( + logPath, 12, Collections.emptyList(), Collections.emptyList(), 1) + }.getMessage + assert(exMsg === "No files to read") + } + + test("constructor -- if deltas non-empty then first delta must equal checkpointVersion + 1") { + val exMsg = intercept[IllegalArgumentException] { + new LogSegment( + logPath, 12, deltaFs12List, checkpointFs10List, 1) + }.getMessage + assert(exMsg === "First delta file version must equal checkpointVersion + 1") + } + + test("constructor -- if deltas non-empty then last delta must equal version") { + val exMsg = intercept[IllegalArgumentException] { + new LogSegment( + logPath, 12, deltaFs11List, checkpointFs10List, 1) + }.getMessage + assert(exMsg === "Last delta file version must equal the version of this LogSegment") + } + + test("constructor -- if no deltas then checkpointVersion must equal version") { + val exMsg = intercept[IllegalArgumentException] { + new LogSegment( + logPath, 11, Collections.emptyList(), checkpointFs10List, 1) + }.getMessage + assert(exMsg === + "If there are no deltas, then checkpointVersion must equal the version of this LogSegment") + } +}