diff --git a/build.sbt b/build.sbt
index 9270f5fb6cb..85088b71036 100644
--- a/build.sbt
+++ b/build.sbt
@@ -401,7 +401,7 @@ lazy val spark = (project in file("spark"))
"org.apache.spark" %% "spark-sql" % sparkVersion.value % "provided",
"org.apache.spark" %% "spark-core" % sparkVersion.value % "provided",
"org.apache.spark" %% "spark-catalyst" % sparkVersion.value % "provided",
- // For DynamoDBCommitStore
+ // For DynamoDBCommitCoordinator
"com.amazonaws" % "aws-java-sdk" % "1.12.262" % "provided",
// Test deps
diff --git a/connectors/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala b/connectors/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala
index 585aa06e5ae..94178fa6fd5 100644
--- a/connectors/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala
+++ b/connectors/standalone/src/main/scala/io/delta/standalone/internal/OptimisticTransactionImpl.scala
@@ -527,7 +527,9 @@ private[internal] class OptimisticTransactionImpl(
private def shouldCheckpoint(committedVersion: Long): Boolean = {
val checkpointingEnabled =
deltaLog.hadoopConf.getBoolean(StandaloneHadoopConf.CHECKPOINTING_ENABLED, true)
- checkpointingEnabled && committedVersion != 0 && committedVersion % deltaLog.checkpointInterval == 0
+ checkpointingEnabled &&
+ committedVersion != 0 &&
+ committedVersion % deltaLog.checkpointInterval == 0
}
/** Returns the next attempt version given the last attempted version */
diff --git a/protocol_rfcs/README.md b/protocol_rfcs/README.md
index f42282bf10d..fcaad710a8e 100644
--- a/protocol_rfcs/README.md
+++ b/protocol_rfcs/README.md
@@ -20,7 +20,7 @@ Here is the history of all the RFCs propose/accepted/rejected since Feb 6, 2024,
|:--------------|:---------------------------------------------------------------------------------------------------------------------------------|:----------------------------------------------|:---------------------------------------|
| 2023-02-02 | [in-commit-timestamps.md](https://github.com/delta-io/delta/blob/master/protocol_rfcs/in-commit-timestamps.md) | https://github.com/delta-io/delta/issues/2532 | In-Commit Timestamps |
| 2023-02-09 | [type-widening.md](https://github.com/delta-io/delta/blob/master/protocol_rfcs/type-widening.md) | https://github.com/delta-io/delta/issues/2623 | Type Widening |
-| 2023-02-14 | [managed-commits.md](https://github.com/delta-io/delta/blob/master/protocol_rfcs/managed-commits.md) | https://github.com/delta-io/delta/issues/2598 | Managed Commits |
+| 2023-02-14 | [coordinated-commits.md](https://github.com/delta-io/delta/blob/master/protocol_rfcs/coordinated-commits.md) | https://github.com/delta-io/delta/issues/2598 | Coordinated Commits |
| 2023-02-26 | [column-mapping-usage.tracking.md](https://github.com/delta-io/delta/blob/master/protocol_rfcs/column-mapping-usage-tracking.md) | https://github.com/delta-io/delta/issues/2682 | Column Mapping Usage Tracking |
| 2023-04-24 | [variant-type.md](https://github.com/delta-io/delta/blob/master/protocol_rfcs/variant-type.md) | https://github.com/delta-io/delta/issues/2864 | Variant Data Type |
| 2024-04-30 | [collated-string-type.md](https://github.com/delta-io/delta/blob/master/protocol_rfcs/collated-string-type.md) | https://github.com/delta-io/delta/issues/2894 | Collated String Type |
diff --git a/protocol_rfcs/managed-commits.md b/protocol_rfcs/coordinated-commits.md
similarity index 57%
rename from protocol_rfcs/managed-commits.md
rename to protocol_rfcs/coordinated-commits.md
index ee35e7bf315..730cb9ceaa1 100644
--- a/protocol_rfcs/managed-commits.md
+++ b/protocol_rfcs/coordinated-commits.md
@@ -1,9 +1,9 @@
-# Managed Commits
+# Coordinated Commits
**Associated Github issue for discussions: https://github.com/delta-io/delta/issues/2598**
-This RFC proposes a new table feature `managedCommit` which changes the way Delta Lake performs commits.
+This RFC proposes a new table feature `coordinatedCommits` which changes the way Delta Lake performs commits.
-Today’s Delta commit protocol relies on the filesystem to provide commit atomicity. This feature request is to allow Delta tables which gets commit atomicity using an external commit-owner and not
+Today’s Delta commit protocol relies on the filesystem to provide commit atomicity. This feature request is to allow Delta tables which gets commit atomicity using an external commit-coordinator and not
the filesystem (s3, abfs etc). This allows us to deal with various limitations of Delta:
1. No reliable way for the table's owner to participate in commits.
@@ -11,7 +11,7 @@ the filesystem (s3, abfs etc). This allows us to deal with various limitations o
- No clear path to transactions that could span multiple tables and/or involve catalog updates, because filesystem commits cannot be made conditionally or atomically.
2. No way to tie commit ownership to a table.
- In general, Delta tables have no way to advertise that they are managed by catalog or LogStore X (at endpoint Y).
- - No way to express different commit owners for different tables. For example, Delta spark supports a notion of a "[log store](https://delta.io/blog/2022-05-18-multi-cluster-writes-to-delta-lake-storage-in-s3/)" or commit service for enforcing commit atomicity in S3, but it's a cluster-level setting that affects all tables indiscriminately, with no way to validate whether the mapping is even correct.
+ - No way to express different commit coordinators for different tables. For example, Delta spark supports a notion of a "[log store](https://delta.io/blog/2022-05-18-multi-cluster-writes-to-delta-lake-storage-in-s3/)" or commit service for enforcing commit atomicity in S3, but it's a cluster-level setting that affects all tables indiscriminately, with no way to validate whether the mapping is even correct.
- There is no central entity that needs to be contacted in order to commit to the table. So if the underlying file system is missing _putIfAbsent_ semantics, then there is no way to ensure that a commit is atomic, which could lead
to lost writes when concurrent writers are writing to the table.
@@ -26,8 +26,8 @@ Delta files are stored as JSON in a directory at the root of the table named `_d
~~Delta files are the unit of atomicity for a table, and are named using the next available version number, zero-padded to 20 digits.~~
They are the unit of atomicity for a table.
-**Note:** If [managed commits](#managed-commits) table feature is enabled on the table, recently committed delta files may reside in the `_delta_log/_commits` directory. Delta clients have to contact
-the corresponding commit-owner of the table in order to find the information about the [un-backfilled commits](#commit-backfills).
+**Note:** If [coordinated commits](#coordinated-commits) table feature is enabled on the table, recently committed delta files may reside in the `_delta_log/_commits` directory. Delta clients have to contact
+the corresponding commit-coordinator of the table in order to find the information about the [un-backfilled commits](#commit-backfills).
The delta files in `_delta_log` directory are named using the next available version number, zero-padded to 20 digits.
For example:
@@ -47,8 +47,8 @@ For example:
./_delta_log/_commits/00000000000000000002.3ae45b72-24e1-865a-a211-34987ae02f2a.json
```
-The `_delta_log/_commits` directory may contain uncommitted delta files. The [commit-owner](#commit-owner) is the source of truth about which of those delta
-files map to committed versions. Refer to [managed commits](#managed-commits) for more details.
+The `_delta_log/_commits` directory may contain uncommitted delta files. The [commit-coordinator](#commit-coordinator) is the source of truth about which of those delta
+files map to committed versions. Refer to [coordinated commits](#coordinated-commits) for more details.
~~Delta files use new-line delimited JSON format, where every action is stored as a single line JSON document.
A delta file, `n.json`, contains an atomic set of [_actions_](#Actions) that should be applied to the previous table state, `n-1.json`, in order to the construct `n`th snapshot of the table.
@@ -77,42 +77,42 @@ An action changes one aspect of the table's state, for example, adding or removi
Checkpoints are also stored in the `_delta_log` directory, and can be created at any time, for any committed version of the table.
For performance reasons, readers should prefer to use the newest complete checkpoint possible.
-**Note:** If [managed commits](#managed-commits) table feature is enabled on the table, a checkpoint can be created only for commit
-versions which are backfilled. Refer to [maintenance operations on managed-commit tables](#maintenance-operations-on-managed-commit-tables) section
+**Note:** If [coordinated commits](#coordinated-commits) table feature is enabled on the table, a checkpoint can be created only for commit
+versions which are backfilled. Refer to [maintenance operations on coordinated-commits tables](#maintenance-operations-on-coordinated-commits-tables) section
for more details
### Log Compaction Files
> ***Change to [existing section](https://github.com/delta-io/delta/blob/master/PROTOCOL.md#log-compaction-files)***
-**Note:** If [managed commits](#managed-commits) table feature is enabled on the table, a log compaction file for commit
+**Note:** If [coordinated commits](#coordinated-commits) table feature is enabled on the table, a log compaction file for commit
range `[x, y]` i.e. `x.y.compacted.json` can be created only when commit `y` is already backfilled i.e. `_delta_log/.json` must exist.
-Refer to [maintenance operations on managed-commit tables](#maintenance-operations-on-managed-commit-tables) section
+Refer to [maintenance operations on coordinated-commits tables](#maintenance-operations-on-coordinated-commits-tables) section
for more details.
> ***The next set of sections will be added to the existing spec just before [Iceberg Compatibility V1](https://github.com/delta-io/delta/blob/master/PROTOCOL.md#iceberg-compatibility-v1) section***
-# Managed Commits
+# Coordinated Commits
With this feature enabled:
- The file system remains the source of truth for the _content_ of a (proposed) commit.
-- The [commit-owner](#commit-owner) becomes the source of truth for whether a given commit succeeded.
+- The [commit-coordinator](#commit-coordinator) becomes the source of truth for whether a given commit succeeded.
-The following is a high-level overview of how commits work in a table with managed-commits enabled:
+The following is a high-level overview of how commits work in a table with coordinated-commits enabled:
-1. Delta client passes the actions that need to be committed to the [commit-owner](#commit-owner).
-2. The [commit-owner](#commit-owner) abstracts the commit process and defines the atomicity protocol for
+1. Delta client passes the actions that need to be committed to the [commit-coordinator](#commit-coordinator).
+2. The [commit-coordinator](#commit-coordinator) abstracts the commit process and defines the atomicity protocol for
commits to that table. It writes the actions in a [delta file](#delta-log-entries) and atomically makes
this file part of the table. Refer to [commit protocol](#commit protocol) section for details around how
- the commit-owner performs commits.
-3. In case of no conflict, the [commit-owner](#commit-owner) responds with success to the delta client.
-4. Delta clients could contact the commit-owner to get the information about the table's most recent commits.
+ the commit-coordinator performs commits.
+3. In case of no conflict, the [commit-coordinator](#commit-coordinator) responds with success to the delta client.
+4. Delta clients could contact the commit-coordinator to get the information about the table's most recent commits.
-Essentially the [managed-commits](#managed-commits) table feature defines the overall [commit protocol](#commit-protocol) (e.g. atomicity requirements, backfills, etc), and the
-commit-owner is responsible to implement that protocol.
+Essentially the [coordinated-commits](#coordinated-commits) table feature defines the overall [commit protocol](#commit-protocol) (e.g. atomicity requirements, backfills, etc), and the
+commit-coordinator is responsible to implement that protocol.
-## Commit Owner
+## Commit Coordinator
-A commit-owner is an external entity which manages the commits on a delta table. It could be backed by a database, a file system, or any other persistence mechanism. Each commit-owner has its own spec around how Delta clients should contact them, and how they perform a commit.
+A commit-coordinator is an external entity which manages the commits on a delta table. It could be backed by a database, a file system, or any other persistence mechanism. Each commit-coordinator has its own spec around how Delta clients should contact them, and how they perform a commit.
## Commit Files
@@ -122,57 +122,57 @@ There are two types of commit files:
1. **Un-backfilled commit files**: These reside in the `_delta_log/_commits` directory.
- The filename must follow the pattern: `..json`. Here the `uuid` is a random UUID that is generated for each commit and `version` is the version `v` which is being committed, zero-padded to 20 digits.
- Mere existence of these files does not mean that the file is a _valid_ commit. It might correspond to a failed or in-progress commit.
- The commit-owner is the source of truth around which un-backfilled commits are valid.
- - The commit-owner must track these files until they are backfilled to the `_delta_log` directory.
+ The commit-coordinator is the source of truth around which un-backfilled commits are valid.
+ - The commit-coordinator must track these files until they are backfilled to the `_delta_log` directory.
2. **Backfilled commit files**: These reside in the `_delta_log` directory.
- The filename must follow the pattern: `.json`. Here the `version` is the version `v` which is being committed, zero-padded to 20 digits.
- - The existence of a `.json` file proves that the corresponding version `v` is committed, even for managed-commit tables. Filesystem based Delta clients can use filesystem listing operations to directly discover such commits.
+ - The existence of a `.json` file proves that the corresponding version `v` is committed, even for coordinated-commits tables. Filesystem based Delta clients can use filesystem listing operations to directly discover such commits.
-Without [managed-commits](#managed-commits), a delta client must always write commit files directly to the `_delta_log` directory, relying on filesystem atomicity
+Without [coordinated-commits](#coordinated-commits), a delta client must always write commit files directly to the `_delta_log` directory, relying on filesystem atomicity
to prevent lost writes when multiple writers attempt to commit the same version at the same time.
-With [managed-commits](#managed-commits), the delta client asks the [commit-owner](#commit-owner) to commit the version `v` and the commit-owner
-decides which type of commit file to write, based on the [managed commit protocol](#commit-protocol).
+With [coordinated-commits](#coordinated-commits), the delta client asks the [commit-coordinator](#commit-coordinator) to commit the version `v` and the commit-coordinator
+decides which type of commit file to write, based on the [coordinated commits protocol](#commit-protocol).
-## Commit Owner API
+## Commit Coordinator API
-When managed commits are enabled, a `commit-owner` performs commits to the table on behalf of the Delta client. A commit-owner always has a client-side component (which the Delta client interacts with directly). It may also
-involve a server-side component (which the client-side component would be responsible to communicate with). The Delta client is responsible to define the client-side API that commit-owners should target, and commit-owners
-are responsible to define the commit atomicity and backfill protocols which the commit-owner client should implement.
+When coordinated commits are enabled, a `commit-coordinator` performs commits to the table on behalf of the Delta client. A commit-coordinator always has a client-side component (which the Delta client interacts with directly). It may also
+involve a server-side component (which the client-side component would be responsible to communicate with). The Delta client is responsible to define the client-side API that commit-coordinators should target, and commit-coordinators
+are responsible to define the commit atomicity and backfill protocols which the commit-coordinator client should implement.
-At a high level, the `commit-owner` needs to provide:
+At a high level, the `commit-coordinator` needs to provide:
- API to atomically commit a version `x` with given set of `actions`. This is explained in detail in the [commit protocol](#commit-protocol) section.
-- API to retrieve information about the recent commits and the latest ratified version on the table. This is explained in detail in the [getting un-backfilled commits from commit-owner](#getting-un-backfilled-commits-from-commit-owner) section.
+- API to retrieve information about the recent commits and the latest ratified version on the table. This is explained in detail in the [getting un-backfilled commits from commit-coordinator](#getting-un-backfilled-commits-from-commit-coordinator) section.
### Commit Protocol
-When a `commit-owner` receives a request to commit version `v`, it must first verify that the previous version `v-1` already exists, and that version `v` does not yet exist. It then has following choices to publish the commit:
+When a `commit-coordinator` receives a request to commit version `v`, it must first verify that the previous version `v-1` already exists, and that version `v` does not yet exist. It then has following choices to publish the commit:
1. Write the actions to an 'un-backfilled' [commit file](#commit-files) in the `_delta_log/_commits` directory, and **atomically** record that the new file now corresponds to version `v`.
2. Atomically write a backfilled [commit file](#commit-files) in the `_delta_log` directory. Note that the commit will be considered to have succeeded as soon as the file becomes visible to
other clients in the filesystem, regardless of when or whether the originating client receives a response.
- - A commit-owner must not write a backfilled commit until the previous commit has been backfilled.
+ - A commit-coordinator must not write a backfilled commit until the previous commit has been backfilled.
-The commit-owner must track the un-backfilled commits until they are [backfilled](#commit-backfills).
+The commit-coordinator must track the un-backfilled commits until they are [backfilled](#commit-backfills).
-### Getting Un-backfilled Commits from Commit Owner
+### Getting Un-backfilled Commits from Commit Coordinator
Even after a commit succeeds, Delta clients can only discover the commit through filesystem operations if the commit is [backfilled](#backfills). If the commit is not backfilled, then delta implementations
have no way to determine which file in `_delta_log/_commits` directory corresponds to the actual commit `v`.
-The commit-owner is responsible to implement an API (defined by the Delta client) that Delta clients can use to retrieve information about un-backfilled commits maintained
-by the commit-owner. The API must also return the latest version of the table ratified by the commit-owner (if any).
-Providing the latest ratified table version helps address potential race conditions between listing commits and contacting the commit-owner.
-For example, if a client performs a listing before a recently ratified commit is backfilled, and then contacts the commit-owner after the backfill completes,
-the commit-owner may return an empty list of un-backfilled commits. Without knowing the latest ratified version, the client might incorrectly assume their listing was complete
+The commit-coordinator is responsible to implement an API (defined by the Delta client) that Delta clients can use to retrieve information about un-backfilled commits maintained
+by the commit-coordinator. The API must also return the latest version of the table ratified by the commit-coordinator (if any).
+Providing the latest ratified table version helps address potential race conditions between listing commits and contacting the commit-coordinator.
+For example, if a client performs a listing before a recently ratified commit is backfilled, and then contacts the commit-coordinator after the backfill completes,
+the commit-coordinator may return an empty list of un-backfilled commits. Without knowing the latest ratified version, the client might incorrectly assume their listing was complete
and read a stale snapshot.
-Delta clients who are unaware of the commit-owner (or unwilling to talk to it), may not see recent un-backfilled commits and thus may encounter stale reads.
+Delta clients who are unaware of the commit-coordinator (or unwilling to talk to it), may not see recent un-backfilled commits and thus may encounter stale reads.
-## Sample Commit Owner API
+## Sample Commit Coordinator API
-The following is an example of a possible commit-owner API which some Java-based Delta client might require commit-owner implementations to target:
+The following is an example of a possible commit-coordinator API which some Java-based Delta client might require commit-coordinator implementations to target:
```java
@@ -193,24 +193,24 @@ interface CommitStore {
* API to get the un-backfilled commits for the table represented by the given `tablePath` where
* `startVersion` <= version <= endVersion.
* If endVersion is -1, then it means that we want to get all the commits starting from `startVersion`
- * till the latest version tracked by commit-owner.
+ * till the latest version tracked by commit-coordinator.
* The returned commits are contiguous and in ascending version order.
* Note that the first version returned by this API may not be equal to the `startVersion`. This
* happens when few versions starting from `startVersion` are already backfilled and so
* CommitStore may have stopped tracking them.
- * The returned latestTableVersion is the maximum commit version ratified by the Commit-Owner.
- * Note that returning latestTableVersion as -1 is acceptable only if the commit-owner never
+ * The returned latestTableVersion is the maximum commit version ratified by the Commit-Coordinator.
+ * Note that returning latestTableVersion as -1 is acceptable only if the commit-coordinator never
* ratified any version i.e. it never accepted any un-backfilled commit.
*
* @return GetCommitsResponse which contains a list of `Commit`s and the latestTableVersion
- * tracked by the commit-owner.
+ * tracked by the commit-coordinator.
*/
def getCommits(
startVersion: Long,
endVersion: Long): GetCommitsResponse
/**
- * API to ask the commit-owner to backfill all commits <= given `version`.
+ * API to ask the commit-coordinator to backfill all commits <= given `version`.
*/
def backfillToVersion(version: Long): Unit
}
@@ -219,62 +219,62 @@ interface CommitStore {
## Commit Backfills
Backfilling is the process of copying the un-backfilled commits i.e. `_delta_log/_commits/..json` to `_delta_log/.json`.
With the help of backfilling, the [delta files](#delta-log-entries) are visible even to the filesystem based Delta clients that do not
-understand `managed-commits`. Backfill also allows the commit-owner to reduce the number of commits it must track internally.
+understand `coordinated-commits`. Backfill also allows the commit-coordinator to reduce the number of commits it must track internally.
-Backfill must be sequential. In other words, a commit-owner must ensure that backfill of commit `v-1` is complete before initiating backfill of commit `v`.
+Backfill must be sequential. In other words, a commit-coordinator must ensure that backfill of commit `v-1` is complete before initiating backfill of commit `v`.
-`commit-owner`s are encouraged to backfill the commits frequently. This has several advantages:
+`commit-coordinator`s are encouraged to backfill the commits frequently. This has several advantages:
1. Filesystem-based Delta implementations may only understand backfilled commits, and frequent backfill allows them to access the most recent table snapshots.
-2. Frequent backfilling minimizes the impact to readers in case the `commit-owner` is unavailable or loses state.
-3. Some maintenance operations (such as checkpoints, log compaction, and metadata cleanup) can be performed only on the backfilled part of the table. Refer to the [Maintenance operations on managed-commit tables](#maintenance-operations-on-managed-commit-tables) section for more details.
+2. Frequent backfilling minimizes the impact to readers in case the `commit-coordinator` is unavailable or loses state.
+3. Some maintenance operations (such as checkpoints, log compaction, and metadata cleanup) can be performed only on the backfilled part of the table. Refer to the [Maintenance operations on coordinated-commits tables](#maintenance-operations-on-coordinated-commits-tables) section for more details.
-The commit-owner also needs to expose an API to backfill the commits. This will allow clients to ask the commit-owner to backfill the commits if needed in order to do some maintenance operations.
+The commit-coordinator also needs to expose an API to backfill the commits. This will allow clients to ask the commit-coordinator to backfill the commits if needed in order to do some maintenance operations.
-Since commit backfills may happen at a later point in time, so the `file modification timestamp` of the backfilled file might be very different than the time of actual commit. For this reason, the `managed-commit` feature depends on another writer feature called [in-commit-timestamps](#TODO-Put-Relevant-Link) to make the commit timestamps more reliable. Refer to [Writer Requirements for Managed Commits](#writer-requirements-for-managed-commits) section for more details.
+Since commit backfills may happen at a later point in time, so the `file modification timestamp` of the backfilled file might be very different than the time of actual commit. For this reason, the `coordinated-commits` feature depends on another writer feature called [in-commit-timestamps](#TODO-Put-Relevant-Link) to make the commit timestamps more reliable. Refer to [Writer Requirements for Coordinated Commits](#writer-requirements-for-coordinated-commits) section for more details.
-## Converting an existing filesystem based table to managed-commit table
-In order for a commit-owner to successfully take over an existing filesystem-based Delta table, the following invariants must hold:
-- The commit-owner must agree to take ownership of the table, by accepting a proposed commit that would install it. This essentially follows the normal commit protocol, except…
-- The commit-owner and client must both recognize that the ownership change only officially takes effect when the ownership-change is successfully backfilled. Unlike the backfill of a normal commit, this ownership-change backfill must
+## Converting an existing filesystem based table to coordinated-commits table
+In order for a commit-coordinator to successfully take over an existing filesystem-based Delta table, the following invariants must hold:
+- The commit-coordinator must agree to take ownership of the table, by accepting a proposed commit that would install it. This essentially follows the normal commit protocol, except…
+- The commit-coordinator and client must both recognize that the ownership change only officially takes effect when the ownership-change is successfully backfilled. Unlike the backfill of a normal commit, this ownership-change backfill must
be atomic because it is also a filesystem-based commit that potentially races with other filesystem-based commit attempts.
-Assuming the client follows the commit-owner’s protocol for ownership changes, the commit-owner MUST NOT refuse ownership after the backfill succeeds. Otherwise, the table would become permanently unusable, because the advertised commit-owner refuses
-to ratify the very commits that would repair the table by removing that commit-owner.
+Assuming the client follows the commit-coordinator’s protocol for ownership changes, the commit-coordinator MUST NOT refuse ownership after the backfill succeeds. Otherwise, the table would become permanently unusable, because the advertised commit-coordinator refuses
+to ratify the very commits that would repair the table by removing that commit-coordinator.
-Thus, the commit-owner and client effectively perform a two-phase commit, where the commit-owner persists its commitment to own the table, and the actual commit point is the PUT-if-absent.
-Notifying the commit-owner that backfill has completed becomes a post-commit cleanup operation. If the put-if-absent fails (because somebody else gets there first), the commit-owner forgets
+Thus, the commit-coordinator and client effectively perform a two-phase commit, where the commit-coordinator persists its commitment to own the table, and the actual commit point is the PUT-if-absent.
+Notifying the commit-coordinator that backfill has completed becomes a post-commit cleanup operation. If the put-if-absent fails (because somebody else gets there first), the commit-coordinator forgets
about the proposed ownership change.
-Once the backfill succeeds, clients will start contacting the commit-owner for any further commits. Meanwhile, any clients who were already attempting filesystem-based commits will encounter
+Once the backfill succeeds, clients will start contacting the commit-coordinator for any further commits. Meanwhile, any clients who were already attempting filesystem-based commits will encounter
a physical conflict, see the protocol change, and either abort the commit or route it to the new owner.
-## Creating a new managed-commit table
+## Creating a new coordinated-commits table
-Conceptually, creating a new managed-commit table is very similar to proposing an ownership change of an existing filesystem-based table that happens to not yet contain any commits. This means that, until commit 0
-has been backfilled, there is a risk of multiple clients racing to create the same table with different commit-owners (or to create a filesystem-based table).
+Conceptually, creating a new coordinated-commits table is very similar to proposing an ownership change of an existing filesystem-based table that happens to not yet contain any commits. This means that, until commit 0
+has been backfilled, there is a risk of multiple clients racing to create the same table with different commit-coordinators (or to create a filesystem-based table).
To avoid such races, Commit-owners are encouraged to use a put-if-absent API (if available) to write the backfilled commit directly (i.e. `_delta_log/00000000000000000000.json`).
-If such put-if-absent is not available, then it is the responsibility of commit-owners to take whatever measures they deem appropriate to avoid or respond to such races.
+If such put-if-absent is not available, then it is the responsibility of commit-coordinators to take whatever measures they deem appropriate to avoid or respond to such races.
-## Converting a managed-commit table to filesystem table
+## Converting a coordinated-commits table to filesystem table
-In order to convert a managed-commit table to a filesystem-based table, the Delta client needs to initiate a commit which tries to remove the commit-owner information
-from [change-metadata](#change-metadata) and also removes the table feature from the [protocol](#protocol-evolution) action. The commit-owner is not required to give
+In order to convert a coordinated-commits table to a filesystem-based table, the Delta client needs to initiate a commit which tries to remove the commit-coordinator information
+from [change-metadata](#change-metadata) and also removes the table feature from the [protocol](#protocol-evolution) action. The commit-coordinator is not required to give
up ownership, and may reject the request. If it chooses to honor such a request, it must:
1. Ensure that all prior commit files are backfilled.
2. Not accept any new commits on the table.
3. Write the commit which removes the ownership.
- - Either the commit-owner writes the backfilled commit file directly.
+ - Either the commit-coordinator writes the backfilled commit file directly.
- Or it writes an unbackfilled commit and ensures that it is backfilled reliably. Until the backfill is done, table will be in unusable state:
- - the filesystem based delta clients won't be able to write to such table as they still believe that table has managed-commit enabled.
- - the managed-commit aware delta clients won't be able to write to such table as the commit-owner won't accept any new
+ - the filesystem based delta clients won't be able to write to such table as they still believe that table has coordinated-commits enabled.
+ - the coordinated-commits aware delta clients won't be able to write to such table as the commit-coordinator won't accept any new
commits. In such a scenario, they could backfill required commit themselves (preferably using PUT-if-absent) to unblock themselves.
-## Reading managed-commit tables
+## Reading coordinated-commits tables
-With `managed-commits` enabled, a table could have some part of table already backfilled and some part of the table yet-to-be-backfilled.
-The precise information about what are the valid un-backfilled commits is maintained by the commit-owner.
+With `coordinated-commits` enabled, a table could have some part of table already backfilled and some part of the table yet-to-be-backfilled.
+The precise information about what are the valid un-backfilled commits is maintained by the commit-coordinator.
E.g.
```
@@ -297,7 +297,7 @@ _delta_log/_commits/00000000000000000010.0f707846-cd18-4e01-b40e-84ee0ae987b0.js
_delta_log/_commits/00000000000000000010.7a980438-cb67-4b89-82d2-86f73239b6d6.json
```
-Suppose the commit-owner is tracking:
+Suppose the commit-coordinator is tracking:
```
{
6 -> "00000000000000000006.3a0d65cd-4056-49b8-937b-95f9e3ee90e5.json",
@@ -309,28 +309,28 @@ Suppose the commit-owner is tracking:
Delta clients have two choices to read such tables:
1. Any Delta client can read such table by listing the `_delta_log` directory and reading the delta/checkpoint/log-compaction files.
- Without contacting the commit owner, they cannot access recent un-backfilled commits in the `_delta_log/_commits` directory, and may construct a stale snapshot.
+ Without contacting the commit coordinator, they cannot access recent un-backfilled commits in the `_delta_log/_commits` directory, and may construct a stale snapshot.
- In the above example, such delta implementation will see version 7 as the latest snapshot.
-2. A client can guarantee freshness by additionally requesting the set of recent un-backfilled commits from the commit-owner.
- - In the above example, a delta implementation could get information about versions 0 through 7 from `_delta_log` directory and get information about un-backfilled commits (v8, v9) from the commit-owner.
+2. A client can guarantee freshness by additionally requesting the set of recent un-backfilled commits from the commit-coordinator.
+ - In the above example, a delta implementation could get information about versions 0 through 7 from `_delta_log` directory and get information about un-backfilled commits (v8, v9) from the commit-coordinator.
-## Maintenance operations on managed-commit tables
+## Maintenance operations on coordinated-commits tables
[Checkpoints](#checkpoints-1) and [log compaction files](#log-compaction-files) can only be created for commits in the `_delta_log` directory. In other words, in order to
checkpoint version `v` or produce a compacted log file for commit range x <= v <= y, `_delta_log/.json` must exist. Otherwise, filesystem-based readers who encountered
the seemingly-extra files might think the table metadata was corrupted.
-## Managed Commit Enablement
+## Coordinated Commits Enablement
-The managed-commit feature is supported and active when:
+The coordinated-commits feature is supported and active when:
- The table must be on Writer Version 7.
-- The table has a `protocol` action with `writerFeatures` containing the feature `managedCommit`.
-- The table has a metadata property `delta.managedCommit.commitOwner` in the [change-metadata](#change-metadata)'s configuration.
-- The table may have a metadata property `delta.managedCommit.commitOwnerConf` in the [change-metadata](#change-metadata)'s configuration. The value of this property is a json-coded string-to-string map.
- - A commit-owner can store additional information (e.g. configuration information such as service endpoints) in this field, for use by the commit-owner client (it is opaque to the Delta client).
+- The table has a `protocol` action with `writerFeatures` containing the feature `coordinatedCommits`.
+- The table has a metadata property `delta.coordinatedCommits.commitCoordinator` in the [change-metadata](#change-metadata)'s configuration.
+- The table may have a metadata property `delta.coordinatedCommits.commitCoordinatorConf` in the [change-metadata](#change-metadata)'s configuration. The value of this property is a json-coded string-to-string map.
+ - A commit-coordinator can store additional information (e.g. configuration information such as service endpoints) in this field, for use by the commit-coordinator client (it is opaque to the Delta client).
- This field should never include secrets such as auth tokens or credentials, because any reader with access to the table's storage location can see them.
-Note that a table is in invalid state if the change-metadata contains the `delta.managedCommit.commitOwner` property but the table does not have the `managedCommit` feature in the `protocol` action (or vice versa).
+Note that a table is in invalid state if the change-metadata contains the `delta.coordinatedCommits.commitCoordinator` property but the table does not have the `coordinatedCommits` feature in the `protocol` action (or vice versa).
E.g.
```json
@@ -342,27 +342,27 @@ E.g.
"partitionColumns":[],
"configuration":{
"appendOnly": "true",
- "delta.managedCommit.commitOwner": "commit-owner-1",
- "delta.managedCommit.commitOwnerConf":
+ "delta.coordinatedCommits.commitCoordinator": "commit-coordinator-1",
+ "delta.coordinatedCommits.commitCoordinatorConf":
"{\"endpoint\":\"http://sample-url.com/commit\", \"authenticationMode\":\"oauth2\"}"
}
}
}
```
-## Writer Requirements for Managed Commits
+## Writer Requirements for Coordinated Commits
When supported and active:
- The `inCommitTimestamp` table feature must also be supported and active.
-- Writer must follow the commit-owner's [commit protocol](#commit-protocol) and must not perform filesystem-based commits.
+- Writer must follow the commit-coordinator's [commit protocol](#commit-protocol) and must not perform filesystem-based commits.
- Writer must only create checkpoints or log compaction files for commits in the `_delta_log` directory.
- Metadata cleanup must always preserve the newest k >= 1 backfilled commits.
-## Reader Requirements for Managed Commits
+## Reader Requirements for Coordinated Commits
Managed commits is a writer feature. So it doesn't put any restrictions on the reader.
-- Filesystem-based delta readers which do not understand [managed commits](#managed-commits) may only
+- Filesystem-based delta readers which do not understand [coordinated commits](#coordinated-commits) may only
be able to read the backfilled commits. They may see a stale snapshot of the table if the recent commits are not backfilled.
-- The [managed commits](#managed-commits) aware delta readers could additionally contact the commit-owner to
+- The [coordinated commits](#coordinated-commits) aware delta readers could additionally contact the commit-coordinator to
get the information about the recent un-backfilled commits. This allows them to get the most recent snapshot of the table.
diff --git a/spark/src/main/java/io/delta/dynamodbcommitstore/ManagedCommitUtils.java b/spark/src/main/java/io/delta/dynamodbcommitcoordinator/CoordinatedCommitsUtils.java
similarity index 61%
rename from spark/src/main/java/io/delta/dynamodbcommitstore/ManagedCommitUtils.java
rename to spark/src/main/java/io/delta/dynamodbcommitcoordinator/CoordinatedCommitsUtils.java
index bb457558828..c7d34779034 100644
--- a/spark/src/main/java/io/delta/dynamodbcommitstore/ManagedCommitUtils.java
+++ b/spark/src/main/java/io/delta/dynamodbcommitcoordinator/CoordinatedCommitsUtils.java
@@ -14,24 +14,24 @@
* limitations under the License.
*/
-package io.delta.dynamodbcommitstore;
+package io.delta.dynamodbcommitcoordinator;
-import org.apache.spark.sql.delta.managedcommit.AbstractMetadata;
-import org.apache.spark.sql.delta.managedcommit.UpdatedActions;
+import org.apache.spark.sql.delta.coordinatedcommits.AbstractMetadata;
+import org.apache.spark.sql.delta.coordinatedcommits.UpdatedActions;
import org.apache.hadoop.fs.Path;
import java.util.UUID;
-public class ManagedCommitUtils {
+public class CoordinatedCommitsUtils {
- private ManagedCommitUtils() {}
+ private CoordinatedCommitsUtils() {}
/** The subdirectory in which to store the unbackfilled commit files. */
final static String COMMIT_SUBDIR = "_commits";
- /** The configuration key for the managed commit owner. */
- private static final String MANAGED_COMMIT_OWNER_CONF_KEY =
- "delta.managedCommit.commitOwner-preview";
+ /** The configuration key for the coordinated commits owner. */
+ private static final String COORDINATED_COMMITS_COORDINATOR_CONF_KEY =
+ "delta.coordinatedCommits.commitCoordinator-preview";
/**
* Creates a new unbackfilled delta file path for the given commit version.
@@ -55,23 +55,23 @@ public static Path getBackfilledDeltaFilePath(
return new Path(logPath, String.format("%020d.json", version));
}
- private static String getManagedCommitOwner(AbstractMetadata metadata) {
+ private static String getCoordinatedCommitsCoordinator(AbstractMetadata metadata) {
return metadata
.getConfiguration()
- .get(MANAGED_COMMIT_OWNER_CONF_KEY)
+ .get(COORDINATED_COMMITS_COORDINATOR_CONF_KEY)
.getOrElse(() -> "");
}
/**
- * Returns true if the commit is a managed commit to filesystem conversion.
+ * Returns true if the commit is a coordinated commits to filesystem conversion.
*/
- public static boolean isManagedCommitToFSConversion(
+ public static boolean isCoordinatedCommitsToFSConversion(
Long commitVersion,
UpdatedActions updatedActions) {
- boolean oldMetadataHasManagedCommit =
- !getManagedCommitOwner(updatedActions.getOldMetadata()).isEmpty();
- boolean newMetadataHasManagedCommit =
- !getManagedCommitOwner(updatedActions.getNewMetadata()).isEmpty();
- return oldMetadataHasManagedCommit && !newMetadataHasManagedCommit && commitVersion > 0;
+ boolean oldMetadataHasCoordinatedCommits =
+ !getCoordinatedCommitsCoordinator(updatedActions.getOldMetadata()).isEmpty();
+ boolean newMetadataHasCoordinatedCommits =
+ !getCoordinatedCommitsCoordinator(updatedActions.getNewMetadata()).isEmpty();
+ return oldMetadataHasCoordinatedCommits && !newMetadataHasCoordinatedCommits && commitVersion > 0;
}
}
diff --git a/spark/src/main/java/io/delta/dynamodbcommitstore/DynamoDBCommitOwnerClient.java b/spark/src/main/java/io/delta/dynamodbcommitcoordinator/DynamoDBCommitCoordinatorClient.java
similarity index 85%
rename from spark/src/main/java/io/delta/dynamodbcommitstore/DynamoDBCommitOwnerClient.java
rename to spark/src/main/java/io/delta/dynamodbcommitcoordinator/DynamoDBCommitCoordinatorClient.java
index 2ec4b52b626..7d0e858b854 100644
--- a/spark/src/main/java/io/delta/dynamodbcommitstore/DynamoDBCommitOwnerClient.java
+++ b/spark/src/main/java/io/delta/dynamodbcommitcoordinator/DynamoDBCommitCoordinatorClient.java
@@ -14,11 +14,11 @@
* limitations under the License.
*/
-package io.delta.dynamodbcommitstore;
+package io.delta.dynamodbcommitcoordinator;
import com.amazonaws.services.dynamodbv2.AmazonDynamoDB;
import com.amazonaws.services.dynamodbv2.model.*;
-import org.apache.spark.sql.delta.managedcommit.*;
+import org.apache.spark.sql.delta.coordinatedcommits.*;
import org.apache.spark.sql.delta.storage.LogStore;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
@@ -40,15 +40,15 @@
import java.util.HashMap;
/**
- * A commit owner client that uses DynamoDB as the commit owner. The table schema is as follows:
+ * A commit coordinator client that uses DynamoDB as the commit coordinator. The table schema is as follows:
* tableId: String --- The unique identifier for the table. This is a UUID.
* path: String --- The fully qualified path of the table in the file system. e.g. s3://bucket/path.
- * acceptingCommits: Boolean --- Whether the commit owner is accepting new commits. This will only
- * be set to false when the table is converted from managed commits to file system commits.
+ * acceptingCommits: Boolean --- Whether the commit coordinator is accepting new commits. This will only
+ * be set to false when the table is converted from coordinated commits to file system commits.
* tableVersion: Number --- The version of the latest commit.
* tableTimestamp: Number --- The inCommitTimestamp of the latest commit.
* schemaVersion: Number --- The version of the schema used to store the data.
- * hasAcceptedCommits: Boolean --- Whether any actual commits have been accepted by this commit owner
+ * hasAcceptedCommits: Boolean --- Whether any actual commits have been accepted by this commit coordinator
* after `registerTable`.
* commits: --- The list of unbackfilled commits.
* version: Number --- The version of the commit.
@@ -57,13 +57,13 @@
* fsLength: Number --- The length of the unbackfilled file.
* fsTimestamp: Number --- The modification time of the unbackfilled file.
*/
-public class DynamoDBCommitOwnerClient implements CommitOwnerClient {
- private static final Logger LOG = LoggerFactory.getLogger(DynamoDBCommitOwnerClient.class);
+public class DynamoDBCommitCoordinatorClient implements CommitCoordinatorClient {
+ private static final Logger LOG = LoggerFactory.getLogger(DynamoDBCommitCoordinatorClient.class);
/**
* The name of the DynamoDB table used to store unbackfilled commits.
*/
- final String managedCommitTableName;
+ final String coordinatedCommitsTableName;
/**
* The DynamoDB client used to interact with the DynamoDB table.
@@ -100,7 +100,7 @@ public class DynamoDBCommitOwnerClient implements CommitOwnerClient {
final boolean skipPathCheck;
/**
- * The key used to store the tableId in the managed commit table configuration.
+ * The key used to store the tableId in the coordinated commits table configuration.
*/
final static String TABLE_CONF_TABLE_ID_KEY = "tableId";
@@ -123,13 +123,13 @@ private static class GetCommitsResultInternal {
}
- public DynamoDBCommitOwnerClient(
- String managedCommitTableName,
+ public DynamoDBCommitCoordinatorClient(
+ String coordinatedCommitsTableName,
String endpoint,
AmazonDynamoDB client,
long backfillBatchSize) throws IOException {
this(
- managedCommitTableName,
+ coordinatedCommitsTableName,
endpoint,
client,
backfillBatchSize,
@@ -138,15 +138,15 @@ public DynamoDBCommitOwnerClient(
false /* skipPathCheck */);
}
- public DynamoDBCommitOwnerClient(
- String managedCommitTableName,
+ public DynamoDBCommitCoordinatorClient(
+ String coordinatedCommitsTableName,
String endpoint,
AmazonDynamoDB client,
long backfillBatchSize,
long readCapacityUnits,
long writeCapacityUnits,
boolean skipPathCheck) throws IOException {
- this.managedCommitTableName = managedCommitTableName;
+ this.coordinatedCommitsTableName = coordinatedCommitsTableName;
this.endpoint = endpoint;
this.client = client;
this.backfillBatchSize = backfillBatchSize;
@@ -156,29 +156,29 @@ public DynamoDBCommitOwnerClient(
tryEnsureTableExists();
}
- private String getTableId(Map managedCommitTableConf) {
- return managedCommitTableConf.get(TABLE_CONF_TABLE_ID_KEY).getOrElse(() -> {
+ private String getTableId(Map coordinatedCommitsTableConf) {
+ return coordinatedCommitsTableConf.get(TABLE_CONF_TABLE_ID_KEY).getOrElse(() -> {
throw new RuntimeException("tableId not found");
});
}
/**
- * Fetches the entry from the commit owner for the given table. Only the attributes defined
+ * Fetches the entry from the commit coordinator for the given table. Only the attributes defined
* in attributesToGet will be fetched.
*/
- private GetItemResult getEntryFromCommitOwner(
- Map managedCommitTableConf, String... attributesToGet) {
+ private GetItemResult getEntryFromCommitCoordinator(
+ Map coordinatedCommitsTableConf, String... attributesToGet) {
GetItemRequest request = new GetItemRequest()
- .withTableName(managedCommitTableName)
+ .withTableName(coordinatedCommitsTableName)
.addKeyEntry(
DynamoDBTableEntryConstants.TABLE_ID,
- new AttributeValue().withS(getTableId(managedCommitTableConf)))
+ new AttributeValue().withS(getTableId(coordinatedCommitsTableConf)))
.withAttributesToGet(attributesToGet);
return client.getItem(request);
}
/**
- * Commits the given file to the commit owner.
+ * Commits the given file to the commit coordinator.
* A conditional write is performed to the DynamoDB table entry associated with this Delta
* table.
* If the conditional write goes through, the filestatus of the UUID delta file will be
@@ -188,8 +188,8 @@ private GetItemResult getEntryFromCommitOwner(
* For the conditional write to go through, the following conditions must be met right before
* the write is performed:
* 1. The latest table version in DynamoDB is equal to attemptVersion - 1.
- * 2. The commit owner is accepting new commits.
- * 3. The schema version of the commit owner matches the schema version of the client.
+ * 2. The commit coordinator is accepting new commits.
+ * 3. The schema version of the commit coordinator matches the schema version of the client.
* 4. The table path stored in DynamoDB matches the path of the table. This check is skipped
* if `skipPathCheck` is set to true.
* If the conditional write fails, we retrieve the current entry in DynamoDB to figure out
@@ -203,7 +203,7 @@ private GetItemResult getEntryFromCommitOwner(
*/
protected CommitResponse commitToOwner(
Path logPath,
- Map managedCommitTableConf,
+ Map coordinatedCommitsTableConf,
long attemptVersion,
FileStatus commitFile,
long inCommitTimestamp,
@@ -248,10 +248,10 @@ protected CommitResponse commitToOwner(
new AttributeValue().withN(Long.toString(commitFile.getModificationTime())));
UpdateItemRequest request = new UpdateItemRequest()
- .withTableName(managedCommitTableName)
+ .withTableName(coordinatedCommitsTableName)
.addKeyEntry(
DynamoDBTableEntryConstants.TABLE_ID,
- new AttributeValue().withS(getTableId(managedCommitTableConf)))
+ new AttributeValue().withS(getTableId(coordinatedCommitsTableConf)))
.addAttributeUpdatesEntry(
DynamoDBTableEntryConstants.TABLE_LATEST_VERSION, new AttributeValueUpdate()
.withValue(new AttributeValue().withN(Long.toString(attemptVersion)))
@@ -279,7 +279,7 @@ DynamoDBTableEntryConstants.TABLE_LATEST_TIMESTAMP, new AttributeValueUpdate()
.withExpected(expectedValuesBeforeUpdate);
if (isMCtoFSConversion) {
- // If this table is being converted from managed commits to file system commits, we need
+ // If this table is being converted from coordinated commits to file system commits, we need
// to set acceptingCommits to false.
request = request
.addAttributeUpdatesEntry(
@@ -296,8 +296,8 @@ DynamoDBTableEntryConstants.TABLE_LATEST_TIMESTAMP, new AttributeValueUpdate()
// Conditional check failed. The exception will not indicate which condition failed.
// We need to check the conditions ourselves by fetching the item and checking the
// values.
- GetItemResult latestEntry = getEntryFromCommitOwner(
- managedCommitTableConf,
+ GetItemResult latestEntry = getEntryFromCommitCoordinator(
+ coordinatedCommitsTableConf,
DynamoDBTableEntryConstants.TABLE_LATEST_VERSION,
DynamoDBTableEntryConstants.ACCEPTING_COMMITS,
DynamoDBTableEntryConstants.TABLE_PATH,
@@ -309,8 +309,8 @@ DynamoDBTableEntryConstants.TABLE_LATEST_TIMESTAMP, new AttributeValueUpdate()
throw new CommitFailedException(
false /* retryable */,
false /* conflict */,
- "The schema version of the commit owner does not match the current" +
- "DynamoDBCommitOwnerClient version. The data schema version is " +
+ "The schema version of the commit coordinator does not match the current" +
+ "DynamoDBCommitCoordinatorClient version. The data schema version is " +
" " + schemaVersion + " while the client version is " +
CLIENT_VERSION + ". Make sure that the correct client is being " +
"used to access this table." );
@@ -330,7 +330,7 @@ DynamoDBTableEntryConstants.TABLE_LATEST_TIMESTAMP, new AttributeValueUpdate()
throw new CommitFailedException(
false /* retryable */,
false /* conflict */,
- "The commit owner is not accepting any new commits for this table.");
+ "The commit coordinator is not accepting any new commits for this table.");
}
if (latestTableVersion != attemptVersion - 1) {
// The commit is only retryable if the conflict is due to someone else committing
@@ -352,7 +352,7 @@ public CommitResponse commit(
LogStore logStore,
Configuration hadoopConf,
Path logPath,
- Map managedCommitTableConf,
+ Map coordinatedCommitsTableConf,
long commitVersion,
Iterator actions,
UpdatedActions updatedActions) {
@@ -365,18 +365,18 @@ public CommitResponse commit(
try {
FileSystem fs = logPath.getFileSystem(hadoopConf);
Path commitPath =
- ManagedCommitUtils.generateUnbackfilledDeltaFilePath(logPath, commitVersion);
+ CoordinatedCommitsUtils.generateUnbackfilledDeltaFilePath(logPath, commitVersion);
logStore.write(commitPath, actions, true /* overwrite */, hadoopConf);
FileStatus commitFileStatus = fs.getFileStatus(commitPath);
long inCommitTimestamp = updatedActions.getCommitInfo().getCommitTimestamp();
boolean isMCtoFSConversion =
- ManagedCommitUtils.isManagedCommitToFSConversion(commitVersion, updatedActions);
+ CoordinatedCommitsUtils.isCoordinatedCommitsToFSConversion(commitVersion, updatedActions);
LOG.info("Committing version {} with UUID delta file {} to DynamoDB.",
commitVersion, commitPath);
CommitResponse res = commitToOwner(
logPath,
- managedCommitTableConf,
+ coordinatedCommitsTableConf,
commitVersion,
commitFileStatus,
inCommitTimestamp,
@@ -387,7 +387,7 @@ public CommitResponse commit(
boolean shouldBackfillOnEveryCommit = backfillBatchSize <= 1;
boolean isBatchBackfillDue = commitVersion % backfillBatchSize == 0;
boolean shouldBackfill = shouldBackfillOnEveryCommit || isBatchBackfillDue ||
- // Always attempt a backfill for managed commit to filesystem conversion.
+ // Always attempt a backfill for coordinated commits to filesystem conversion.
// Even if this fails, the next reader will attempt to backfill.
isMCtoFSConversion;
if (shouldBackfill) {
@@ -395,7 +395,7 @@ public CommitResponse commit(
logStore,
hadoopConf,
logPath,
- managedCommitTableConf,
+ coordinatedCommitsTableConf,
commitVersion,
Option.empty());
}
@@ -410,7 +410,7 @@ private GetCommitsResultInternal getCommitsImpl(
Map tableConf,
Option startVersion,
Option endVersion) throws IOException {
- GetItemResult latestEntry = getEntryFromCommitOwner(
+ GetItemResult latestEntry = getEntryFromCommitCoordinator(
tableConf,
DynamoDBTableEntryConstants.COMMITS,
DynamoDBTableEntryConstants.TABLE_LATEST_VERSION,
@@ -421,7 +421,7 @@ private GetCommitsResultInternal getCommitsImpl(
Long.parseLong(item.get(DynamoDBTableEntryConstants.TABLE_LATEST_VERSION).getN());
AttributeValue allStoredCommits = item.get(DynamoDBTableEntryConstants.COMMITS);
ArrayList commits = new ArrayList<>();
- Path unbackfilledCommitsPath = new Path(logPath, ManagedCommitUtils.COMMIT_SUBDIR);
+ Path unbackfilledCommitsPath = new Path(logPath, CoordinatedCommitsUtils.COMMIT_SUBDIR);
for(AttributeValue attr: allStoredCommits.getL()) {
java.util.Map commitMap = attr.getM();
long commitVersion =
@@ -458,16 +458,16 @@ private GetCommitsResultInternal getCommitsImpl(
@Override
public GetCommitsResponse getCommits(
Path logPath,
- Map managedCommitTableConf,
+ Map coordinatedCommitsTableConf,
Option startVersion,
Option endVersion) {
try {
GetCommitsResultInternal res =
- getCommitsImpl(logPath, managedCommitTableConf, startVersion, endVersion);
+ getCommitsImpl(logPath, coordinatedCommitsTableConf, startVersion, endVersion);
long latestTableVersionToReturn = res.response.getLatestTableVersion();
if (!res.hasAcceptedCommits) {
/*
- * If the commit owner has not accepted any commits after `registerTable`, we should
+ * If the commit coordinator has not accepted any commits after `registerTable`, we should
* return -1 as the latest table version.
* ┌───────────────────────────────────┬─────────────────────────────────────────────────────┬────────────────────────────────┐
* │ Action │ Internal State │ Version returned on GetCommits │
@@ -498,7 +498,7 @@ private void writeActionsToBackfilledFile(
scala.collection.Iterator actions,
Configuration hadoopConf,
boolean shouldOverwrite) throws FileAlreadyExistsException {
- Path targetPath = ManagedCommitUtils.getBackfilledDeltaFilePath(logPath, version);
+ Path targetPath = CoordinatedCommitsUtils.getBackfilledDeltaFilePath(logPath, version);
logStore.write(targetPath, actions, shouldOverwrite, hadoopConf);
}
@@ -510,7 +510,7 @@ private void validateBackfilledFileExists(
}
long version = (long) lastKnownBackfilledVersion.get();
Path lastKnownBackfilledFile =
- ManagedCommitUtils.getBackfilledDeltaFilePath(logPath, version);
+ CoordinatedCommitsUtils.getBackfilledDeltaFilePath(logPath, version);
FileSystem fs = logPath.getFileSystem(hadoopConf);
if (!fs.exists(lastKnownBackfilledFile)) {
throw new IllegalArgumentException(
@@ -523,11 +523,11 @@ private void validateBackfilledFileExists(
}
/**
- * Backfills all the unbackfilled commits returned by the commit owner and notifies the commit
+ * Backfills all the unbackfilled commits returned by the commit coordinator and notifies the commit
* owner of the backfills.
* The version parameter is ignored in this implementation and all the unbackfilled commits
* are backfilled. This method will not throw any exception if the physical backfill
- * succeeds but the update to the commit owner fails.
+ * succeeds but the update to the commit coordinator fails.
* @throws IllegalArgumentException if the requested backfill version is greater than the latest
* version for the table.
*/
@@ -536,7 +536,7 @@ public void backfillToVersion(
LogStore logStore,
Configuration hadoopConf,
Path logPath,
- Map managedCommitTableConf,
+ Map coordinatedCommitsTableConf,
long version,
Option lastKnownBackfilledVersion) {
LOG.info("Backfilling all unbackfilled commits.");
@@ -544,7 +544,7 @@ public void backfillToVersion(
try {
resp = getCommitsImpl(
logPath,
- managedCommitTableConf,
+ coordinatedCommitsTableConf,
lastKnownBackfilledVersion,
Option.empty()).response;
} catch (IOException e) {
@@ -579,10 +579,10 @@ public void backfillToVersion(
}
}
UpdateItemRequest request = new UpdateItemRequest()
- .withTableName(managedCommitTableName)
+ .withTableName(coordinatedCommitsTableName)
.addKeyEntry(
DynamoDBTableEntryConstants.TABLE_ID,
- new AttributeValue().withS(getTableId(managedCommitTableConf)))
+ new AttributeValue().withS(getTableId(coordinatedCommitsTableConf)))
.addAttributeUpdatesEntry(
DynamoDBTableEntryConstants.COMMITS,
new AttributeValueUpdate()
@@ -612,11 +612,11 @@ public void backfillToVersion(
client.updateItem(request);
} catch (ConditionalCheckFailedException e) {
// Ignore the exception. The backfill succeeded but the update to
- // the commit owner failed. The main purpose of a backfill operation is to ensure that
+ // the commit coordinator failed. The main purpose of a backfill operation is to ensure that
// UUID commit is physically copied to a standard commit file path. A failed update to
- // the commit owner is not critical.
- LOG.warn("Backfill succeeded but the update to the commit owner failed. This is probably" +
- " due to a concurrent update to the commit owner. This is not a critical error and " +
+ // the commit coordinator is not critical.
+ LOG.warn("Backfill succeeded but the update to the commit coordinator failed. This is probably" +
+ " due to a concurrent update to the commit coordinator. This is not a critical error and " +
" should rectify itself.");
}
}
@@ -635,7 +635,7 @@ public Map registerTable(
// We maintain the invariant that a commit will only succeed if the latestVersion stored
// in the table is equal to attemptVersion - 1. To maintain this, even though the
// filesystem-based commit after register table can fail, we still treat the attemptVersion
- // at registration as a valid version. Since it is expected that the commit owner will
+ // at registration as a valid version. Since it is expected that the commit coordinator will
// return -1 as the table version if no commits have been accepted after registration, we
// use another attribute (HAS_ACCEPTED_COMMITS) to track whether any commits have been
// accepted. This attribute is set to true whenever any commit is accepted.
@@ -644,7 +644,7 @@ public Map registerTable(
item.put(
DynamoDBTableEntryConstants.TABLE_LATEST_VERSION,
new AttributeValue().withN(Long.toString(attemptVersion)));
- // Used to indicate that no real commits have gone through the commit owner yet.
+ // Used to indicate that no real commits have gone through the commit coordinator yet.
item.put(
DynamoDBTableEntryConstants.HAS_ACCEPTED_COMMITS,
new AttributeValue().withBOOL(false));
@@ -660,7 +660,7 @@ public Map registerTable(
new AttributeValue().withN(Integer.toString(CLIENT_VERSION)));
PutItemRequest request = new PutItemRequest()
- .withTableName(managedCommitTableName)
+ .withTableName(coordinatedCommitsTableName)
.withItem(item)
.withConditionExpression(
String.format(
@@ -686,14 +686,14 @@ private void tryEnsureTableExists() throws IOException {
while(retries < 20) {
String status = "CREATING";
try {
- DescribeTableResult result = client.describeTable(managedCommitTableName);
+ DescribeTableResult result = client.describeTable(coordinatedCommitsTableName);
TableDescription descr = result.getTable();
status = descr.getTableStatus();
} catch (ResourceNotFoundException e) {
LOG.info(
"DynamoDB table `{}` for endpoint `{}` does not exist. " +
"Creating it now with provisioned throughput of {} RCUs and {} WCUs.",
- managedCommitTableName, endpoint, readCapacityUnits, writeCapacityUnits);
+ coordinatedCommitsTableName, endpoint, readCapacityUnits, writeCapacityUnits);
try {
client.createTable(
// attributeDefinitions
@@ -702,7 +702,7 @@ private void tryEnsureTableExists() throws IOException {
DynamoDBTableEntryConstants.TABLE_ID,
ScalarAttributeType.S)
),
- managedCommitTableName,
+ coordinatedCommitsTableName,
// keySchema
java.util.Collections.singletonList(
new KeySchemaElement(
@@ -718,23 +718,23 @@ private void tryEnsureTableExists() throws IOException {
}
if (status.equals("ACTIVE")) {
if (created) {
- LOG.info("Successfully created DynamoDB table `{}`", managedCommitTableName);
+ LOG.info("Successfully created DynamoDB table `{}`", coordinatedCommitsTableName);
} else {
- LOG.info("Table `{}` already exists", managedCommitTableName);
+ LOG.info("Table `{}` already exists", coordinatedCommitsTableName);
}
break;
} else if (status.equals("CREATING")) {
retries += 1;
- LOG.info("Waiting for `{}` table creation", managedCommitTableName);
+ LOG.info("Waiting for `{}` table creation", coordinatedCommitsTableName);
try {
Thread.sleep(1000);
} catch(InterruptedException e) {
throw new InterruptedIOException(e.getMessage());
}
} else {
- LOG.error("table `{}` status: {}", managedCommitTableName, status);
- throw new RuntimeException("DynamoDBCommitOwnerCliet: Unable to create table with " +
- "name " + managedCommitTableName + " for endpoint " + endpoint + ". Ensure " +
+ LOG.error("table `{}` status: {}", coordinatedCommitsTableName, status);
+ throw new RuntimeException("DynamoDBCommitCoordinatorCliet: Unable to create table with " +
+ "name " + coordinatedCommitsTableName + " for endpoint " + endpoint + ". Ensure " +
"that the credentials provided have the necessary permissions to create " +
"tables in DynamoDB. If the table already exists, ensure that the table " +
"is in the ACTIVE state.");
@@ -743,12 +743,12 @@ private void tryEnsureTableExists() throws IOException {
}
@Override
- public boolean semanticEquals(CommitOwnerClient other) {
- if (!(other instanceof DynamoDBCommitOwnerClient)) {
+ public boolean semanticEquals(CommitCoordinatorClient other) {
+ if (!(other instanceof DynamoDBCommitCoordinatorClient)) {
return false;
}
- DynamoDBCommitOwnerClient otherStore = (DynamoDBCommitOwnerClient) other;
- return this.managedCommitTableName.equals(otherStore.managedCommitTableName)
+ DynamoDBCommitCoordinatorClient otherStore = (DynamoDBCommitCoordinatorClient) other;
+ return this.coordinatedCommitsTableName.equals(otherStore.coordinatedCommitsTableName)
&& this.endpoint.equals(otherStore.endpoint);
}
}
diff --git a/spark/src/main/java/io/delta/dynamodbcommitstore/DynamoDBCommitOwnerClientBuilder.java b/spark/src/main/java/io/delta/dynamodbcommitcoordinator/DynamoDBCommitCoordinatorClientBuilder.java
similarity index 71%
rename from spark/src/main/java/io/delta/dynamodbcommitstore/DynamoDBCommitOwnerClientBuilder.java
rename to spark/src/main/java/io/delta/dynamodbcommitcoordinator/DynamoDBCommitCoordinatorClientBuilder.java
index 26dd78640d0..e4fd73b6555 100644
--- a/spark/src/main/java/io/delta/dynamodbcommitstore/DynamoDBCommitOwnerClientBuilder.java
+++ b/spark/src/main/java/io/delta/dynamodbcommitcoordinator/DynamoDBCommitCoordinatorClientBuilder.java
@@ -14,13 +14,13 @@
* limitations under the License.
*/
-package io.delta.dynamodbcommitstore;
+package io.delta.dynamodbcommitcoordinator;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.services.dynamodbv2.AmazonDynamoDB;
import com.amazonaws.services.dynamodbv2.AmazonDynamoDBClient;
-import org.apache.spark.sql.delta.managedcommit.CommitOwnerBuilder;
-import org.apache.spark.sql.delta.managedcommit.CommitOwnerClient;
+import org.apache.spark.sql.delta.coordinatedcommits.CommitCoordinatorBuilder;
+import org.apache.spark.sql.delta.coordinatedcommits.CommitCoordinatorClient;
import org.apache.spark.sql.delta.sources.DeltaSQLConf;
import org.apache.hadoop.conf.Configuration;
import org.apache.spark.sql.SparkSession;
@@ -28,7 +28,7 @@
import java.io.IOException;
-public class DynamoDBCommitOwnerClientBuilder implements CommitOwnerBuilder {
+public class DynamoDBCommitCoordinatorClientBuilder implements CommitCoordinatorBuilder {
private final long BACKFILL_BATCH_SIZE = 1L;
@@ -42,7 +42,7 @@ public String getName() {
* commits for this owner. The value of this key is stored in the `conf`
* which is passed to the `build` method.
*/
- private static final String MANAGED_COMMITS_TABLE_NAME_KEY = "dynamoDBTableName";
+ private static final String COORDINATED_COMMITSS_TABLE_NAME_KEY = "dynamoDBTableName";
/**
* The endpoint of the DynamoDB service. The value of this key is stored in the
* `conf` which is passed to the `build` method.
@@ -50,29 +50,29 @@ public String getName() {
private static final String DYNAMO_DB_ENDPOINT_KEY = "dynamoDBEndpoint";
@Override
- public CommitOwnerClient build(SparkSession spark, Map conf) {
- String managedCommitTableName = conf.get(MANAGED_COMMITS_TABLE_NAME_KEY).getOrElse(() -> {
- throw new RuntimeException(MANAGED_COMMITS_TABLE_NAME_KEY + " not found");
+ public CommitCoordinatorClient build(SparkSession spark, Map conf) {
+ String coordinatedCommitsTableName = conf.get(COORDINATED_COMMITSS_TABLE_NAME_KEY).getOrElse(() -> {
+ throw new RuntimeException(COORDINATED_COMMITSS_TABLE_NAME_KEY + " not found");
});
String dynamoDBEndpoint = conf.get(DYNAMO_DB_ENDPOINT_KEY).getOrElse(() -> {
throw new RuntimeException(DYNAMO_DB_ENDPOINT_KEY + " not found");
});
String awsCredentialsProviderName =
- spark.conf().get(DeltaSQLConf.MANAGED_COMMIT_DDB_AWS_CREDENTIALS_PROVIDER_NAME());
+ spark.conf().get(DeltaSQLConf.COORDINATED_COMMITS_DDB_AWS_CREDENTIALS_PROVIDER_NAME());
int readCapacityUnits = Integer.parseInt(
- spark.conf().get(DeltaSQLConf.MANAGED_COMMIT_DDB_READ_CAPACITY_UNITS().key()));
+ spark.conf().get(DeltaSQLConf.COORDINATED_COMMITS_DDB_READ_CAPACITY_UNITS().key()));
int writeCapacityUnits = Integer.parseInt(
- spark.conf().get(DeltaSQLConf.MANAGED_COMMIT_DDB_WRITE_CAPACITY_UNITS().key()));
+ spark.conf().get(DeltaSQLConf.COORDINATED_COMMITS_DDB_WRITE_CAPACITY_UNITS().key()));
boolean skipPathCheck = Boolean.parseBoolean(
- spark.conf().get(DeltaSQLConf.MANAGED_COMMIT_DDB_SKIP_PATH_CHECK().key()));
+ spark.conf().get(DeltaSQLConf.COORDINATED_COMMITS_DDB_SKIP_PATH_CHECK().key()));
try {
AmazonDynamoDB ddbClient = createAmazonDDBClient(
dynamoDBEndpoint,
awsCredentialsProviderName,
spark.sessionState().newHadoopConf()
);
- return getDynamoDBCommitOwnerClient(
- managedCommitTableName,
+ return getDynamoDBCommitCoordinatorClient(
+ coordinatedCommitsTableName,
dynamoDBEndpoint,
ddbClient,
BACKFILL_BATCH_SIZE,
@@ -85,8 +85,8 @@ public CommitOwnerClient build(SparkSession spark, Map conf) {
}
}
- protected DynamoDBCommitOwnerClient getDynamoDBCommitOwnerClient(
- String managedCommitTableName,
+ protected DynamoDBCommitCoordinatorClient getDynamoDBCommitCoordinatorClient(
+ String coordinatedCommitsTableName,
String dynamoDBEndpoint,
AmazonDynamoDB ddbClient,
long backfillBatchSize,
@@ -94,8 +94,8 @@ protected DynamoDBCommitOwnerClient getDynamoDBCommitOwnerClient(
int writeCapacityUnits,
boolean skipPathCheck
) throws IOException {
- return new DynamoDBCommitOwnerClient(
- managedCommitTableName,
+ return new DynamoDBCommitCoordinatorClient(
+ coordinatedCommitsTableName,
dynamoDBEndpoint,
ddbClient,
backfillBatchSize,
diff --git a/spark/src/main/java/io/delta/dynamodbcommitstore/DynamoDBTableEntryConstants.java b/spark/src/main/java/io/delta/dynamodbcommitcoordinator/DynamoDBTableEntryConstants.java
similarity index 90%
rename from spark/src/main/java/io/delta/dynamodbcommitstore/DynamoDBTableEntryConstants.java
rename to spark/src/main/java/io/delta/dynamodbcommitcoordinator/DynamoDBTableEntryConstants.java
index 23b51a8f624..5baa4d2523d 100644
--- a/spark/src/main/java/io/delta/dynamodbcommitstore/DynamoDBTableEntryConstants.java
+++ b/spark/src/main/java/io/delta/dynamodbcommitcoordinator/DynamoDBTableEntryConstants.java
@@ -14,7 +14,7 @@
* limitations under the License.
*/
-package io.delta.dynamodbcommitstore;
+package io.delta.dynamodbcommitcoordinator;
/**
* Defines the field names used in the DynamoDB table entry.
@@ -28,14 +28,14 @@ private DynamoDBTableEntryConstants() {}
public static final String TABLE_LATEST_VERSION = "tableVersion";
/** The inCommitTimestamp of the latest commit in the corresponding Delta table. */
public static final String TABLE_LATEST_TIMESTAMP = "tableTimestamp";
- /** Whether this commit owner is accepting more commits for the corresponding Delta table. */
+ /** Whether this commit coordinator is accepting more commits for the corresponding Delta table. */
public static final String ACCEPTING_COMMITS = "acceptingCommits";
/** The path of the corresponding Delta table. */
public static final String TABLE_PATH = "path";
/** The schema version of this DynamoDB table entry. */
public static final String SCHEMA_VERSION = "schemaVersion";
/**
- * Whether this commit owner has accepted any commits after `registerTable`.
+ * Whether this commit coordinator has accepted any commits after `registerTable`.
*/
public static final String HAS_ACCEPTED_COMMITS = "hasAcceptedCommits";
/** The name of the field used to store unbackfilled commits. */
diff --git a/spark/src/main/java/io/delta/dynamodbcommitstore/ReflectionUtils.java b/spark/src/main/java/io/delta/dynamodbcommitcoordinator/ReflectionUtils.java
similarity index 98%
rename from spark/src/main/java/io/delta/dynamodbcommitstore/ReflectionUtils.java
rename to spark/src/main/java/io/delta/dynamodbcommitcoordinator/ReflectionUtils.java
index 3383ee7b526..0fa58f10cd1 100644
--- a/spark/src/main/java/io/delta/dynamodbcommitstore/ReflectionUtils.java
+++ b/spark/src/main/java/io/delta/dynamodbcommitcoordinator/ReflectionUtils.java
@@ -14,7 +14,7 @@
* limitations under the License.
*/
-package io.delta.dynamodbcommitstore;
+package io.delta.dynamodbcommitcoordinator;
import com.amazonaws.auth.AWSCredentialsProvider;
import org.apache.hadoop.conf.Configuration;
diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/Checkpoints.scala b/spark/src/main/scala/org/apache/spark/sql/delta/Checkpoints.scala
index 4c6320cb952..b9ee1de5a9c 100644
--- a/spark/src/main/scala/org/apache/spark/sql/delta/Checkpoints.scala
+++ b/spark/src/main/scala/org/apache/spark/sql/delta/Checkpoints.scala
@@ -345,11 +345,11 @@ trait Checkpoints extends DeltaLogging {
}
protected def writeCheckpointFiles(snapshotToCheckpoint: Snapshot): LastCheckpointInfo = {
- // With Managed-Commits, commit files are not guaranteed to be backfilled immediately in the
+ // With Coordinated-Commits, commit files are not guaranteed to be backfilled immediately in the
// _delta_log dir. While it is possible to compute a checkpoint file without backfilling,
// writing the checkpoint file in the log directory before backfilling the relevant commits
// will leave gaps in the dir structure. This can cause issues for readers that are not
- // communicating with the commit-owner.
+ // communicating with the commit-coordinator.
//
// Sample directory structure with a gap if we don't backfill commit files:
// _delta_log/
diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/ConflictChecker.scala b/spark/src/main/scala/org/apache/spark/sql/delta/ConflictChecker.scala
index a839448a95e..83cc97a4786 100644
--- a/spark/src/main/scala/org/apache/spark/sql/delta/ConflictChecker.scala
+++ b/spark/src/main/scala/org/apache/spark/sql/delta/ConflictChecker.scala
@@ -23,7 +23,7 @@ import scala.collection.mutable
import org.apache.spark.sql.delta.RowId.RowTrackingMetadataDomain
import org.apache.spark.sql.delta.actions._
-import org.apache.spark.sql.delta.managedcommit.UpdatedActions
+import org.apache.spark.sql.delta.coordinatedcommits.UpdatedActions
import org.apache.spark.sql.delta.metering.DeltaLogging
import org.apache.spark.sql.delta.sources.DeltaSQLConf
import org.apache.spark.sql.delta.util.DeltaSparkPlanUtils.CheckDeterministicOptions
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 6f7bc245ff5..7432d040907 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
@@ -736,31 +736,31 @@ trait DeltaConfigsBase extends DeltaLogging {
validationFunction = _ => true,
helpMessage = "needs to be a boolean.")
- val MANAGED_COMMIT_OWNER_NAME = buildConfig[Option[String]](
- "managedCommit.commitOwner-preview",
+ val COORDINATED_COMMITS_COORDINATOR_NAME = buildConfig[Option[String]](
+ "coordinatedCommits.commitCoordinator-preview",
null,
v => Option(v),
_ => true,
- """The managed commit-owner name for this table. This is used to determine which
- |implementation of commit-owner to use when committing to this table. If this property is not
+ """The commit-coordinator name for this table. This is used to determine which implementation
+ |of commit-coordinator to use when committing to this table. If this property is not
|set, the table will be considered as file system table and commits will be done via
|atomically publishing the commit file.
|""".stripMargin)
- val MANAGED_COMMIT_OWNER_CONF = buildConfig[Map[String, String]](
- "managedCommit.commitOwnerConf-preview",
+ val COORDINATED_COMMITS_COORDINATOR_CONF = buildConfig[Map[String, String]](
+ "coordinatedCommits.commitCoordinatorConf-preview",
null,
v => JsonUtils.fromJson[Map[String, String]](Option(v).getOrElse("{}")),
_ => true,
- "A string-to-string map of configuration properties for the managed commit-owner.")
+ "A string-to-string map of configuration properties for the commit-coordinator.")
- val MANAGED_COMMIT_TABLE_CONF = buildConfig[Map[String, String]](
- "managedCommit.tableConf-preview",
+ val COORDINATED_COMMITS_TABLE_CONF = buildConfig[Map[String, String]](
+ "coordinatedCommits.tableConf-preview",
null,
v => JsonUtils.fromJson[Map[String, String]](Option(v).getOrElse("{}")),
_ => true,
"A string-to-string map of configuration properties for describing the table to" +
- " managed commit-owner.")
+ " commit-coordinator.")
val IN_COMMIT_TIMESTAMPS_ENABLED = buildConfig[Boolean](
"enableInCommitTimestamps-preview",
diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaLog.scala b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaLog.scala
index e84933251ae..b82864d2126 100644
--- a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaLog.scala
+++ b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaLog.scala
@@ -31,7 +31,7 @@ import com.databricks.spark.util.TagDefinitions._
import org.apache.spark.sql.delta.actions._
import org.apache.spark.sql.delta.commands.WriteIntoDelta
import org.apache.spark.sql.delta.files.{TahoeBatchFileIndex, TahoeLogFileIndex}
-import org.apache.spark.sql.delta.managedcommit.ManagedCommitUtils
+import org.apache.spark.sql.delta.coordinatedcommits.CoordinatedCommitsUtils
import org.apache.spark.sql.delta.metering.DeltaLogging
import org.apache.spark.sql.delta.schema.{SchemaMergingUtils, SchemaUtils}
import org.apache.spark.sql.delta.sources._
@@ -332,7 +332,7 @@ class DeltaLog private(
def getChangeLogFiles(
startVersion: Long,
failOnDataLoss: Boolean = false): Iterator[(Long, FileStatus)] = {
- val deltasWithVersion = ManagedCommitUtils.commitFilesIterator(this, startVersion)
+ val deltasWithVersion = CoordinatedCommitsUtils.commitFilesIterator(this, startVersion)
// Subtract 1 to ensure that we have the same check for the inclusive startVersion
var lastSeenVersion = startVersion - 1
deltasWithVersion.map { case (status, version) =>
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 ade021e0ff8..24be600f7e8 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
@@ -36,7 +36,7 @@ import org.apache.spark.sql.delta.commands.cdc.CDCReader
import org.apache.spark.sql.delta.files._
import org.apache.spark.sql.delta.hooks.{CheckpointHook, GenerateSymlinkManifest, HudiConverterHook, IcebergConverterHook, PostCommitHook, UpdateCatalogFactory}
import org.apache.spark.sql.delta.implicits.addFileEncoder
-import org.apache.spark.sql.delta.managedcommit._
+import org.apache.spark.sql.delta.coordinatedcommits._
import org.apache.spark.sql.delta.metering.DeltaLogging
import org.apache.spark.sql.delta.schema.{SchemaMergingUtils, SchemaUtils}
import org.apache.spark.sql.delta.sources.DeltaSQLConf
@@ -378,10 +378,10 @@ trait OptimisticTransactionImpl extends TransactionalWrite
private[delta] var preCommitLogSegment: LogSegment =
snapshot.logSegment.copy(checkpointProvider = snapshot.checkpointProvider)
- // The commit-owner of a table shouldn't change. If it is changed by a concurrent commit, then it
- // will be detected as a conflict and the transaction will anyway fail.
- private[delta] val readSnapshotTableCommitOwnerClientOpt: Option[TableCommitOwnerClient] =
- snapshot.tableCommitOwnerClientOpt
+ // The commit-coordinator of a table shouldn't change. If it is changed by a concurrent commit,
+ // then it will be detected as a conflict and the transaction will anyway fail.
+ private[delta] val readSnapshotTableCommitCoordinatorClientOpt:
+ Option[TableCommitCoordinatorClient] = snapshot.tableCommitCoordinatorClientOpt
/**
* Generates a timestamp which is greater than the commit timestamp
@@ -712,8 +712,8 @@ trait OptimisticTransactionImpl extends TransactionalWrite
* to enable these pre-requisite features.
*/
private def getMetadataWithDependentFeaturesEnabled(metadata: Metadata): Metadata = {
- DeltaConfigs.MANAGED_COMMIT_OWNER_NAME.fromMetaData(metadata).map { _ =>
- // managed-commits requires ICT to be enabled as per the spec.
+ DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_NAME.fromMetaData(metadata).map { _ =>
+ // coordinated-commits requires ICT to be enabled as per the spec.
// If ICT is just in Protocol and not in Metadata,
// then it is in a 'supported' state but not enabled.
// In order to enable ICT, we have to set the table property in Metadata.
@@ -1156,9 +1156,9 @@ trait OptimisticTransactionImpl extends TransactionalWrite
txnId = Some(txnId))
val firstAttemptVersion = getFirstAttemptVersion
- val metadataUpdatedWithManagedCommitInfo = updateMetadataWithManagedCommitConfs()
+ val metadataUpdatedWithCoordinatedCommitsInfo = updateMetadataWithCoordinatedCommitsConfs()
val metadataUpdatedWithIctInfo = updateMetadataWithInCommitTimestamp(commitInfo)
- if (metadataUpdatedWithIctInfo || metadataUpdatedWithManagedCommitInfo) {
+ if (metadataUpdatedWithIctInfo || metadataUpdatedWithCoordinatedCommitsInfo) {
preparedActions = preparedActions.map {
case _: Metadata => metadata
case other => other
@@ -1236,41 +1236,43 @@ trait OptimisticTransactionImpl extends TransactionalWrite
}
/**
- * This method makes the necessary changes to Metadata based on managed-commit: If the table is
- * being converted from file-system to managed commits, then it registers the table with the
- * commit-owner and updates the Metadata with the necessary configuration information from the
- * commit-owner.
+ * This method makes the necessary changes to Metadata based on coordinated-commits: If the table
+ * is being converted from file-system to coordinated commits, then it registers the table with
+ * the commit-coordinator and updates the Metadata with the necessary configuration information
+ * from the commit-coordinator.
*
* @return A boolean which represents whether we have updated the table Metadata with
- * managed-commit information. If no changed were made, returns false.
+ * coordinated-commits information. If no changed were made, returns false.
*/
- protected def updateMetadataWithManagedCommitConfs(): Boolean = {
- validateManagedCommitConfInMetadata(newMetadata)
- val newManagedCommitTableConfOpt = registerTableForManagedCommitIfNeeded(metadata, protocol)
- val newManagedCommitTableConf = newManagedCommitTableConfOpt.getOrElse {
+ protected def updateMetadataWithCoordinatedCommitsConfs(): Boolean = {
+ validateCoordinatedCommitsConfInMetadata(newMetadata)
+ val newCoordinatedCommitsTableConfOpt =
+ registerTableForCoordinatedCommitsIfNeeded(metadata, protocol)
+ val newCoordinatedCommitsTableConf = newCoordinatedCommitsTableConfOpt.getOrElse {
return false
}
// FS to MC conversion
val finalMetadata = metadata
- val managedCommitTableConfJson = JsonUtils.toJson(newManagedCommitTableConf)
- val extraKVConf = DeltaConfigs.MANAGED_COMMIT_TABLE_CONF.key -> managedCommitTableConfJson
+ val coordinatedCommitsTableConfJson = JsonUtils.toJson(newCoordinatedCommitsTableConf)
+ val extraKVConf =
+ DeltaConfigs.COORDINATED_COMMITS_TABLE_CONF.key -> coordinatedCommitsTableConfJson
newMetadata = Some(finalMetadata.copy(
configuration = finalMetadata.configuration + extraKVConf))
true
}
- protected def validateManagedCommitConfInMetadata(newMetadataOpt: Option[Metadata]): Unit = {
- // Validate that the [[DeltaConfigs.MANAGED_COMMIT_OWNER_CONF]] is json parse-able.
+ protected def validateCoordinatedCommitsConfInMetadata(newMetadataOpt: Option[Metadata]): Unit = {
+ // Validate that the [[DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_CONF]] is json parse-able.
// Also do this validation if this table property has changed.
newMetadataOpt
.filter { newMetadata =>
- val newManagedCommitConf =
- newMetadata.configuration.get(DeltaConfigs.MANAGED_COMMIT_OWNER_CONF.key)
- val oldManagedCommitConf =
- snapshot.metadata.configuration.get(DeltaConfigs.MANAGED_COMMIT_OWNER_CONF.key)
- newManagedCommitConf != oldManagedCommitConf
- }.foreach(DeltaConfigs.MANAGED_COMMIT_OWNER_CONF.fromMetaData)
+ val newCoordinatedCommitsConf =
+ newMetadata.configuration.get(DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_CONF.key)
+ val oldCoordinatedCommitsConf =
+ snapshot.metadata.configuration.get(DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_CONF.key)
+ newCoordinatedCommitsConf != oldCoordinatedCommitsConf
+ }.foreach(DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_CONF.fromMetaData)
}
@@ -1318,20 +1320,22 @@ trait OptimisticTransactionImpl extends TransactionalWrite
newProtocol = newProtocolOpt
// If a feature requires another feature to be enabled, we enable the required
// feature in the metadata (if needed) and add it to the protocol.
- // e.g. Managed Commits requires ICT and VacuumProtocolCheck to be enabled.
+ // e.g. Coordinated Commits requires ICT and VacuumProtocolCheck to be enabled.
updateMetadataAndProtocolWithRequiredFeatures(newMetadata)
def recordCommitLargeFailure(ex: Throwable, op: DeltaOperations.Operation): Unit = {
- val managedCommitExceptionOpt = ex match {
+ val coordinatedCommitsExceptionOpt = ex match {
case e: CommitFailedException => Some(e)
case _ => None
}
val data = Map(
"exception" -> Utils.exceptionString(ex),
"operation" -> op.name,
- "fromManagedCommit" -> managedCommitExceptionOpt.isDefined,
- "fromManagedCommitConflict" -> managedCommitExceptionOpt.map(_.getConflict).getOrElse(""),
- "fromManagedCommitRetryable" -> managedCommitExceptionOpt.map(_.getRetryable).getOrElse(""))
+ "fromCoordinatedCommits" -> coordinatedCommitsExceptionOpt.isDefined,
+ "fromCoordinatedCommitsConflict" ->
+ coordinatedCommitsExceptionOpt.map(_.getConflict).getOrElse(""),
+ "fromCoordinatedCommitsRetryable" ->
+ coordinatedCommitsExceptionOpt.map(_.getRetryable).getOrElse(""))
recordDeltaEvent(deltaLog, "delta.commitLarge.failure", data = data)
}
@@ -1366,7 +1370,7 @@ trait OptimisticTransactionImpl extends TransactionalWrite
// Initialize everything needed to maintain auto-compaction stats.
partitionsAddedToOpt = Some(new mutable.HashSet[Map[String, String]])
val acStatsCollector = createAutoCompactStatsCollector()
- updateMetadataWithManagedCommitConfs()
+ updateMetadataWithCoordinatedCommitsConfs()
updateMetadataWithInCommitTimestamp(commitInfo)
var allActions =
@@ -1420,18 +1424,19 @@ trait OptimisticTransactionImpl extends TransactionalWrite
}
val fsWriteStartNano = System.nanoTime()
val jsonActions = allActions.map(_.json)
- val effectiveTableCommitOwnerClient = readSnapshotTableCommitOwnerClientOpt.getOrElse {
- TableCommitOwnerClient(
- commitOwnerClient = new FileSystemBasedCommitOwnerClient(deltaLog),
- deltaLog = deltaLog,
- managedCommitTableConf = snapshot.metadata.managedCommitTableConf)
- }
+ val effectiveTableCommitCoordinatorClient = readSnapshotTableCommitCoordinatorClientOpt
+ .getOrElse {
+ TableCommitCoordinatorClient(
+ commitCoordinatorClient = new FileSystemBasedCommitCoordinatorClient(deltaLog),
+ deltaLog = deltaLog,
+ coordinatedCommitsTableConf = snapshot.metadata.coordinatedCommitsTableConf)
+ }
val updatedActions = UpdatedActions(
commitInfo, metadata, protocol, snapshot.metadata, snapshot.protocol)
val commitResponse = TransactionExecutionObserver.withObserver(executionObserver) {
- effectiveTableCommitOwnerClient.commit(attemptVersion, jsonActions, updatedActions)
+ effectiveTableCommitCoordinatorClient.commit(attemptVersion, jsonActions, updatedActions)
}
- // TODO(managed-commits): Use the right timestamp method on top of CommitInfo once ICT is
+ // TODO(coordinated-commits): Use the right timestamp method on top of CommitInfo once ICT is
// merged.
// If the metadata didn't change, `newMetadata` is empty, and we can re-use the old id.
acStatsCollector.finalizeStats(newMetadata.map(_.id).getOrElse(this.snapshot.metadata.id))
@@ -1485,7 +1490,7 @@ trait OptimisticTransactionImpl extends TransactionalWrite
recordCommitLargeFailure(e, op)
// Actions of a commit which went in before ours.
// Requires updating deltaLog to retrieve these actions, as another writer may have used
- // CommitOwnerClient for writing.
+ // CommitCoordinatorClient for writing.
val logs = deltaLog.store.readAsIterator(
DeltaCommitFileProvider(deltaLog.update()).deltaFile(attemptVersion),
deltaLog.newDeltaHadoopConf())
@@ -1505,58 +1510,61 @@ trait OptimisticTransactionImpl extends TransactionalWrite
}
/**
- * This method registers the table with the commit-owner via the [[CommitOwnerClient]] if the
- * table is transitioning from file-system based table to managed-commit table.
+ * This method registers the table with the commit-coordinator via the [[CommitCoordinatorClient]]
+ * if the table is transitioning from file-system based table to coordinated-commits table.
* @param finalMetadata the effective [[Metadata]] of the table. Note that this refers to the
* new metadata if this commit is updating the table Metadata.
* @param finalProtocol the effective [[Protocol]] of the table. Note that this refers to the
* new protocol if this commit is updating the table Protocol.
- * @return The new managed-commit table metadata if the table is transitioning from file-system
- * based table to managed-commit table. Otherwise, None.
+ * @return The new coordinated-commits table metadata if the table is transitioning from
+ * file-system based table to coordinated-commits table. Otherwise, None.
* This metadata should be added to the [[Metadata.configuration]] before doing the
* commit.
*/
- protected def registerTableForManagedCommitIfNeeded(
+ protected def registerTableForCoordinatedCommitsIfNeeded(
finalMetadata: Metadata,
finalProtocol: Protocol): Option[Map[String, String]] = {
- val (oldOwnerName, oldOwnerConf) = ManagedCommitUtils.getManagedCommitConfs(snapshot.metadata)
- var newManagedCommitTableConf: Option[Map[String, String]] = None
+ val (oldOwnerName, oldOwnerConf) =
+ CoordinatedCommitsUtils.getCoordinatedCommitsConfs(snapshot.metadata)
+ var newCoordinatedCommitsTableConf: Option[Map[String, String]] = None
if (finalMetadata.configuration != snapshot.metadata.configuration || snapshot.version == -1L) {
- val newCommitOwnerClientOpt =
- ManagedCommitUtils.getCommitOwnerClient(spark, finalMetadata, finalProtocol)
- (newCommitOwnerClientOpt, readSnapshotTableCommitOwnerClientOpt) match {
- case (Some(newCommitOwnerClient), None) =>
+ val newCommitCoordinatorClientOpt =
+ CoordinatedCommitsUtils.getCommitCoordinatorClient(spark, finalMetadata, finalProtocol)
+ (newCommitCoordinatorClientOpt, readSnapshotTableCommitCoordinatorClientOpt) match {
+ case (Some(newCommitCoordinatorClient), None) =>
// FS -> MC conversion
- val (commitOwnerName, commitOwnerConf) =
- ManagedCommitUtils.getManagedCommitConfs(finalMetadata)
+ val (commitCoordinatorName, commitCoordinatorConf) =
+ CoordinatedCommitsUtils.getCoordinatedCommitsConfs(finalMetadata)
logInfo(s"Table ${deltaLog.logPath} transitioning from file-system based table to " +
- s"managed-commit table: [commit-owner: $commitOwnerName, conf: $commitOwnerConf]")
- newManagedCommitTableConf = Some(newCommitOwnerClient.registerTable(
+ s"coordinated-commits table: [commit-coordinator: $commitCoordinatorName, " +
+ s"conf: $commitCoordinatorConf]")
+ newCoordinatedCommitsTableConf = Some(newCommitCoordinatorClient.registerTable(
deltaLog.logPath, readVersion, finalMetadata, protocol))
- case (None, Some(readCommitOwnerClient)) =>
+ case (None, Some(readCommitCoordinatorClient)) =>
// MC -> FS conversion
val (newOwnerName, newOwnerConf) =
- ManagedCommitUtils.getManagedCommitConfs(snapshot.metadata)
- logInfo(s"Table ${deltaLog.logPath} transitioning from managed-commit table to " +
- s"file-system table: [commit-owner: $newOwnerName, conf: $newOwnerConf]")
- case (Some(newCommitOwnerClient), Some(readCommitOwnerClient))
- if !readCommitOwnerClient.semanticsEquals(newCommitOwnerClient) =>
+ CoordinatedCommitsUtils.getCoordinatedCommitsConfs(snapshot.metadata)
+ logInfo(s"Table ${deltaLog.logPath} transitioning from coordinated-commits table to " +
+ s"file-system table: [commit-coordinator: $newOwnerName, conf: $newOwnerConf]")
+ case (Some(newCommitCoordinatorClient), Some(readCommitCoordinatorClient))
+ if !readCommitCoordinatorClient.semanticsEquals(newCommitCoordinatorClient) =>
// MC1 -> MC2 conversion is not allowed.
- // In order to transfer the table from one commit-owner to another, transfer the table
- // from current commit-owner to filesystem first and then filesystem to the commit-owner.
+ // In order to transfer the table from one commit-coordinator to another, transfer the
+ // table from current commit-coordinator to filesystem first and then filesystem to
+ // the commit-coordinator.
val (newOwnerName, newOwnerConf) =
- ManagedCommitUtils.getManagedCommitConfs(finalMetadata)
- val message = s"Transition of table ${deltaLog.logPath} from one commit-owner to" +
- s" another commit-owner is not allowed: [old commit-owner: $oldOwnerName," +
- s" new commit-owner: $newOwnerName, old commit-owner conf: $oldOwnerConf," +
- s" new commit-owner conf: $newOwnerConf]."
+ CoordinatedCommitsUtils.getCoordinatedCommitsConfs(finalMetadata)
+ val message = s"Transition of table ${deltaLog.logPath} from one commit-coordinator to" +
+ s" another commit-coordinator is not allowed: [old commit-coordinator: $oldOwnerName," +
+ s" new commit-coordinator: $newOwnerName, old commit-coordinator conf: $oldOwnerConf," +
+ s" new commit-coordinator conf: $newOwnerConf]."
throw new IllegalStateException(message)
case _ =>
// no owner change
()
}
}
- newManagedCommitTableConf
+ newCoordinatedCommitsTableConf
}
/** Update the table now that the commit has been made, and write a checkpoint. */
@@ -1683,9 +1691,9 @@ trait OptimisticTransactionImpl extends TransactionalWrite
// `assertMetadata` call above.
performCdcColumnMappingCheck(finalActions, op)
- // Ensure Commit Directory exists when managed commits is enabled on an existing table.
- lazy val isFsToMcConversion = snapshot.metadata.managedCommitOwnerName.isEmpty &&
- newMetadata.flatMap(_.managedCommitOwnerName).nonEmpty
+ // Ensure Commit Directory exists when coordinated commits is enabled on an existing table.
+ lazy val isFsToMcConversion = snapshot.metadata.coordinatedCommitsCoordinatorName.isEmpty &&
+ newMetadata.flatMap(_.coordinatedCommitsCoordinatorName).nonEmpty
val shouldCreateLogDirs = snapshot.version == -1 || isFsToMcConversion
if (shouldCreateLogDirs) {
deltaLog.createLogDirectoriesIfNotExists()
@@ -1902,7 +1910,8 @@ trait OptimisticTransactionImpl extends TransactionalWrite
var commitVersion = attemptVersion
var updatedCurrentTransactionInfo = currentTransactionInfo
val isFsToMcCommit =
- snapshot.metadata.managedCommitOwnerName.isEmpty && metadata.managedCommitOwnerName.nonEmpty
+ snapshot.metadata.coordinatedCommitsCoordinatorName.isEmpty &&
+ metadata.coordinatedCommitsCoordinatorName.nonEmpty
val maxRetryAttempts = spark.conf.get(DeltaSQLConf.DELTA_MAX_RETRY_COMMIT_ATTEMPTS)
val maxNonConflictRetryAttempts =
spark.conf.get(DeltaSQLConf.DELTA_MAX_NON_CONFLICT_RETRY_COMMIT_ATTEMPTS)
@@ -1927,18 +1936,20 @@ trait OptimisticTransactionImpl extends TransactionalWrite
// Don't retry if this commit tries to upgrade the table from filesystem to managed
// commits and the first attempt failed due to a conflict.
throw DeltaErrors.concurrentWriteException(conflictingCommit = None)
- case _: FileAlreadyExistsException if readSnapshotTableCommitOwnerClientOpt.isEmpty =>
+ case _: FileAlreadyExistsException
+ if readSnapshotTableCommitCoordinatorClientOpt.isEmpty =>
// For filesystem based tables, we use LogStore to do the commit. On a conflict,
// LogStore returns FileAlreadyExistsException necessitating conflict resolution.
- // For commit-owners, FileAlreadyExistsException isn't expected under normal operations
- // and thus retries are not performed if this exception is thrown by CommitOwnerClient.
+ // For commit-coordinators, FileAlreadyExistsException isn't expected under normal
+ // operations and thus retries are not performed if this exception is thrown by
+ // CommitCoordinatorClient.
shouldCheckForConflicts = true
// Do nothing, retry with next available attemptVersion
case ex: CommitFailedException if ex.getRetryable && ex.getConflict =>
shouldCheckForConflicts = true
// Reset nonConflictAttemptNumber if a conflict is detected.
nonConflictAttemptNumber = 0
- // For managed-commits, only retry with next available attemptVersion when
+ // For coordinated-commits, only retry with next available attemptVersion when
// retryable is set and it was a case of conflict.
case ex: CommitFailedException if ex.getRetryable && !ex.getConflict =>
if (nonConflictAttemptNumber < maxNonConflictRetryAttempts) {
@@ -2087,12 +2098,13 @@ trait OptimisticTransactionImpl extends TransactionalWrite
postCommitSnapshot
}
- class FileSystemBasedCommitOwnerClient(val deltaLog: DeltaLog) extends CommitOwnerClient {
+ class FileSystemBasedCommitCoordinatorClient(val deltaLog: DeltaLog)
+ extends CommitCoordinatorClient {
override def commit(
logStore: LogStore,
hadoopConf: Configuration,
logPath: Path,
- managedCommitTableConf: Map[String, String],
+ coordinatedCommitsTableConf: Map[String, String],
commitVersion: Long,
actions: Iterator[String],
updatedActions: UpdatedActions): CommitResponse = {
@@ -2130,7 +2142,7 @@ trait OptimisticTransactionImpl extends TransactionalWrite
override def getCommits(
logPath: Path,
- managedCommitTableConf: Map[String, String],
+ coordinatedCommitsTableConf: Map[String, String],
startVersion: Option[Long],
endVersion: Option[Long]): GetCommitsResponse =
GetCommitsResponse(Seq.empty, -1)
@@ -2139,19 +2151,19 @@ trait OptimisticTransactionImpl extends TransactionalWrite
logStore: LogStore,
hadoopConf: Configuration,
logPath: Path,
- managedCommitTableConf: Map[String, String],
+ coordinatedCommitsTableConf: Map[String, String],
version: Long,
lastKnownBackfilledVersion: Option[Long] = None): Unit = {}
/**
- * [[FileSystemBasedCommitOwnerClient]] is supposed to be treated as a singleton object for a
- * Delta Log and is equal to all other instances of [[FileSystemBasedCommitOwnerClient]] for the
- * same Delta Log.
+ * [[FileSystemBasedCommitCoordinatorClient]] is supposed to be treated as a singleton object
+ * for a Delta Log and is equal to all other instances of
+ * [[FileSystemBasedCommitCoordinatorClient]] for the same Delta Log.
*/
- override def semanticEquals(other: CommitOwnerClient): Boolean = {
+ override def semanticEquals(other: CommitCoordinatorClient): Boolean = {
other match {
- case fsCommitOwnerClient: FileSystemBasedCommitOwnerClient =>
- fsCommitOwnerClient.deltaLog == deltaLog
+ case fsCommitCoordinatorClient: FileSystemBasedCommitCoordinatorClient =>
+ fsCommitCoordinatorClient.deltaLog == deltaLog
case _ => false
}
}
@@ -2159,7 +2171,7 @@ trait OptimisticTransactionImpl extends TransactionalWrite
/**
* Writes the json actions provided to the commit file corresponding to attemptVersion.
- * If managed-commits are enabled, this method must return a non-empty [[Commit]]
+ * If coordinated-commits are enabled, this method must return a non-empty [[Commit]]
* since we can't guess it from the FileSystem.
*/
protected def writeCommitFile(
@@ -2167,27 +2179,30 @@ trait OptimisticTransactionImpl extends TransactionalWrite
jsonActions: Iterator[String],
currentTransactionInfo: CurrentTransactionInfo)
: (Option[VersionChecksum], Commit) = {
- val commitOwnerClient = readSnapshotTableCommitOwnerClientOpt.getOrElse {
- TableCommitOwnerClient(
- new FileSystemBasedCommitOwnerClient(deltaLog),
+ val commitCoordinatorClient = readSnapshotTableCommitCoordinatorClientOpt.getOrElse {
+ TableCommitCoordinatorClient(
+ new FileSystemBasedCommitCoordinatorClient(deltaLog),
deltaLog,
- snapshot.metadata.managedCommitTableConf)
+ snapshot.metadata.coordinatedCommitsTableConf)
}
- val commitFile =
- writeCommitFileImpl(attemptVersion, jsonActions, commitOwnerClient, currentTransactionInfo)
+ val commitFile = writeCommitFileImpl(
+ attemptVersion,
+ jsonActions,
+ commitCoordinatorClient,
+ currentTransactionInfo)
(None, commitFile)
}
protected def writeCommitFileImpl(
attemptVersion: Long,
jsonActions: Iterator[String],
- tableCommitOwnerClient: TableCommitOwnerClient,
+ tableCommitCoordinatorClient: TableCommitCoordinatorClient,
currentTransactionInfo: CurrentTransactionInfo
): Commit = {
val updatedActions =
currentTransactionInfo.getUpdatedActions(snapshot.metadata, snapshot.protocol)
val commitResponse = TransactionExecutionObserver.withObserver(executionObserver) {
- tableCommitOwnerClient.commit(attemptVersion, jsonActions, updatedActions)
+ tableCommitCoordinatorClient.commit(attemptVersion, jsonActions, updatedActions)
}
if (attemptVersion == 0L) {
val expectedPathForCommitZero = unsafeDeltaFile(deltaLog.logPath, version = 0L).toUri
@@ -2281,8 +2296,9 @@ trait OptimisticTransactionImpl extends TransactionalWrite
/** Returns the conflicting commit information */
protected def getConflictingVersions(previousAttemptVersion: Long): Seq[FileStatus] = {
assert(previousAttemptVersion == preCommitLogSegment.version + 1)
- val (newPreCommitLogSegment, newCommitFileStatuses) =
- deltaLog.getUpdatedLogSegment(preCommitLogSegment, readSnapshotTableCommitOwnerClientOpt)
+ val (newPreCommitLogSegment, newCommitFileStatuses) = deltaLog.getUpdatedLogSegment(
+ preCommitLogSegment,
+ readSnapshotTableCommitCoordinatorClientOpt)
assert(preCommitLogSegment.version + newCommitFileStatuses.size ==
newPreCommitLogSegment.version)
preCommitLogSegment = newPreCommitLogSegment
@@ -2400,7 +2416,7 @@ trait OptimisticTransactionImpl extends TransactionalWrite
}
}
- // Backfill any unbackfilled commits if managed commits are disabled -- in the Optimistic
+ // Backfill any unbackfilled commits if coordinated commits are disabled -- in the Optimistic
// Transaction constructor.
- ManagedCommitUtils.backfillWhenManagedCommitDisabled(snapshot)
+ CoordinatedCommitsUtils.backfillWhenCoordinatedCommitsDisabled(snapshot)
}
diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/PreDowngradeTableFeatureCommand.scala b/spark/src/main/scala/org/apache/spark/sql/delta/PreDowngradeTableFeatureCommand.scala
index 4bf6435ed7f..856c398c24b 100644
--- a/spark/src/main/scala/org/apache/spark/sql/delta/PreDowngradeTableFeatureCommand.scala
+++ b/spark/src/main/scala/org/apache/spark/sql/delta/PreDowngradeTableFeatureCommand.scala
@@ -23,7 +23,7 @@ import scala.util.control.NonFatal
import org.apache.spark.sql.delta.catalog.DeltaTableV2
import org.apache.spark.sql.delta.commands.{AlterTableSetPropertiesDeltaCommand, AlterTableUnsetPropertiesDeltaCommand, DeltaReorgTableCommand, DeltaReorgTableMode, DeltaReorgTableSpec}
import org.apache.spark.sql.delta.commands.columnmapping.RemoveColumnMappingCommand
-import org.apache.spark.sql.delta.managedcommit.ManagedCommitUtils
+import org.apache.spark.sql.delta.coordinatedcommits.CoordinatedCommitsUtils
import org.apache.spark.sql.delta.metering.DeltaLogging
import org.apache.spark.sql.delta.schema.SchemaMergingUtils
import org.apache.spark.sql.delta.util.{Utils => DeltaUtils}
@@ -206,15 +206,15 @@ case class VacuumProtocolCheckPreDowngradeCommand(table: DeltaTableV2)
override def removeFeatureTracesIfNeeded(): Boolean = false
}
-case class ManagedCommitPreDowngradeCommand(table: DeltaTableV2)
+case class CoordinatedCommitsPreDowngradeCommand(table: DeltaTableV2)
extends PreDowngradeTableFeatureCommand
with DeltaLogging {
/**
* We disable the feature by removing the following table properties:
- * 1. DeltaConfigs.MANAGED_COMMIT_OWNER_NAME.key
- * 2. DeltaConfigs.MANAGED_COMMIT_OWNER_CONF.key
- * 3. DeltaConfigs.MANAGED_COMMIT_TABLE_CONF.key
+ * 1. DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_NAME.key
+ * 2. DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_CONF.key
+ * 3. DeltaConfigs.COORDINATED_COMMITS_TABLE_CONF.key
* If these properties have been removed but unbackfilled commits are still present, we
* backfill them.
*
@@ -227,14 +227,14 @@ case class ManagedCommitPreDowngradeCommand(table: DeltaTableV2)
var traceRemovalNeeded = false
var exceptionOpt = Option.empty[Throwable]
- val propertyPresenceLogs = ManagedCommitUtils.TABLE_PROPERTY_KEYS.map( key =>
+ val propertyPresenceLogs = CoordinatedCommitsUtils.TABLE_PROPERTY_KEYS.map( key =>
key -> table.initialSnapshot.metadata.configuration.contains(key).toString
)
- if (ManagedCommitUtils.tablePropertiesPresent(table.initialSnapshot.metadata)) {
+ if (CoordinatedCommitsUtils.tablePropertiesPresent(table.initialSnapshot.metadata)) {
traceRemovalNeeded = true
try {
AlterTableUnsetPropertiesDeltaCommand(
- table, ManagedCommitUtils.TABLE_PROPERTY_KEYS, ifExists = true).run(table.spark)
+ table, CoordinatedCommitsUtils.TABLE_PROPERTY_KEYS, ifExists = true).run(table.spark)
} catch {
case NonFatal(e) =>
exceptionOpt = Some(e)
@@ -243,18 +243,18 @@ case class ManagedCommitPreDowngradeCommand(table: DeltaTableV2)
var postDisablementUnbackfilledCommitsPresent = false
if (exceptionOpt.isEmpty) {
val snapshotAfterDisabling = table.deltaLog.update()
- assert(snapshotAfterDisabling.tableCommitOwnerClientOpt.isEmpty)
+ assert(snapshotAfterDisabling.tableCommitCoordinatorClientOpt.isEmpty)
postDisablementUnbackfilledCommitsPresent =
- ManagedCommitUtils.unbackfilledCommitsPresent(snapshotAfterDisabling)
+ CoordinatedCommitsUtils.unbackfilledCommitsPresent(snapshotAfterDisabling)
if (postDisablementUnbackfilledCommitsPresent) {
traceRemovalNeeded = true
// Managed commits have already been disabled but there are unbackfilled commits.
- ManagedCommitUtils.backfillWhenManagedCommitDisabled(snapshotAfterDisabling)
+ CoordinatedCommitsUtils.backfillWhenCoordinatedCommitsDisabled(snapshotAfterDisabling)
}
}
recordDeltaEvent(
table.deltaLog,
- opType = "delta.managedCommitFeatureRemovalMetrics",
+ opType = "delta.coordinatedCommitsFeatureRemovalMetrics",
data = Map(
"downgradeTimeMs" -> TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeNs),
"traceRemovalNeeded" -> traceRemovalNeeded.toString,
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 5307354ebd1..7f8026784c2 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
@@ -21,7 +21,7 @@ import scala.collection.mutable
import org.apache.spark.sql.delta.actions._
import org.apache.spark.sql.delta.actions.Action.logSchema
-import org.apache.spark.sql.delta.managedcommit.{CommitOwnerClient, CommitOwnerProvider, ManagedCommitUtils, TableCommitOwnerClient}
+import org.apache.spark.sql.delta.coordinatedcommits.{CommitCoordinatorClient, CommitCoordinatorProvider, CoordinatedCommitsUtils, TableCommitCoordinatorClient}
import org.apache.spark.sql.delta.metering.DeltaLogging
import org.apache.spark.sql.delta.schema.SchemaUtils
import org.apache.spark.sql.delta.sources.DeltaSQLConf
@@ -228,13 +228,15 @@ class Snapshot(
}
/**
- * [[CommitOwnerClient]] for the given delta table as of this snapshot.
- * - This must be present when managed commit is enabled.
- * - This must be None when managed commit is disabled.
+ * [[CommitCoordinatorClient]] for the given delta table as of this snapshot.
+ * - This must be present when coordinated commits is enabled.
+ * - This must be None when coordinated commits is disabled.
*/
- val tableCommitOwnerClientOpt: Option[TableCommitOwnerClient] = initializeTableCommitOwner()
- protected def initializeTableCommitOwner(): Option[TableCommitOwnerClient] = {
- ManagedCommitUtils.getTableCommitOwner(spark, this)
+ val tableCommitCoordinatorClientOpt: Option[TableCommitCoordinatorClient] = {
+ initializeTableCommitCoordinator()
+ }
+ protected def initializeTableCommitCoordinator(): Option[TableCommitCoordinatorClient] = {
+ CoordinatedCommitsUtils.getTableCommitCoordinator(spark, this)
}
/** Number of columns to collect stats on for data skipping */
@@ -474,20 +476,20 @@ class Snapshot(
* Ensures that commit files are backfilled up to the current version in the snapshot.
*
* This method checks if there are any un-backfilled versions up to the current version and
- * triggers the backfilling process using the commit-owner. It verifies that the delta file for
- * the current version exists after the backfilling process.
+ * triggers the backfilling process using the commit-coordinator. It verifies that the delta file
+ * for the current version exists after the backfilling process.
*
* @throws IllegalStateException
* if the delta file for the current version is not found after backfilling.
*/
def ensureCommitFilesBackfilled(): Unit = {
- val tableCommitOwnerClient = tableCommitOwnerClientOpt.getOrElse {
+ val tableCommitCoordinatorClient = tableCommitCoordinatorClientOpt.getOrElse {
return
}
val minUnbackfilledVersion = DeltaCommitFileProvider(this).minUnbackfilledVersion
if (minUnbackfilledVersion <= version) {
val hadoopConf = deltaLog.newDeltaHadoopConf()
- tableCommitOwnerClient.backfillToVersion(
+ tableCommitCoordinatorClient.backfillToVersion(
version,
lastKnownBackfilledVersion = Some(minUnbackfilledVersion - 1))
val fs = deltaLog.logPath.getFileSystem(hadoopConf)
@@ -594,7 +596,7 @@ class InitialSnapshot(
override def protocol: Protocol = computedState.protocol
override protected lazy val getInCommitTimestampOpt: Option[Long] = None
- // The [[InitialSnapshot]] is not backed by any external commit-owner.
- override def initializeTableCommitOwner(): Option[TableCommitOwnerClient] = None
+ // The [[InitialSnapshot]] is not backed by any external commit-coordinator.
+ override def initializeTableCommitCoordinator(): Option[TableCommitCoordinatorClient] = None
override def timestamp: Long = -1L
}
diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/SnapshotManagement.scala b/spark/src/main/scala/org/apache/spark/sql/delta/SnapshotManagement.scala
index 4c32d6fb98a..a4db5df2ab1 100644
--- a/spark/src/main/scala/org/apache/spark/sql/delta/SnapshotManagement.scala
+++ b/spark/src/main/scala/org/apache/spark/sql/delta/SnapshotManagement.scala
@@ -29,7 +29,7 @@ import scala.util.control.NonFatal
import com.databricks.spark.util.TagDefinitions.TAG_ASYNC
import org.apache.spark.sql.delta.actions.Metadata
-import org.apache.spark.sql.delta.managedcommit.{Commit, GetCommitsResponse, TableCommitOwnerClient}
+import org.apache.spark.sql.delta.coordinatedcommits.{Commit, GetCommitsResponse, TableCommitCoordinatorClient}
import org.apache.spark.sql.delta.sources.DeltaSQLConf
import org.apache.spark.sql.delta.util.FileNames._
import org.apache.spark.sql.delta.util.JsonUtils
@@ -142,16 +142,16 @@ trait SnapshotManagement { self: DeltaLog =>
/**
* This method is designed to efficiently and reliably list delta, compacted delta, and
* checkpoint files associated with a Delta Lake table. It makes parallel calls to both the
- * file-system and a commit-owner (if available), reconciles the results to account for
+ * file-system and a commit-coordinator (if available), reconciles the results to account for
* asynchronous backfill operations, and ensures a comprehensive list of file statuses without
* missing any concurrently backfilled files.
- * *Note*: If table is a managed-commit table, the commit store MUST be passed to correctly list
- * the commits.
+ * *Note*: If table is a coordinated-commits table, the commit store MUST be passed to correctly
+ * list the commits.
* The function also collects the latest checksum file found in the listings and returns it.
*
* @param startVersion the version to start. Inclusive.
- * @param tableCommitOwnerClientOpt the optional commit store to use for fetching un-backfilled
- * commits.
+ * @param tableCommitCoordinatorClientOpt the optional commit store to use for fetching
+ * un-backfilled commits.
* @param versionToLoad the optional parameter to set the max version we should return. Inclusive.
* @param includeMinorCompactions Whether to include minor compaction files in the result
* @return A tuple where the first element is an array of log files (possibly empty, if no
@@ -160,20 +160,20 @@ trait SnapshotManagement { self: DeltaLog =>
*/
protected def listDeltaCompactedDeltaCheckpointFilesAndLatestChecksumFile(
startVersion: Long,
- tableCommitOwnerClientOpt: Option[TableCommitOwnerClient],
+ tableCommitCoordinatorClientOpt: Option[TableCommitCoordinatorClient],
versionToLoad: Option[Long],
includeMinorCompactions: Boolean): (Option[Array[FileStatus]], Option[FileStatus]) = {
- val tableCommitOwnerClient = tableCommitOwnerClientOpt.getOrElse {
+ val tableCommitCoordinatorClient = tableCommitCoordinatorClientOpt.getOrElse {
val (filesOpt, checksumOpt) =
listFromFileSystemInternal(startVersion, versionToLoad, includeMinorCompactions)
return (filesOpt.map(_.map(_._1)), checksumOpt)
}
// Submit a potential async call to get commits from commit store if available
- val threadPool = SnapshotManagement.commitOwnerGetCommitsThreadPool
+ val threadPool = SnapshotManagement.commitCoordinatorGetCommitsThreadPool
def getCommitsTask(async: Boolean): GetCommitsResponse = {
recordFrameProfile("DeltaLog", s"CommitStore.getCommits.async=$async") {
- tableCommitOwnerClient.getCommits(Some(startVersion), endVersion = versionToLoad)
+ tableCommitCoordinatorClient.getCommits(Some(startVersion), endVersion = versionToLoad)
}
}
val unbackfilledCommitsResponseFuture =
@@ -205,12 +205,12 @@ trait SnapshotManagement { self: DeltaLog =>
unbackfilledCommitsResponseFuture.get()
} catch {
case e: java.util.concurrent.ExecutionException =>
- throw new CommitOwnerGetCommitsFailedException(e.getCause)
+ throw new CommitCoordinatorGetCommitsFailedException(e.getCause)
}
def requiresAdditionalListing(): Boolean = {
// A gap in delta versions may occur if some delta files are backfilled "after" the
- // file-system listing but before the commit-owner listing. To handle this scenario, we
+ // file-system listing but before the commit-coordinator listing. To handle this scenario, we
// perform an additional listing from the file system because those missing files would be
// backfilled by now and show up in the file-system.
// Note: We only care about missing delta files with version <= versionToLoad
@@ -243,8 +243,8 @@ trait SnapshotManagement { self: DeltaLog =>
}
}
if (requiresAdditionalListing()) {
- // We should not have any gaps in File-System versions and CommitOwner versions after the
- // additional listing.
+ // We should not have any gaps in File-System versions and CommitCoordinator versions after
+ // the additional listing.
val eventData = Map(
"initialLogsFromFsListingOpt" ->
initialLogTuplesFromFsListingOpt.map(_.map(_._1.getPath.toString)),
@@ -293,15 +293,15 @@ trait SnapshotManagement { self: DeltaLog =>
/**
* This method is designed to efficiently and reliably list delta, compacted delta, and
* checkpoint files associated with a Delta Lake table. It makes parallel calls to both the
- * file-system and a commit-owner (if available), reconciles the results to account for
+ * file-system and a commit-coordinator (if available), reconciles the results to account for
* asynchronous backfill operations, and ensures a comprehensive list of file statuses without
* missing any concurrently backfilled files.
- * *Note*: If table is a managed-commit table, the commit-owner client MUST be passed to correctly
- * list the commits.
+ * *Note*: If table is a coordinated-commits table, the commit-coordinator client MUST be passed
+ * to correctly list the commits.
*
* @param startVersion the version to start. Inclusive.
- * @param tableCommitOwnerClientOpt the optional commit-owner client to use for fetching
- * un-backfilled commits.
+ * @param tableCommitCoordinatorClientOpt the optional commit-coordinator client to use for
+ * fetching un-backfilled commits.
* @param versionToLoad the optional parameter to set the max version we should return. Inclusive.
* @param includeMinorCompactions Whether to include minor compaction files in the result
* @return Some array of files found (possibly empty, if no usable commit files are present), or
@@ -309,13 +309,13 @@ trait SnapshotManagement { self: DeltaLog =>
*/
protected final def listDeltaCompactedDeltaAndCheckpointFiles(
startVersion: Long,
- tableCommitOwnerClientOpt: Option[TableCommitOwnerClient],
+ tableCommitCoordinatorClientOpt: Option[TableCommitCoordinatorClient],
versionToLoad: Option[Long],
includeMinorCompactions: Boolean): Option[Array[FileStatus]] = {
recordDeltaOperation(self, "delta.deltaLog.listDeltaAndCheckpointFiles") {
val (logTuplesOpt, latestChecksumOpt) =
listDeltaCompactedDeltaCheckpointFilesAndLatestChecksumFile(
- startVersion, tableCommitOwnerClientOpt, versionToLoad, includeMinorCompactions)
+ startVersion, tableCommitCoordinatorClientOpt, versionToLoad, includeMinorCompactions)
lastSeenChecksumFileStatusOpt = latestChecksumOpt
logTuplesOpt
}
@@ -341,7 +341,7 @@ trait SnapshotManagement { self: DeltaLog =>
protected def createLogSegment(
versionToLoad: Option[Long] = None,
oldCheckpointProviderOpt: Option[UninitializedCheckpointProvider] = None,
- tableCommitOwnerClientOpt: Option[TableCommitOwnerClient] = None,
+ tableCommitCoordinatorClientOpt: Option[TableCommitCoordinatorClient] = None,
lastCheckpointInfo: Option[LastCheckpointInfo] = None): Option[LogSegment] = {
// List based on the last known checkpoint version.
// if that is -1, list from version 0L
@@ -350,13 +350,13 @@ trait SnapshotManagement { self: DeltaLog =>
val includeMinorCompactions =
spark.conf.get(DeltaSQLConf.DELTALOG_MINOR_COMPACTION_USE_FOR_READS)
val newFiles = listDeltaCompactedDeltaAndCheckpointFiles(
- listingStartVersion, tableCommitOwnerClientOpt, versionToLoad, includeMinorCompactions)
+ listingStartVersion, tableCommitCoordinatorClientOpt, versionToLoad, includeMinorCompactions)
getLogSegmentForVersion(
versionToLoad,
newFiles,
validateLogSegmentWithoutCompactedDeltas = true,
oldCheckpointProviderOpt = oldCheckpointProviderOpt,
- tableCommitOwnerClientOpt = tableCommitOwnerClientOpt,
+ tableCommitCoordinatorClientOpt = tableCommitCoordinatorClientOpt,
lastCheckpointInfo = lastCheckpointInfo
)
}
@@ -364,7 +364,7 @@ trait SnapshotManagement { self: DeltaLog =>
private def createLogSegment(previousSnapshot: Snapshot): Option[LogSegment] = {
createLogSegment(
oldCheckpointProviderOpt = Some(previousSnapshot.checkpointProvider),
- tableCommitOwnerClientOpt = previousSnapshot.tableCommitOwnerClientOpt)
+ tableCommitCoordinatorClientOpt = previousSnapshot.tableCommitCoordinatorClientOpt)
}
/**
@@ -419,14 +419,14 @@ trait SnapshotManagement { self: DeltaLog =>
/**
* Helper function for the getLogSegmentForVersion above. Called with a provided files list,
* and will then try to construct a new LogSegment using that.
- * *Note*: If table is a managed-commit table, the commit-owner MUST be passed to correctly list
- * the commits.
+ * *Note*: If table is a coordinated-commits table, the commit-coordinator MUST be passed to
+ * correctly list the commits.
*/
protected def getLogSegmentForVersion(
versionToLoad: Option[Long],
files: Option[Array[FileStatus]],
validateLogSegmentWithoutCompactedDeltas: Boolean,
- tableCommitOwnerClientOpt: Option[TableCommitOwnerClient],
+ tableCommitCoordinatorClientOpt: Option[TableCommitCoordinatorClient],
oldCheckpointProviderOpt: Option[UninitializedCheckpointProvider],
lastCheckpointInfo: Option[LastCheckpointInfo]): Option[LogSegment] = {
recordFrameProfile("Delta", "SnapshotManagement.getLogSegmentForVersion") {
@@ -471,7 +471,7 @@ trait SnapshotManagement { self: DeltaLog =>
recordDeltaEvent(this, "delta.checkpoint.error.partial")
val snapshotVersion = versionToLoad.getOrElse(deltaVersion(deltas.last))
getLogSegmentWithMaxExclusiveCheckpointVersion(
- snapshotVersion, lastCheckpointVersion, tableCommitOwnerClientOpt)
+ snapshotVersion, lastCheckpointVersion, tableCommitCoordinatorClientOpt)
.foreach { alternativeLogSegment => return Some(alternativeLogSegment) }
// No alternative found, but the directory contains files so we cannot return None.
@@ -630,7 +630,7 @@ trait SnapshotManagement { self: DeltaLog =>
val snapshot = getUpdatedSnapshot(
oldSnapshotOpt = None,
initialSegmentForNewSnapshot = initialSegmentForNewSnapshot,
- initialTableCommitOwnerClient = None,
+ initialTableCommitCoordinatorClient = None,
isAsync = false)
CapturedSnapshot(snapshot, snapshotInitWallclockTime)
}
@@ -668,14 +668,14 @@ trait SnapshotManagement { self: DeltaLog =>
protected def createSnapshot(
initSegment: LogSegment,
- tableCommitOwnerClientOpt: Option[TableCommitOwnerClient],
+ tableCommitCoordinatorClientOpt: Option[TableCommitCoordinatorClient],
checksumOpt: Option[VersionChecksum]): Snapshot = {
val startingFrom = if (!initSegment.checkpointProvider.isEmpty) {
s" starting from checkpoint version ${initSegment.checkpointProvider.version}."
} else "."
logInfo(s"Loading version ${initSegment.version}$startingFrom")
createSnapshotFromGivenOrEquivalentLogSegment(
- initSegment, tableCommitOwnerClientOpt) { segment =>
+ initSegment, tableCommitCoordinatorClientOpt) { segment =>
new Snapshot(
path = logPath,
version = segment.version,
@@ -693,13 +693,13 @@ trait SnapshotManagement { self: DeltaLog =>
* 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`.
- * *Note*: If table is a managed-commit table, the commit-owner MUST be passed to correctly list
- * the commits.
+ * *Note*: If table is a coordinated-commits table, the commit-coordinator MUST be passed to
+ * correctly list the commits.
*/
private def getLogSegmentWithMaxExclusiveCheckpointVersion(
snapshotVersion: Long,
maxExclusiveCheckpointVersion: Long,
- tableCommitOwnerClientOpt: Option[TableCommitOwnerClient]): Option[LogSegment] = {
+ tableCommitCoordinatorClientOpt: Option[TableCommitCoordinatorClient]): Option[LogSegment] = {
assert(
snapshotVersion >= maxExclusiveCheckpointVersion,
s"snapshotVersion($snapshotVersion) is less than " +
@@ -711,7 +711,7 @@ trait SnapshotManagement { self: DeltaLog =>
case Some(cp) =>
val filesSinceCheckpointVersion = listDeltaCompactedDeltaAndCheckpointFiles(
startVersion = cp.version,
- tableCommitOwnerClientOpt = tableCommitOwnerClientOpt,
+ tableCommitCoordinatorClientOpt = tableCommitCoordinatorClientOpt,
versionToLoad = Some(snapshotVersion),
includeMinorCompactions = false
).getOrElse(Array.empty)
@@ -752,7 +752,7 @@ trait SnapshotManagement { self: DeltaLog =>
val listFromResult =
listDeltaCompactedDeltaAndCheckpointFiles(
startVersion = 0,
- tableCommitOwnerClientOpt = tableCommitOwnerClientOpt,
+ tableCommitCoordinatorClientOpt = tableCommitCoordinatorClientOpt,
versionToLoad = Some(snapshotVersion),
includeMinorCompactions = false)
val (deltas, deltaVersions) =
@@ -785,7 +785,7 @@ trait SnapshotManagement { self: DeltaLog =>
newChecksumOpt: Option[VersionChecksum],
preCommitLogSegment: LogSegment,
commit: Commit,
- tableCommitOwnerClientOpt: Option[TableCommitOwnerClient],
+ tableCommitCoordinatorClientOpt: Option[TableCommitCoordinatorClient],
oldCheckpointProvider: CheckpointProvider): LogSegment = recordFrameProfile(
"Delta", "SnapshotManagement.getLogSegmentAfterCommit") {
// If the table doesn't have any competing updates, then go ahead and use the optimized
@@ -799,12 +799,12 @@ trait SnapshotManagement { self: DeltaLog =>
} else {
val latestCheckpointProvider =
Seq(preCommitLogSegment.checkpointProvider, oldCheckpointProvider).maxBy(_.version)
- getLogSegmentAfterCommit(tableCommitOwnerClientOpt, latestCheckpointProvider)
+ getLogSegmentAfterCommit(tableCommitCoordinatorClientOpt, latestCheckpointProvider)
}
}
protected[delta] def getLogSegmentAfterCommit(
- tableCommitOwnerClientOpt: Option[TableCommitOwnerClient],
+ tableCommitCoordinatorClientOpt: Option[TableCommitCoordinatorClient],
oldCheckpointProvider: UninitializedCheckpointProvider): LogSegment = {
/**
* We can't specify `versionToLoad = committedVersion` for the call below.
@@ -817,7 +817,7 @@ trait SnapshotManagement { self: DeltaLog =>
*/
createLogSegment(
oldCheckpointProviderOpt = Some(oldCheckpointProvider),
- tableCommitOwnerClientOpt = tableCommitOwnerClientOpt
+ tableCommitCoordinatorClientOpt = tableCommitCoordinatorClientOpt
).getOrElse {
// This shouldn't be possible right after a commit
logError(s"No delta log found for the Delta table at $logPath")
@@ -832,7 +832,7 @@ trait SnapshotManagement { self: DeltaLog =>
*/
protected def createSnapshotFromGivenOrEquivalentLogSegment(
initSegment: LogSegment,
- tableCommitOwnerClientOpt: Option[TableCommitOwnerClient])
+ tableCommitCoordinatorClientOpt: Option[TableCommitCoordinatorClient])
(snapshotCreator: LogSegment => Snapshot): Snapshot = {
val numRetries =
spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_SNAPSHOT_LOADING_MAX_RETRIES)
@@ -855,7 +855,7 @@ trait SnapshotManagement { self: DeltaLog =>
segment = getLogSegmentWithMaxExclusiveCheckpointVersion(
segment.version,
segment.checkpointProvider.version,
- tableCommitOwnerClientOpt).getOrElse {
+ tableCommitCoordinatorClientOpt).getOrElse {
// Throw the first error if we cannot find an equivalent `LogSegment`.
throw firstError
}
@@ -882,11 +882,12 @@ trait SnapshotManagement { self: DeltaLog =>
*/
def getUpdatedLogSegment(
oldLogSegment: LogSegment,
- tableCommitOwnerClientOpt: Option[TableCommitOwnerClient]): (LogSegment, Seq[FileStatus]) = {
+ tableCommitCoordinatorClientOpt: Option[TableCommitCoordinatorClient]
+ ): (LogSegment, Seq[FileStatus]) = {
val includeCompactions = spark.conf.get(DeltaSQLConf.DELTALOG_MINOR_COMPACTION_USE_FOR_READS)
val newFilesOpt = listDeltaCompactedDeltaAndCheckpointFiles(
startVersion = oldLogSegment.version + 1,
- tableCommitOwnerClientOpt = tableCommitOwnerClientOpt,
+ tableCommitCoordinatorClientOpt = tableCommitCoordinatorClientOpt,
versionToLoad = None,
includeMinorCompactions = includeCompactions)
val newFiles = newFilesOpt.getOrElse {
@@ -904,7 +905,7 @@ trait SnapshotManagement { self: DeltaLog =>
versionToLoad = None,
files = Some(allFiles),
validateLogSegmentWithoutCompactedDeltas = false,
- tableCommitOwnerClientOpt = tableCommitOwnerClientOpt,
+ tableCommitCoordinatorClientOpt = tableCommitCoordinatorClientOpt,
lastCheckpointInfo = lastCheckpointInfo,
oldCheckpointProviderOpt = Some(oldLogSegment.checkpointProvider)
).getOrElse(oldLogSegment)
@@ -1036,17 +1037,18 @@ trait SnapshotManagement { self: DeltaLog =>
val newSnapshot = getUpdatedSnapshot(
oldSnapshotOpt = Some(previousSnapshot),
initialSegmentForNewSnapshot = segmentOpt,
- initialTableCommitOwnerClient = previousSnapshot.tableCommitOwnerClientOpt,
+ initialTableCommitCoordinatorClient = previousSnapshot.tableCommitCoordinatorClientOpt,
isAsync = isAsync)
installSnapshot(newSnapshot, updateStartTimeMs)
}
/**
* Updates and installs a new snapshot in the `currentSnapshot`.
- * This method takes care of recursively creating new snapshots if the commit-owner has changed.
+ * This method takes care of recursively creating new snapshots if the commit-coordinator has
+ * changed.
* @param oldSnapshotOpt The previous snapshot, if any.
* @param initialSegmentForNewSnapshot the log segment constructed for the new snapshot
- * @param initialTableCommitOwnerClient the commit-owner used for constructing the
+ * @param initialTableCommitCoordinatorClient the commit-coordinator used for constructing the
* `initialSegmentForNewSnapshot`
* @param isAsync Whether the update is async.
* @return The new snapshot.
@@ -1054,26 +1056,27 @@ trait SnapshotManagement { self: DeltaLog =>
protected def getUpdatedSnapshot(
oldSnapshotOpt: Option[Snapshot],
initialSegmentForNewSnapshot: Option[LogSegment],
- initialTableCommitOwnerClient: Option[TableCommitOwnerClient],
+ initialTableCommitCoordinatorClient: Option[TableCommitCoordinatorClient],
isAsync: Boolean): Snapshot = {
var newSnapshot = getSnapshotForLogSegmentInternal(
oldSnapshotOpt,
initialSegmentForNewSnapshot,
- initialTableCommitOwnerClient,
+ initialTableCommitCoordinatorClient,
isAsync
)
- // Identify whether the snapshot was created using a "stale" commit-owner. If yes, we need to
- // again invoke [[updateSnapshot]] so that we could get the latest commits from the updated
- // commit-owner client. We need to do it only once as the delta spec mandates the commit which
- // changes the commit-owner to be backfilled.
- val usedStaleCommitOwner = newSnapshot.tableCommitOwnerClientOpt.exists { newStore =>
- initialTableCommitOwnerClient.forall(!_.semanticsEquals(newStore))
- }
- if (usedStaleCommitOwner) {
+ // Identify whether the snapshot was created using a "stale" commit-coordinator. If yes, we need
+ // to again invoke [[updateSnapshot]] so that we could get the latest commits from the updated
+ // commit-coordinator client. We need to do it only once as the delta spec mandates the commit
+ // which changes the commit-coordinator to be backfilled.
+ val usedStaleCommitCoordinator =
+ newSnapshot.tableCommitCoordinatorClientOpt.exists { newStore =>
+ initialTableCommitCoordinatorClient.forall(!_.semanticsEquals(newStore))
+ }
+ if (usedStaleCommitCoordinator) {
val segmentOpt = createLogSegment(newSnapshot)
newSnapshot =
getSnapshotForLogSegmentInternal(
- Some(newSnapshot), segmentOpt, newSnapshot.tableCommitOwnerClientOpt, isAsync)
+ Some(newSnapshot), segmentOpt, newSnapshot.tableCommitCoordinatorClientOpt, isAsync)
}
newSnapshot
}
@@ -1082,7 +1085,7 @@ trait SnapshotManagement { self: DeltaLog =>
protected def getSnapshotForLogSegmentInternal(
previousSnapshotOpt: Option[Snapshot],
segmentOpt: Option[LogSegment],
- tableCommitOwnerClientOpt: Option[TableCommitOwnerClient],
+ tableCommitCoordinatorClientOpt: Option[TableCommitCoordinatorClient],
isAsync: Boolean): Snapshot = {
segmentOpt.map { segment =>
if (previousSnapshotOpt.exists(_.logSegment == segment)) {
@@ -1090,7 +1093,7 @@ trait SnapshotManagement { self: DeltaLog =>
} else {
val newSnapshot = createSnapshot(
initSegment = segment,
- tableCommitOwnerClientOpt = tableCommitOwnerClientOpt,
+ tableCommitCoordinatorClientOpt = tableCommitCoordinatorClientOpt,
checksumOpt = None)
previousSnapshotOpt.foreach(logMetadataTableIdChange(_, newSnapshot))
logInfo(s"Updated snapshot to $newSnapshot")
@@ -1149,12 +1152,12 @@ trait SnapshotManagement { self: DeltaLog =>
protected def createSnapshotAfterCommit(
initSegment: LogSegment,
newChecksumOpt: Option[VersionChecksum],
- tableCommitOwnerClientOpt: Option[TableCommitOwnerClient],
+ tableCommitCoordinatorClientOpt: Option[TableCommitCoordinatorClient],
committedVersion: Long): Snapshot = {
logInfo(s"Creating a new snapshot v${initSegment.version} for commit version $committedVersion")
createSnapshot(
initSegment,
- tableCommitOwnerClientOpt = tableCommitOwnerClientOpt,
+ tableCommitCoordinatorClientOpt = tableCommitCoordinatorClientOpt,
checksumOpt = newChecksumOpt
)
}
@@ -1183,7 +1186,7 @@ trait SnapshotManagement { self: DeltaLog =>
newChecksumOpt,
preCommitLogSegment,
commit,
- previousSnapshot.tableCommitOwnerClientOpt,
+ previousSnapshot.tableCommitCoordinatorClientOpt,
previousSnapshot.checkpointProvider)
// This likely implies a list-after-write inconsistency
@@ -1198,7 +1201,7 @@ trait SnapshotManagement { self: DeltaLog =>
val newSnapshot = createSnapshotAfterCommit(
segment,
newChecksumOpt,
- previousSnapshot.tableCommitOwnerClientOpt,
+ previousSnapshot.tableCommitCoordinatorClientOpt,
committedVersion)
logMetadataTableIdChange(previousSnapshot, newSnapshot)
logInfo(s"Updated snapshot to $newSnapshot")
@@ -1254,7 +1257,7 @@ trait SnapshotManagement { self: DeltaLog =>
val logSegmentOpt = createLogSegment(
versionToLoad = Some(version),
oldCheckpointProviderOpt = lastCheckpointProviderOpt,
- tableCommitOwnerClientOpt = upperBoundSnapshot.tableCommitOwnerClientOpt,
+ tableCommitCoordinatorClientOpt = upperBoundSnapshot.tableCommitCoordinatorClientOpt,
lastCheckpointInfo = lastCheckpointInfoOpt)
val logSegment = logSegmentOpt.getOrElse {
// We can't return InitialSnapshot because our caller asked for a specific snapshot version.
@@ -1262,7 +1265,7 @@ trait SnapshotManagement { self: DeltaLog =>
}
createSnapshot(
initSegment = logSegment,
- tableCommitOwnerClientOpt = upperBoundSnapshot.tableCommitOwnerClientOpt,
+ tableCommitCoordinatorClientOpt = upperBoundSnapshot.tableCommitCoordinatorClientOpt,
checksumOpt = None)
}
@@ -1284,10 +1287,10 @@ object SnapshotManagement {
new DeltaThreadPool(tpe)
}
- private lazy val commitOwnerGetCommitsThreadPool = {
+ private lazy val commitCoordinatorGetCommitsThreadPool = {
val numThreads = SparkSession.active.sessionState.conf
- .getConf(DeltaSQLConf.MANAGED_COMMIT_GET_COMMITS_THREAD_POOL_SIZE)
- val tpe = ThreadUtils.newDaemonCachedThreadPool("commit-owner-get-commits", numThreads)
+ .getConf(DeltaSQLConf.COORDINATED_COMMITS_GET_COMMITS_THREAD_POOL_SIZE)
+ val tpe = ThreadUtils.newDaemonCachedThreadPool("commit-coordinator-get-commits", numThreads)
new DeltaThreadPool(tpe)
}
@@ -1405,8 +1408,8 @@ case class LogSegment(
}
}
-/** Exception thrown When [[TableCommitOwnerClient.getCommits]] fails due to any reason. */
-class CommitOwnerGetCommitsFailedException(cause: Throwable) extends Exception(cause)
+/** Exception thrown When [[TableCommitCoordinatorClient.getCommits]] fails due to any reason. */
+class CommitCoordinatorGetCommitsFailedException(cause: Throwable) extends Exception(cause)
object LogSegment {
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 25513c2e657..36ce1ce987b 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
@@ -21,7 +21,7 @@ import java.util.Locale
import org.apache.spark.sql.delta.actions._
import org.apache.spark.sql.delta.catalog.DeltaTableV2
import org.apache.spark.sql.delta.constraints.{Constraints, Invariants}
-import org.apache.spark.sql.delta.managedcommit.ManagedCommitUtils
+import org.apache.spark.sql.delta.coordinatedcommits.CoordinatedCommitsUtils
import org.apache.spark.sql.delta.schema.SchemaMergingUtils
import org.apache.spark.sql.delta.schema.SchemaUtils
import org.apache.spark.sql.delta.sources.DeltaSQLConf
@@ -344,7 +344,7 @@ object TableFeature {
RowTrackingFeature,
InCommitTimestampTableFeature,
VariantTypeTableFeature,
- ManagedCommitTableFeature)
+ CoordinatedCommitsTableFeature)
if (DeltaUtils.isTesting) {
features ++= Set(
TestLegacyWriterFeature,
@@ -672,9 +672,9 @@ object V2CheckpointTableFeature
V2CheckpointPreDowngradeCommand(table)
}
-/** Table feature to represent tables whose commits are managed by separate commit-owner */
-object ManagedCommitTableFeature
- extends WriterFeature(name = "managedCommit-preview")
+/** Table feature to represent tables whose commits are managed by separate commit-coordinator */
+object CoordinatedCommitsTableFeature
+ extends WriterFeature(name = "coordinatedCommits-preview")
with FeatureAutomaticallyEnabledByMetadata
with RemovableFeature {
@@ -683,18 +683,18 @@ object ManagedCommitTableFeature
override def metadataRequiresFeatureToBeEnabled(
metadata: Metadata,
spark: SparkSession): Boolean = {
- DeltaConfigs.MANAGED_COMMIT_OWNER_NAME.fromMetaData(metadata).nonEmpty
+ DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_NAME.fromMetaData(metadata).nonEmpty
}
override def requiredFeatures: Set[TableFeature] =
Set(InCommitTimestampTableFeature, VacuumProtocolCheckTableFeature)
override def preDowngradeCommand(table: DeltaTableV2)
- : PreDowngradeTableFeatureCommand = ManagedCommitPreDowngradeCommand(table)
+ : PreDowngradeTableFeatureCommand = CoordinatedCommitsPreDowngradeCommand(table)
override def validateRemoval(snapshot: Snapshot): Boolean = {
- !ManagedCommitUtils.tablePropertiesPresent(snapshot.metadata) &&
- !ManagedCommitUtils.unbackfilledCommitsPresent(snapshot)
+ !CoordinatedCommitsUtils.tablePropertiesPresent(snapshot.metadata) &&
+ !CoordinatedCommitsUtils.unbackfilledCommitsPresent(snapshot)
}
// This is a writer feature, so it should directly return false.
diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/actions/actions.scala b/spark/src/main/scala/org/apache/spark/sql/delta/actions/actions.scala
index 291fa46e711..7b64f581553 100644
--- a/spark/src/main/scala/org/apache/spark/sql/delta/actions/actions.scala
+++ b/spark/src/main/scala/org/apache/spark/sql/delta/actions/actions.scala
@@ -28,7 +28,7 @@ import scala.util.control.NonFatal
import org.apache.spark.sql.delta._
import org.apache.spark.sql.delta.commands.DeletionVectorUtils
-import org.apache.spark.sql.delta.managedcommit.{AbstractCommitInfo, AbstractMetadata, AbstractProtocol}
+import org.apache.spark.sql.delta.coordinatedcommits.{AbstractCommitInfo, AbstractMetadata, AbstractProtocol}
import org.apache.spark.sql.delta.sources.DeltaSQLConf
import org.apache.spark.sql.delta.util.{JsonUtils, Utils => DeltaUtils}
import org.apache.spark.sql.delta.util.FileNames
@@ -1040,22 +1040,22 @@ case class Metadata(
GeneratedColumn.getOptimizablePartitionExpressions(schema, partitionSchema)
/**
- * The name of commit-owner which arbitrates the commits to the table. This must be available
- * if this is a managed-commit table.
+ * The name of commit-coordinator which arbitrates the commits to the table. This must be
+ * available if this is a coordinated-commits table.
*/
@JsonIgnore
- lazy val managedCommitOwnerName: Option[String] =
- DeltaConfigs.MANAGED_COMMIT_OWNER_NAME.fromMetaData(this)
+ lazy val coordinatedCommitsCoordinatorName: Option[String] =
+ DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_NAME.fromMetaData(this)
- /** The configuration to uniquely identify the commit-owner for managed-commit. */
+ /** The configuration to uniquely identify the commit-coordinator for coordinated-commits. */
@JsonIgnore
- lazy val managedCommitOwnerConf: Map[String, String] =
- DeltaConfigs.MANAGED_COMMIT_OWNER_CONF.fromMetaData(this)
+ lazy val coordinatedCommitsCoordinatorConf: Map[String, String] =
+ DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_CONF.fromMetaData(this)
- /** The table specific configuration for managed-commit. */
+ /** The table specific configuration for coordinated-commits. */
@JsonIgnore
- lazy val managedCommitTableConf: Map[String, String] =
- DeltaConfigs.MANAGED_COMMIT_TABLE_CONF.fromMetaData(this)
+ lazy val coordinatedCommitsTableConf: Map[String, String] =
+ DeltaConfigs.COORDINATED_COMMITS_TABLE_CONF.fromMetaData(this)
override def wrap: SingleAction = SingleAction(metaData = this)
diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/managedcommit/AbstractBatchBackfillingCommitOwnerClient.scala b/spark/src/main/scala/org/apache/spark/sql/delta/coordinatedcommits/AbstractBatchBackfillingCommitCoordinatorClient.scala
similarity index 80%
rename from spark/src/main/scala/org/apache/spark/sql/delta/managedcommit/AbstractBatchBackfillingCommitOwnerClient.scala
rename to spark/src/main/scala/org/apache/spark/sql/delta/coordinatedcommits/AbstractBatchBackfillingCommitCoordinatorClient.scala
index 9951a93a6c4..b4aac9ffa2b 100644
--- a/spark/src/main/scala/org/apache/spark/sql/delta/managedcommit/AbstractBatchBackfillingCommitOwnerClient.scala
+++ b/spark/src/main/scala/org/apache/spark/sql/delta/coordinatedcommits/AbstractBatchBackfillingCommitCoordinatorClient.scala
@@ -14,7 +14,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.delta.managedcommit
+package org.apache.spark.sql.delta.coordinatedcommits
import java.nio.file.FileAlreadyExistsException
import java.util.UUID
@@ -31,10 +31,10 @@ import org.apache.hadoop.fs.{FileStatus, Path}
import org.apache.spark.internal.Logging
/**
- * An abstract [[CommitOwnerClient]] which triggers backfills every n commits.
+ * An abstract [[CommitCoordinatorClient]] which triggers backfills every n commits.
* - every commit version which satisfies `commitVersion % batchSize == 0` will trigger a backfill.
*/
-trait AbstractBatchBackfillingCommitOwnerClient extends CommitOwnerClient with Logging {
+trait AbstractBatchBackfillingCommitCoordinatorClient extends CommitCoordinatorClient with Logging {
/**
* Size of batch that should be backfilled. So every commit version which satisfies
@@ -50,7 +50,7 @@ trait AbstractBatchBackfillingCommitOwnerClient extends CommitOwnerClient with L
logStore: LogStore,
hadoopConf: Configuration,
logPath: Path,
- managedCommitTableConf: Map[String, String],
+ coordinatedCommitsTableConf: Map[String, String],
commitVersion: Long,
commitFile: FileStatus,
commitTimestamp: Long): CommitResponse
@@ -59,12 +59,12 @@ trait AbstractBatchBackfillingCommitOwnerClient extends CommitOwnerClient with L
logStore: LogStore,
hadoopConf: Configuration,
logPath: Path,
- managedCommitTableConf: Map[String, String],
+ coordinatedCommitsTableConf: Map[String, String],
commitVersion: Long,
actions: Iterator[String],
updatedActions: UpdatedActions): CommitResponse = {
val executionObserver = TransactionExecutionObserver.threadObserver.get()
- val tablePath = ManagedCommitUtils.getTablePath(logPath)
+ val tablePath = CoordinatedCommitsUtils.getTablePath(logPath)
if (commitVersion == 0) {
throw CommitFailedException(
retryable = false, conflict = false, message = "Commit version 0 must go via filesystem.")
@@ -79,12 +79,12 @@ trait AbstractBatchBackfillingCommitOwnerClient extends CommitOwnerClient with L
logStore,
hadoopConf,
logPath,
- managedCommitTableConf,
+ coordinatedCommitsTableConf,
commitVersion - 1)
}
// Write new commit file in _commits directory
- val fileStatus = ManagedCommitUtils.writeCommitFile(
+ val fileStatus = CoordinatedCommitsUtils.writeCommitFile(
logStore, hadoopConf, logPath, commitVersion, actions, generateUUID())
// Do the actual commit
@@ -94,12 +94,12 @@ trait AbstractBatchBackfillingCommitOwnerClient extends CommitOwnerClient with L
logStore,
hadoopConf,
logPath,
- managedCommitTableConf,
+ coordinatedCommitsTableConf,
commitVersion,
fileStatus,
commitTimestamp)
- val mcToFsConversion = isManagedCommitToFSConversion(commitVersion, updatedActions)
+ val mcToFsConversion = isCoordinatedCommitsToFSConversion(commitVersion, updatedActions)
// Backfill if needed
executionObserver.beginBackfill()
if (batchSize <= 1) {
@@ -116,21 +116,21 @@ trait AbstractBatchBackfillingCommitOwnerClient extends CommitOwnerClient with L
logStore,
hadoopConf,
logPath,
- managedCommitTableConf,
+ coordinatedCommitsTableConf,
commitVersion)
}
logInfo(s"Commit $commitVersion done successfully on table $tablePath")
commitResponse
}
- private def isManagedCommitToFSConversion(
+ private def isCoordinatedCommitsToFSConversion(
commitVersion: Long,
updatedActions: UpdatedActions): Boolean = {
- val oldMetadataHasManagedCommit = updatedActions.getOldMetadata.asInstanceOf[Metadata]
- .managedCommitOwnerName.nonEmpty
- val newMetadataHasManagedCommit = updatedActions.getNewMetadata.asInstanceOf[Metadata]
- .managedCommitOwnerName.nonEmpty
- oldMetadataHasManagedCommit && !newMetadataHasManagedCommit && commitVersion > 0
+ val oldMetadataHasCoordinatedCommits = updatedActions.getOldMetadata.asInstanceOf[Metadata]
+ .coordinatedCommitsCoordinatorName.nonEmpty
+ val newMetadataHasCoordinatedCommits = updatedActions.getNewMetadata.asInstanceOf[Metadata]
+ .coordinatedCommitsCoordinatorName.nonEmpty
+ oldMetadataHasCoordinatedCommits && !newMetadataHasCoordinatedCommits && commitVersion > 0
}
protected def generateUUID(): String = UUID.randomUUID().toString
@@ -139,7 +139,7 @@ trait AbstractBatchBackfillingCommitOwnerClient extends CommitOwnerClient with L
logStore: LogStore,
hadoopConf: Configuration,
logPath: Path,
- managedCommitTableConf: Map[String, String],
+ coordinatedCommitsTableConf: Map[String, String],
version: Long,
lastKnownBackfilledVersionOpt: Option[Long] = None): Unit = {
// Confirm the last backfilled version by checking the backfilled delta file's existence.
@@ -148,7 +148,7 @@ trait AbstractBatchBackfillingCommitOwnerClient extends CommitOwnerClient with L
fs.exists(FileNames.unsafeDeltaFile(logPath, version))
}
val startVersionOpt = validLastKnownBackfilledVersionOpt.map(_ + 1)
- getCommits(logPath, managedCommitTableConf, startVersionOpt, Some(version))
+ getCommits(logPath, coordinatedCommitsTableConf, startVersionOpt, Some(version))
.getCommits
.foreach { commit =>
backfill(logStore, hadoopConf, logPath, commit.getVersion, commit.getFileStatus)
@@ -180,7 +180,9 @@ trait AbstractBatchBackfillingCommitOwnerClient extends CommitOwnerClient with L
}
}
- /** Callback to tell the CommitOwner that all commits <= `backfilledVersion` are backfilled. */
+ /**
+ * Callback to tell the CommitCoordinator that all commits <= `backfilledVersion` are backfilled.
+ */
protected[delta] def registerBackfill(
logPath: Path,
backfilledVersion: Long): Unit
diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/managedcommit/CommitOwnerClient.scala b/spark/src/main/scala/org/apache/spark/sql/delta/coordinatedcommits/CommitCoordinatorClient.scala
similarity index 55%
rename from spark/src/main/scala/org/apache/spark/sql/delta/managedcommit/CommitOwnerClient.scala
rename to spark/src/main/scala/org/apache/spark/sql/delta/coordinatedcommits/CommitCoordinatorClient.scala
index c3d731da157..6f80e3d03f7 100644
--- a/spark/src/main/scala/org/apache/spark/sql/delta/managedcommit/CommitOwnerClient.scala
+++ b/spark/src/main/scala/org/apache/spark/sql/delta/coordinatedcommits/CommitCoordinatorClient.scala
@@ -14,12 +14,12 @@
* limitations under the License.
*/
-package org.apache.spark.sql.delta.managedcommit
+package org.apache.spark.sql.delta.coordinatedcommits
import scala.collection.mutable
import org.apache.spark.sql.delta.storage.LogStore
-import io.delta.dynamodbcommitstore.DynamoDBCommitOwnerClientBuilder
+import io.delta.dynamodbcommitcoordinator.DynamoDBCommitCoordinatorClientBuilder
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileStatus, Path}
@@ -36,7 +36,7 @@ case class Commit(
}
/**
- * Exception raised by [[CommitOwnerClient.commit]] method.
+ * Exception raised by [[CommitCoordinatorClient.commit]] method.
* | retryable | conflict | meaning |
* | no | no | something bad happened (e.g. auth failure) |
* | no | yes | permanent transaction conflict (e.g. multi-table commit failed) |
@@ -51,12 +51,12 @@ case class CommitFailedException(
def getConflict: Boolean = conflict
}
-/** Response container for [[CommitOwnerClient.commit]] API */
+/** Response container for [[CommitCoordinatorClient.commit]] API */
case class CommitResponse(private val commit: Commit) {
def getCommit: Commit = commit
}
-/** Response container for [[CommitOwnerClient.getCommits]] API */
+/** Response container for [[CommitCoordinatorClient.getCommits]] API */
case class GetCommitsResponse(
private val commits: Seq[Commit],
private val latestTableVersion: Long) {
@@ -64,7 +64,10 @@ case class GetCommitsResponse(
def getLatestTableVersion: Long = latestTableVersion
}
-/** A container class to inform the [[CommitOwnerClient]] about any changes in Protocol/Metadata */
+/**
+ * A container class to inform the [[CommitCoordinatorClient]] about any changes in
+ * Protocol/Metadata.
+ */
case class UpdatedActions(
private val commitInfo: AbstractCommitInfo,
private val newMetadata: AbstractMetadata,
@@ -80,31 +83,35 @@ case class UpdatedActions(
}
/**
- * [[CommitOwnerClient]] is responsible for managing commits for a managed-commit delta table.
- * 1. It provides API to commit a new version of the table. See [[CommitOwnerClient.commit]] API.
+ * [[CommitCoordinatorClient]] is responsible for managing commits for a coordinated-commits delta
+ * table.
+ * 1. It provides API to commit a new version of the table. See [[CommitCoordinatorClient.commit]]
+ * API.
* 2. It makes sure that commits are backfilled if/when needed
- * 3. It also tracks and returns unbackfilled commits. See [[CommitOwnerClient.getCommits]] API.
+ * 3. It also tracks and returns unbackfilled commits. See [[CommitCoordinatorClient.getCommits]]
+ * API.
*/
-trait CommitOwnerClient {
+trait CommitCoordinatorClient {
/**
* API to register the table represented by the given `logPath` with the given
* `currentTableVersion`.
- * This API is called when the table is being converted from filesystem table to managed-commit
- * table.
+ * This API is called when the table is being converted from filesystem table to
+ * coordinated-commits table.
* - The `currentTableVersion` is the version of the table just before conversion.
* - The `currentTableVersion` + 1 represents the commit that will do the conversion. This must be
* backfilled atomically.
* - The `currentTableVersion` + 2 represents the first commit after conversion. This will go via
- * the [[CommitOwnerClient]] and it could choose whether it wants to write unbackfilled commits
- * and backfill them later.
- * When a new managed-commit table is being created, the `currentTableVersion` will be -1 and the
- * upgrade commit needs to be a file-system commit which will write the backfilled file directly.
+ * the [[CommitCoordinatorClient]] and it could choose whether it wants to write unbackfilled
+ * commits and backfill them later.
+ * When a new coordinated-commits table is being created, the `currentTableVersion` will be -1 and
+ * the upgrade commit needs to be a file-system commit which will write the backfilled file
+ * directly.
*
- * @return A map of key-value pairs which is issued by the commit-owner to identify the table.
- * This should be stored in the table's metadata. This information needs to be passed in
- * other table specific APIs like commit / getCommits / backfillToVersion to identify the
- * table.
+ * @return A map of key-value pairs which is issued by the commit-coordinator to identify the
+ * table. This should be stored in the table's metadata. This information needs to be
+ * passed in other table specific APIs like commit / getCommits / backfillToVersion to
+ * identify the table.
*/
def registerTable(
logPath: Path,
@@ -123,7 +130,7 @@ trait CommitOwnerClient {
logStore: LogStore,
hadoopConf: Configuration,
logPath: Path,
- managedCommitTableConf: Map[String, String],
+ coordinatedCommitsTableConf: Map[String, String],
commitVersion: Long,
actions: Iterator[String],
updatedActions: UpdatedActions): CommitResponse
@@ -134,59 +141,61 @@ trait CommitOwnerClient {
* returned commits are contiguous and in ascending version order.
* Note that the first version returned by this API may not be equal to the `startVersion`. This
* happens when few versions starting from `startVersion` are already backfilled and so
- * commit-owner may have stopped tracking them.
- * The returned latestTableVersion is the maximum commit version ratified by the Commit-Owner.
- * Note that returning latestTableVersion as -1 is acceptable only if the commit-owner never
+ * commit-coordinator may have stopped tracking them.
+ * The returned latestTableVersion is the maximum commit version ratified by the
+ * Commit-Coordinator.
+ * Note that returning latestTableVersion as -1 is acceptable only if the commit-coordinator never
* ratified any version i.e. it never accepted any un-backfilled commit.
*
* @return GetCommitsResponse which has a list of [[Commit]]s and the latestTableVersion which is
- * tracked by [[CommitOwnerClient]].
+ * tracked by [[CommitCoordinatorClient]].
*/
def getCommits(
logPath: Path,
- managedCommitTableConf: Map[String, String],
+ coordinatedCommitsTableConf: Map[String, String],
startVersion: Option[Long] = None,
endVersion: Option[Long] = None): GetCommitsResponse
/**
- * API to ask the Commit-Owner to backfill all commits > `lastKnownBackfilledVersion` and
+ * API to ask the Commit-Coordinator to backfill all commits > `lastKnownBackfilledVersion` and
* <= `endVersion`.
*
* If this API returns successfully, that means the backfill must have been completed, although
- * the Commit-Owner may not be aware of it yet.
+ * the Commit-Coordinator may not be aware of it yet.
*
- * @param version The version till which the Commit-Owner should backfill.
- * @param lastKnownBackfilledVersion The last known version that was backfilled by Commit-Owner
- * before this API was called. If it's None or invalid, then the
- * Commit-Owner should backfill from the beginning of the table.
+ * @param version The version till which the Commit-Coordinator should backfill.
+ * @param lastKnownBackfilledVersion The last known version that was backfilled by
+ * Commit-Coordinator before this API was called. If it's None
+ * or invalid, then the Commit-Coordinator should backfill from
+ * the beginning.
*/
def backfillToVersion(
logStore: LogStore,
hadoopConf: Configuration,
logPath: Path,
- managedCommitTableConf: Map[String, String],
+ coordinatedCommitsTableConf: Map[String, String],
version: Long,
lastKnownBackfilledVersion: Option[Long]): Unit
/**
- * Determines whether this [[CommitOwnerClient]] is semantically equal to another
- * [[CommitOwnerClient]].
+ * Determines whether this [[CommitCoordinatorClient]] is semantically equal to another
+ * [[CommitCoordinatorClient]].
*
- * Semantic equality is determined by each [[CommitOwnerClient]] implementation based on whether
- * the two instances can be used interchangeably when invoking any of the CommitOwnerClient APIs,
- * such as `commit`, `getCommits`, etc. For e.g., both the instances might be pointing to the same
- * underlying endpoint.
+ * Semantic equality is determined by each [[CommitCoordinatorClient]] implementation based on
+ * whether the two instances can be used interchangeably when invoking any of the
+ * CommitCoordinatorClient APIs, such as `commit`, `getCommits`, etc. For e.g., both the instances
+ * might be pointing to the same underlying endpoint.
*/
- def semanticEquals(other: CommitOwnerClient): Boolean
+ def semanticEquals(other: CommitCoordinatorClient): Boolean
}
-object CommitOwnerClient {
+object CommitCoordinatorClient {
def semanticEquals(
- commitOwnerClientOpt1: Option[CommitOwnerClient],
- commitOwnerClientOpt2: Option[CommitOwnerClient]): Boolean = {
- (commitOwnerClientOpt1, commitOwnerClientOpt2) match {
- case (Some(commitOwnerClient1), Some(commitOwnerClient2)) =>
- commitOwnerClient1.semanticEquals(commitOwnerClient2)
+ commitCoordinatorClientOpt1: Option[CommitCoordinatorClient],
+ commitCoordinatorClientOpt2: Option[CommitCoordinatorClient]): Boolean = {
+ (commitCoordinatorClientOpt1, commitCoordinatorClientOpt2) match {
+ case (Some(commitCoordinatorClient1), Some(commitCoordinatorClient2)) =>
+ commitCoordinatorClient1.semanticEquals(commitCoordinatorClient2)
case (None, None) =>
true
case _ =>
@@ -195,50 +204,52 @@ object CommitOwnerClient {
}
}
-/** A builder interface for [[CommitOwnerClient]] */
-trait CommitOwnerBuilder {
+/** A builder interface for [[CommitCoordinatorClient]] */
+trait CommitCoordinatorBuilder {
- /** Name of the commit-owner */
+ /** Name of the commit-coordinator */
def getName: String
- /** Returns a commit-owner client based on the given conf */
- def build(spark: SparkSession, conf: Map[String, String]): CommitOwnerClient
+ /** Returns a commit-coordinator client based on the given conf */
+ def build(spark: SparkSession, conf: Map[String, String]): CommitCoordinatorClient
}
-/** Factory to get the correct [[CommitOwnerClient]] for a table */
-object CommitOwnerProvider {
- // mapping from different commit-owner names to the corresponding [[CommitOwnerBuilder]]s.
- private val nameToBuilderMapping = mutable.Map.empty[String, CommitOwnerBuilder]
-
- /** Registers a new [[CommitOwnerBuilder]] with the [[CommitOwnerProvider]] */
- def registerBuilder(commitOwnerBuilder: CommitOwnerBuilder): Unit = synchronized {
- nameToBuilderMapping.get(commitOwnerBuilder.getName) match {
- case Some(commitOwnerBuilder: CommitOwnerBuilder) =>
- throw new IllegalArgumentException(s"commit-owner: ${commitOwnerBuilder.getName} already" +
- s" registered with builder ${commitOwnerBuilder.getClass.getName}")
+/** Factory to get the correct [[CommitCoordinatorClient]] for a table */
+object CommitCoordinatorProvider {
+ // mapping from different commit-coordinator names to the corresponding
+ // [[CommitCoordinatorBuilder]]s.
+ private val nameToBuilderMapping = mutable.Map.empty[String, CommitCoordinatorBuilder]
+
+ /** Registers a new [[CommitCoordinatorBuilder]] with the [[CommitCoordinatorProvider]] */
+ def registerBuilder(commitCoordinatorBuilder: CommitCoordinatorBuilder): Unit = synchronized {
+ nameToBuilderMapping.get(commitCoordinatorBuilder.getName) match {
+ case Some(commitCoordinatorBuilder: CommitCoordinatorBuilder) =>
+ throw new IllegalArgumentException(s"commit-coordinator: " +
+ s"${commitCoordinatorBuilder.getName} already registered with builder " +
+ s"${commitCoordinatorBuilder.getClass.getName}")
case None =>
- nameToBuilderMapping.put(commitOwnerBuilder.getName, commitOwnerBuilder)
+ nameToBuilderMapping.put(commitCoordinatorBuilder.getName, commitCoordinatorBuilder)
}
}
- /** Returns a [[CommitOwnerClient]] for the given `name`, `conf`, and `spark` */
- def getCommitOwnerClient(
+ /** Returns a [[CommitCoordinatorClient]] for the given `name`, `conf`, and `spark` */
+ def getCommitCoordinatorClient(
name: String,
conf: Map[String, String],
- spark: SparkSession): CommitOwnerClient = synchronized {
+ spark: SparkSession): CommitCoordinatorClient = synchronized {
nameToBuilderMapping.get(name).map(_.build(spark, conf)).getOrElse {
- throw new IllegalArgumentException(s"Unknown commit-owner: $name")
+ throw new IllegalArgumentException(s"Unknown commit-coordinator: $name")
}
}
// Visible only for UTs
private[delta] def clearNonDefaultBuilders(): Unit = synchronized {
- val initialCommitOwnerNames = initialCommitOwnerBuilders.map(_.getName).toSet
- nameToBuilderMapping.retain((k, _) => initialCommitOwnerNames.contains(k))
+ val initialCommitCoordinatorNames = initialCommitCoordinatorBuilders.map(_.getName).toSet
+ nameToBuilderMapping.retain((k, _) => initialCommitCoordinatorNames.contains(k))
}
- private val initialCommitOwnerBuilders = Seq[CommitOwnerBuilder](
- new DynamoDBCommitOwnerClientBuilder()
+ private val initialCommitCoordinatorBuilders = Seq[CommitCoordinatorBuilder](
+ new DynamoDBCommitCoordinatorClientBuilder()
)
- initialCommitOwnerBuilders.foreach(registerBuilder)
+ initialCommitCoordinatorBuilders.foreach(registerBuilder)
}
diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/managedcommit/ManagedCommitUtils.scala b/spark/src/main/scala/org/apache/spark/sql/delta/coordinatedcommits/CoordinatedCommitsUtils.scala
similarity index 69%
rename from spark/src/main/scala/org/apache/spark/sql/delta/managedcommit/ManagedCommitUtils.scala
rename to spark/src/main/scala/org/apache/spark/sql/delta/coordinatedcommits/CoordinatedCommitsUtils.scala
index 67d1164b90d..cab3e278bad 100644
--- a/spark/src/main/scala/org/apache/spark/sql/delta/managedcommit/ManagedCommitUtils.scala
+++ b/spark/src/main/scala/org/apache/spark/sql/delta/coordinatedcommits/CoordinatedCommitsUtils.scala
@@ -14,9 +14,9 @@
* limitations under the License.
*/
-package org.apache.spark.sql.delta.managedcommit
+package org.apache.spark.sql.delta.coordinatedcommits
-import org.apache.spark.sql.delta.{DeltaConfigs, DeltaLog, ManagedCommitTableFeature, Snapshot, SnapshotDescriptor}
+import org.apache.spark.sql.delta.{DeltaConfigs, DeltaLog, CoordinatedCommitsTableFeature, Snapshot, SnapshotDescriptor}
import org.apache.spark.sql.delta.actions.{Metadata, Protocol}
import org.apache.spark.sql.delta.metering.DeltaLogging
import org.apache.spark.sql.delta.storage.LogStore
@@ -27,7 +27,7 @@ import org.apache.hadoop.fs.{FileStatus, Path}
import org.apache.spark.sql.SparkSession
-object ManagedCommitUtils extends DeltaLogging {
+object CoordinatedCommitsUtils extends DeltaLogging {
/**
* Returns an iterator of commit files starting from startVersion.
@@ -56,11 +56,12 @@ object ManagedCommitUtils extends DeltaLogging {
def tailFromSnapshot(): Iterator[(FileStatus, Long)] = {
val currentSnapshotInDeltaLog = deltaLog.unsafeVolatileSnapshot
if (currentSnapshotInDeltaLog.version == maxVersionSeen &&
- currentSnapshotInDeltaLog.tableCommitOwnerClientOpt.isEmpty) {
+ currentSnapshotInDeltaLog.tableCommitCoordinatorClientOpt.isEmpty) {
// If the last version in listing is same as the `unsafeVolatileSnapshot` in deltaLog and
- // if that snapshot doesn't have a commit-owner => this table was not a managed-commit table
- // at the time of listing. This is because the commit which converts the file-system table
- // to a managed-commit table must be a file-system commit as per the spec.
+ // if that snapshot doesn't have a commit-coordinator => this table was not a
+ // coordinated-commits table at the time of listing. This is because the commit which
+ // converts the file-system table to a coordinated-commits table must be a file-system
+ // commit as per the spec.
return Iterator.empty
}
@@ -113,54 +114,57 @@ object ManagedCommitUtils extends DeltaLogging {
*/
def getTablePath(logPath: Path): Path = logPath.getParent
- def getCommitOwnerClient(
- spark: SparkSession, metadata: Metadata, protocol: Protocol): Option[CommitOwnerClient] = {
- metadata.managedCommitOwnerName.map { commitOwnerStr =>
- assert(protocol.isFeatureSupported(ManagedCommitTableFeature))
- CommitOwnerProvider.getCommitOwnerClient(
- commitOwnerStr, metadata.managedCommitOwnerConf, spark)
+ def getCommitCoordinatorClient(
+ spark: SparkSession,
+ metadata: Metadata,
+ protocol: Protocol): Option[CommitCoordinatorClient] = {
+ metadata.coordinatedCommitsCoordinatorName.map { commitCoordinatorStr =>
+ assert(protocol.isFeatureSupported(CoordinatedCommitsTableFeature))
+ CommitCoordinatorProvider.getCommitCoordinatorClient(
+ commitCoordinatorStr, metadata.coordinatedCommitsCoordinatorConf, spark)
}
}
- def getTableCommitOwner(
+ def getTableCommitCoordinator(
spark: SparkSession,
- snapshotDescriptor: SnapshotDescriptor): Option[TableCommitOwnerClient] = {
- getCommitOwnerClient(spark, snapshotDescriptor.metadata, snapshotDescriptor.protocol).map {
- commitOwner =>
- TableCommitOwnerClient(
- commitOwner,
+ snapshotDescriptor: SnapshotDescriptor): Option[TableCommitCoordinatorClient] = {
+ getCommitCoordinatorClient(
+ spark, snapshotDescriptor.metadata, snapshotDescriptor.protocol).map {
+ commitCoordinator =>
+ TableCommitCoordinatorClient(
+ commitCoordinator,
snapshotDescriptor.deltaLog.logPath,
- snapshotDescriptor.metadata.managedCommitTableConf,
+ snapshotDescriptor.metadata.coordinatedCommitsTableConf,
snapshotDescriptor.deltaLog.newDeltaHadoopConf(),
snapshotDescriptor.deltaLog.store
)
}
}
- def getManagedCommitConfs(metadata: Metadata): (Option[String], Map[String, String]) = {
- metadata.managedCommitOwnerName match {
- case Some(name) => (Some(name), metadata.managedCommitOwnerConf)
+ def getCoordinatedCommitsConfs(metadata: Metadata): (Option[String], Map[String, String]) = {
+ metadata.coordinatedCommitsCoordinatorName match {
+ case Some(name) => (Some(name), metadata.coordinatedCommitsCoordinatorConf)
case None => (None, Map.empty)
}
}
/**
- * The main table properties used to instantiate a TableCommitOwnerClient.
+ * The main table properties used to instantiate a TableCommitCoordinatorClient.
*/
val TABLE_PROPERTY_KEYS: Seq[String] = Seq(
- DeltaConfigs.MANAGED_COMMIT_OWNER_NAME.key,
- DeltaConfigs.MANAGED_COMMIT_OWNER_CONF.key,
- DeltaConfigs.MANAGED_COMMIT_TABLE_CONF.key)
+ DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_NAME.key,
+ DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_CONF.key,
+ DeltaConfigs.COORDINATED_COMMITS_TABLE_CONF.key)
/**
- * Returns true if any ManagedCommit-related table properties is present in the metadata.
+ * Returns true if any CoordinatedCommits-related table properties is present in the metadata.
*/
def tablePropertiesPresent(metadata: Metadata): Boolean = {
- val managedCommitProperties = Seq(
- DeltaConfigs.MANAGED_COMMIT_OWNER_NAME.key,
- DeltaConfigs.MANAGED_COMMIT_OWNER_CONF.key,
- DeltaConfigs.MANAGED_COMMIT_TABLE_CONF.key)
- managedCommitProperties.exists(metadata.configuration.contains)
+ val coordinatedCommitsProperties = Seq(
+ DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_NAME.key,
+ DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_CONF.key,
+ DeltaConfigs.COORDINATED_COMMITS_TABLE_CONF.key)
+ coordinatedCommitsProperties.exists(metadata.configuration.contains)
}
/**
@@ -174,17 +178,17 @@ object ManagedCommitUtils extends DeltaLogging {
}
/**
- * This method takes care of backfilling any unbackfilled delta files when managed commit is
- * not enabled on the table (i.e. commit-owner is not present) but there are still unbackfilled
- * delta files in the table. This can happen if an error occurred during the MC -> FS commit
- * where the commit-owner was able to register the downgrade commit but it failed to backfill
- * it. This method must be invoked before doing the next commit as otherwise there will be a
- * gap in the backfilled commit sequence.
+ * This method takes care of backfilling any unbackfilled delta files when coordinated commits is
+ * not enabled on the table (i.e. commit-coordinator is not present) but there are still
+ * unbackfilled delta files in the table. This can happen if an error occurred during the MC -> FS
+ * commit where the commit-coordinator was able to register the downgrade commit but it failed to
+ * backfill it. This method must be invoked before doing the next commit as otherwise there will
+ * be a gap in the backfilled commit sequence.
*/
- def backfillWhenManagedCommitDisabled(snapshot: Snapshot): Unit = {
- if (snapshot.tableCommitOwnerClientOpt.nonEmpty) {
+ def backfillWhenCoordinatedCommitsDisabled(snapshot: Snapshot): Unit = {
+ if (snapshot.tableCommitCoordinatorClientOpt.nonEmpty) {
// Managed commits is enabled on the table. Don't backfill as backfills are managed by
- // commit-owners.
+ // commit-coordinators.
return
}
val unbackfilledFilesAndVersions = snapshot.logSegment.deltas.collect {
@@ -216,7 +220,7 @@ object ManagedCommitUtils extends DeltaLogging {
}
recordDeltaEvent(
deltaLog,
- opType = "delta.managedCommit.backfillWhenManagedCommitSupportedAndDisabled",
+ opType = "delta.coordinatedCommits.backfillWhenCoordinatedCommitsSupportedAndDisabled",
data = Map(
"numUnbackfilledFiles" -> unbackfilledFilesAndVersions.size,
"unbackfilledFiles" -> unbackfilledFilesAndVersions.map(_._1.getPath.toString),
diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/managedcommit/InMemoryCommitOwner.scala b/spark/src/main/scala/org/apache/spark/sql/delta/coordinatedcommits/InMemoryCommitCoordinator.scala
similarity index 79%
rename from spark/src/main/scala/org/apache/spark/sql/delta/managedcommit/InMemoryCommitOwner.scala
rename to spark/src/main/scala/org/apache/spark/sql/delta/coordinatedcommits/InMemoryCommitCoordinator.scala
index 84320fe6779..4b9859cb697 100644
--- a/spark/src/main/scala/org/apache/spark/sql/delta/managedcommit/InMemoryCommitOwner.scala
+++ b/spark/src/main/scala/org/apache/spark/sql/delta/coordinatedcommits/InMemoryCommitCoordinator.scala
@@ -14,7 +14,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.delta.managedcommit
+package org.apache.spark.sql.delta.coordinatedcommits
import java.util.concurrent.ConcurrentHashMap
import java.util.concurrent.locks.ReentrantReadWriteLock
@@ -28,14 +28,14 @@ import org.apache.hadoop.fs.{FileStatus, Path}
import org.apache.spark.sql.SparkSession
-class InMemoryCommitOwner(val batchSize: Long)
- extends AbstractBatchBackfillingCommitOwnerClient {
+class InMemoryCommitCoordinator(val batchSize: Long)
+ extends AbstractBatchBackfillingCommitCoordinatorClient {
/**
* @param maxCommitVersion represents the max commit version known for the table. This is
* initialized at the time of pre-registration and updated whenever a
- * commit is successfully added to the commit-owner.
- * @param active represents whether this commit-owner has ratified any commit or not.
+ * commit is successfully added to the commit-coordinator.
+ * @param active represents whether this commit-coordinator has ratified any commit or not.
* |----------------------------|------------------|---------------------------|
* | State | maxCommitVersion | active |
* |----------------------------|------------------|---------------------------|
@@ -45,7 +45,7 @@ class InMemoryCommitOwner(val batchSize: Long)
* | and more commits are done | | |
* |----------------------------|------------------|---------------------------|
*/
- private[managedcommit] class PerTableData(
+ private[coordinatedcommits] class PerTableData(
var maxCommitVersion: Long = -1,
var active: Boolean = false
) {
@@ -56,7 +56,7 @@ class InMemoryCommitOwner(val batchSize: Long)
/**
* Returns the last ratified commit version for the table. If no commits have been done from
- * commit-owner yet, returns -1.
+ * commit-coordinator yet, returns -1.
*/
def lastRatifiedCommitVersion: Long = if (!active) -1 else maxCommitVersion
@@ -67,9 +67,9 @@ class InMemoryCommitOwner(val batchSize: Long)
val lock: ReentrantReadWriteLock = new ReentrantReadWriteLock()
}
- private[managedcommit] val perTableMap = new ConcurrentHashMap[Path, PerTableData]()
+ private[coordinatedcommits] val perTableMap = new ConcurrentHashMap[Path, PerTableData]()
- private[managedcommit] def withWriteLock[T](logPath: Path)(operation: => T): T = {
+ private[coordinatedcommits] def withWriteLock[T](logPath: Path)(operation: => T): T = {
val tableData = Option(perTableMap.get(logPath)).getOrElse {
throw new IllegalArgumentException(s"Unknown table $logPath.")
}
@@ -82,7 +82,7 @@ class InMemoryCommitOwner(val batchSize: Long)
}
}
- private[managedcommit] def withReadLock[T](logPath: Path)(operation: => T): T = {
+ private[coordinatedcommits] def withReadLock[T](logPath: Path)(operation: => T): T = {
val tableData = perTableMap.get(logPath)
if (tableData == null) {
throw new IllegalArgumentException(s"Unknown table $logPath.")
@@ -107,7 +107,7 @@ class InMemoryCommitOwner(val batchSize: Long)
logStore: LogStore,
hadoopConf: Configuration,
logPath: Path,
- managedCommitTableConf: Map[String, String],
+ coordinatedCommitsTableConf: Map[String, String],
commitVersion: Long,
commitFile: FileStatus,
commitTimestamp: Long): CommitResponse = {
@@ -133,14 +133,14 @@ class InMemoryCommitOwner(val batchSize: Long)
tableData.commitsMap(commitVersion) = commit
tableData.updateLastRatifiedCommit(commitVersion)
- logInfo(s"Added commit file ${commitFile.getPath} to commit-owner.")
+ logInfo(s"Added commit file ${commitFile.getPath} to commit-coordinator.")
CommitResponse(commit)
}
}
override def getCommits(
logPath: Path,
- managedCommitTableConf: Map[String, String],
+ coordinatedCommitsTableConf: Map[String, String],
startVersion: Option[Long],
endVersion: Option[Long]): GetCommitsResponse = {
withReadLock[GetCommitsResponse](logPath) {
@@ -180,13 +180,15 @@ class InMemoryCommitOwner(val batchSize: Long)
perTableMap.compute(logPath, (_, existingData) => {
if (existingData != null) {
if (existingData.lastRatifiedCommitVersion != -1) {
- throw new IllegalStateException(s"Table $logPath already exists in the commit-owner.")
+ throw new IllegalStateException(
+ s"Table $logPath already exists in the commit-coordinator.")
}
- // If lastRatifiedCommitVersion is -1 i.e. the commit-owner has never attempted any commit
- // for this table => this table was just pre-registered. If there is another
+ // If lastRatifiedCommitVersion is -1 i.e. the commit-coordinator has never attempted any
+ // commit for this table => this table was just pre-registered. If there is another
// pre-registration request for an older version, we reject it and table can't go backward.
if (currentVersion < existingData.maxCommitVersion) {
- throw new IllegalStateException(s"Table $logPath already registered with commit-owner")
+ throw new IllegalStateException(
+ s"Table $logPath already registered with commit-coordinator")
}
}
newPerTableData
@@ -194,21 +196,21 @@ class InMemoryCommitOwner(val batchSize: Long)
Map.empty
}
- override def semanticEquals(other: CommitOwnerClient): Boolean = this == other
+ override def semanticEquals(other: CommitCoordinatorClient): Boolean = this == other
}
/**
- * The [[InMemoryCommitOwnerBuilder]] class is responsible for creating singleton instances of
- * [[InMemoryCommitOwner]] with the specified batchSize.
+ * The [[InMemoryCommitCoordinatorBuilder]] class is responsible for creating singleton instances of
+ * [[InMemoryCommitCoordinator]] with the specified batchSize.
*/
-case class InMemoryCommitOwnerBuilder(batchSize: Long) extends CommitOwnerBuilder {
- private lazy val inMemoryStore = new InMemoryCommitOwner(batchSize)
+case class InMemoryCommitCoordinatorBuilder(batchSize: Long) extends CommitCoordinatorBuilder {
+ private lazy val inMemoryStore = new InMemoryCommitCoordinator(batchSize)
- /** Name of the commit-owner */
+ /** Name of the commit-coordinator */
def getName: String = "in-memory"
- /** Returns a commit-owner based on the given conf */
- def build(spark: SparkSession, conf: Map[String, String]): CommitOwnerClient = {
+ /** Returns a commit-coordinator based on the given conf */
+ def build(spark: SparkSession, conf: Map[String, String]): CommitCoordinatorClient = {
inMemoryStore
}
}
diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/managedcommit/TableCommitOwnerClient.scala b/spark/src/main/scala/org/apache/spark/sql/delta/coordinatedcommits/TableCommitCoordinatorClient.scala
similarity index 52%
rename from spark/src/main/scala/org/apache/spark/sql/delta/managedcommit/TableCommitOwnerClient.scala
rename to spark/src/main/scala/org/apache/spark/sql/delta/coordinatedcommits/TableCommitCoordinatorClient.scala
index e3e4ddf4330..9009c4d1575 100644
--- a/spark/src/main/scala/org/apache/spark/sql/delta/managedcommit/TableCommitOwnerClient.scala
+++ b/spark/src/main/scala/org/apache/spark/sql/delta/coordinatedcommits/TableCommitCoordinatorClient.scala
@@ -14,7 +14,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.delta.managedcommit
+package org.apache.spark.sql.delta.coordinatedcommits
import org.apache.spark.sql.delta.DeltaLog
import org.apache.spark.sql.delta.storage.LogStore
@@ -22,19 +22,19 @@ import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
/**
- * A wrapper around [[CommitOwnerClient]] that provides a more user-friendly API for committing
- * / accessing commits to a specific table. This class takes care of passing the table specific
- * configuration to the underlying [[CommitOwnerClient]] e.g. logPath / logStore
- * / managedCommitTableConf / hadoopConf.
+ * A wrapper around [[CommitCoordinatorClient]] that provides a more user-friendly API for
+ * committing/accessing commits to a specific table. This class takes care of passing the table
+ * specific configuration to the underlying [[CommitCoordinatorClient]] e.g. logPath / logStore
+ * / coordinatedCommitsTableConf / hadoopConf.
*
- * @param commitOwnerClient the underlying [[CommitOwnerClient]]
+ * @param commitCoordinatorClient the underlying [[CommitCoordinatorClient]]
* @param logPath the path to the log directory
- * @param tableConf the table specific managed-commit configuration
+ * @param tableConf the table specific coordinated-commits configuration
* @param hadoopConf hadoop configuration
* @param logStore the log store
*/
-case class TableCommitOwnerClient(
- commitOwnerClient: CommitOwnerClient,
+case class TableCommitCoordinatorClient(
+ commitCoordinatorClient: CommitCoordinatorClient,
logPath: Path,
tableConf: Map[String, String],
hadoopConf: Configuration,
@@ -44,47 +44,52 @@ case class TableCommitOwnerClient(
commitVersion: Long,
actions: Iterator[String],
updatedActions: UpdatedActions): CommitResponse = {
- commitOwnerClient.commit(
+ commitCoordinatorClient.commit(
logStore, hadoopConf, logPath, tableConf, commitVersion, actions, updatedActions)
}
def getCommits(
startVersion: Option[Long] = None,
endVersion: Option[Long] = None): GetCommitsResponse = {
- commitOwnerClient.getCommits(logPath, tableConf, startVersion, endVersion)
+ commitCoordinatorClient.getCommits(logPath, tableConf, startVersion, endVersion)
}
def backfillToVersion(
version: Long,
lastKnownBackfilledVersion: Option[Long] = None): Unit = {
- commitOwnerClient.backfillToVersion(
+ commitCoordinatorClient.backfillToVersion(
logStore, hadoopConf, logPath, tableConf, version, lastKnownBackfilledVersion)
}
/**
- * Checks whether the signature of the underlying backing [[CommitOwnerClient]] is the same as the
- * given `otherCommitOwnerClient`
+ * Checks whether the signature of the underlying backing [[CommitCoordinatorClient]] is the same
+ * as the given `otherCommitCoordinatorClient`
*/
- def semanticsEquals(otherCommitOwnerClient: CommitOwnerClient): Boolean = {
- CommitOwnerClient.semanticEquals(Some(commitOwnerClient), Some(otherCommitOwnerClient))
+ def semanticsEquals(otherCommitCoordinatorClient: CommitCoordinatorClient): Boolean = {
+ CommitCoordinatorClient.semanticEquals(
+ Some(commitCoordinatorClient), Some(otherCommitCoordinatorClient))
}
/**
- * Checks whether the signature of the underlying backing [[CommitOwnerClient]] is the same as the
- * given `otherCommitOwnerClient`
+ * Checks whether the signature of the underlying backing [[CommitCoordinatorClient]] is the same
+ * as the given `otherCommitCoordinatorClient`
*/
- def semanticsEquals(otherCommitOwnerClient: TableCommitOwnerClient): Boolean = {
- semanticsEquals(otherCommitOwnerClient.commitOwnerClient)
+ def semanticsEquals(otherCommitCoordinatorClient: TableCommitCoordinatorClient): Boolean = {
+ semanticsEquals(otherCommitCoordinatorClient.commitCoordinatorClient)
}
}
-object TableCommitOwnerClient {
+object TableCommitCoordinatorClient {
def apply(
- commitOwnerClient: CommitOwnerClient,
+ commitCoordinatorClient: CommitCoordinatorClient,
deltaLog: DeltaLog,
- managedCommitTableConf: Map[String, String]): TableCommitOwnerClient = {
+ coordinatedCommitsTableConf: Map[String, String]): TableCommitCoordinatorClient = {
val hadoopConf = deltaLog.newDeltaHadoopConf()
- new TableCommitOwnerClient(
- commitOwnerClient, deltaLog.logPath, managedCommitTableConf, hadoopConf, deltaLog.store)
+ new TableCommitCoordinatorClient(
+ commitCoordinatorClient,
+ deltaLog.logPath,
+ coordinatedCommitsTableConf,
+ hadoopConf,
+ deltaLog.store)
}
}
diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/managedcommit/abstractActions.scala b/spark/src/main/scala/org/apache/spark/sql/delta/coordinatedcommits/abstractActions.scala
similarity index 97%
rename from spark/src/main/scala/org/apache/spark/sql/delta/managedcommit/abstractActions.scala
rename to spark/src/main/scala/org/apache/spark/sql/delta/coordinatedcommits/abstractActions.scala
index 89d47558084..35a20052083 100644
--- a/spark/src/main/scala/org/apache/spark/sql/delta/managedcommit/abstractActions.scala
+++ b/spark/src/main/scala/org/apache/spark/sql/delta/coordinatedcommits/abstractActions.scala
@@ -14,7 +14,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.delta.managedcommit
+package org.apache.spark.sql.delta.coordinatedcommits
/**
* Interface for protocol actions in Delta. The protocol defines the requirements
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 4892e34d9fc..ccff1c12650 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
@@ -538,32 +538,32 @@ trait DeltaSQLConfBase {
.checkValue(_ > 0, "threadPoolSize must be positive")
.createWithDefault(20)
- val MANAGED_COMMIT_GET_COMMITS_THREAD_POOL_SIZE =
- buildStaticConf("managedCommit.getCommits.threadPoolSize")
+ val COORDINATED_COMMITS_GET_COMMITS_THREAD_POOL_SIZE =
+ buildStaticConf("coordinatedCommits.getCommits.threadPoolSize")
.internal()
- .doc("The size of the thread pool for listing files from the commit-owner.")
+ .doc("The size of the thread pool for listing files from the commit-coordinator.")
.intConf
.checkValue(_ > 0, "threadPoolSize must be positive")
.createWithDefault(5)
//////////////////////////////////////////////
- // DynamoDB Commit Owner-specific configs
+ // DynamoDB Commit Coordinator-specific configs
/////////////////////////////////////////////
- val MANAGED_COMMIT_DDB_AWS_CREDENTIALS_PROVIDER_NAME =
- buildConf("managedCommit.commitOwner.dynamodb.awsCredentialsProviderName")
+ val COORDINATED_COMMITS_DDB_AWS_CREDENTIALS_PROVIDER_NAME =
+ buildConf("coordinatedCommits.commitCoordinator.dynamodb.awsCredentialsProviderName")
.internal()
.doc("The fully qualified class name of the AWS credentials provider to use for " +
- "interacting with DynamoDB in the DynamoDB Commit Owner Client. e.g. " +
+ "interacting with DynamoDB in the DynamoDB Commit Coordinator Client. e.g. " +
"com.amazonaws.auth.DefaultAWSCredentialsProviderChain.")
.stringConf
.createWithDefault("com.amazonaws.auth.DefaultAWSCredentialsProviderChain")
- val MANAGED_COMMIT_DDB_SKIP_PATH_CHECK =
- buildConf("managedCommit.commitOwner.dynamodb.skipPathCheckEnabled")
+ val COORDINATED_COMMITS_DDB_SKIP_PATH_CHECK =
+ buildConf("coordinatedCommits.commitCoordinator.dynamodb.skipPathCheckEnabled")
.internal()
- .doc("When enabled, the DynamoDB Commit Owner will not enforce that the table path of the " +
- "current Delta table matches the stored in the corresponding DynamoDB table. This " +
+ .doc("When enabled, the DynamoDB Commit Coordinator will not enforce that the table path " +
+ "of the current Delta table matches the stored in the corresponding DynamoDB table. This " +
"should only be used when the observed table path for the same physical table varies " +
"depending on how it is accessed (e.g. abfs://path1 vs abfss://path1). Leaving this " +
"enabled can be dangerous as every physical copy of a Delta table with try to write to" +
@@ -571,26 +571,26 @@ trait DeltaSQLConfBase {
.booleanConf
.createWithDefault(false)
- val MANAGED_COMMIT_DDB_READ_CAPACITY_UNITS =
- buildConf("managedCommit.commitOwner.dynamodb.readCapacityUnits")
+ val COORDINATED_COMMITS_DDB_READ_CAPACITY_UNITS =
+ buildConf("coordinatedCommits.commitCoordinator.dynamodb.readCapacityUnits")
.internal()
.doc("Controls the provisioned read capacity units for the DynamoDB table backing the " +
- "DynamoDB Commit Owner. This configuration is only used when the DynamoDB table is first " +
- "provisioned and cannot be used configure an existing table.")
+ "DynamoDB Commit Coordinator. This configuration is only used when the DynamoDB table " +
+ "is first provisioned and cannot be used configure an existing table.")
.intConf
.createWithDefault(5)
- val MANAGED_COMMIT_DDB_WRITE_CAPACITY_UNITS =
- buildConf("managedCommit.commitOwner.dynamodb.writeCapacityUnits")
+ val COORDINATED_COMMITS_DDB_WRITE_CAPACITY_UNITS =
+ buildConf("coordinatedCommits.commitCoordinator.dynamodb.writeCapacityUnits")
.internal()
.doc("Controls the provisioned write capacity units for the DynamoDB table backing the " +
- "DynamoDB Commit Owner. This configuration is only used when the DynamoDB table is first " +
- "provisioned and cannot be used configure an existing table.")
+ "DynamoDB Commit Coordinator. This configuration is only used when the DynamoDB table " +
+ "is first provisioned and cannot be used configure an existing table.")
.intConf
.createWithDefault(5)
//////////////////////////////////////////////
- // DynamoDB Commit Owner-specific configs end
+ // DynamoDB Commit Coordinator-specific configs end
/////////////////////////////////////////////
val DELTA_UPDATE_CATALOG_LONG_FIELD_TRUNCATION_THRESHOLD =
diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/util/DeltaCommitFileProvider.scala b/spark/src/main/scala/org/apache/spark/sql/delta/util/DeltaCommitFileProvider.scala
index 815a24b67d7..69249940d55 100644
--- a/spark/src/main/scala/org/apache/spark/sql/delta/util/DeltaCommitFileProvider.scala
+++ b/spark/src/main/scala/org/apache/spark/sql/delta/util/DeltaCommitFileProvider.scala
@@ -23,11 +23,12 @@ import org.apache.hadoop.fs.Path
/**
* Provides access to resolve Delta commit files names based on the commit-version.
*
- * This class is part of the changes introduced to accommodate the adoption of managed-commits in
- * Delta Lake. Previously, certain code paths assumed the existence of delta files for a specific
- * version at a predictable path `_delta_log/$version.json`. With managed-commits, delta files may
- * alternatively be located at `_delta_log/_commits/$version.$uuid.json`. DeltaCommitFileProvider
- * attempts to locate the correct delta files from the Snapshot's LogSegment.
+ * This class is part of the changes introduced to accommodate the adoption of coordinated-commits
+ * in Delta Lake. Previously, certain code paths assumed the existence of delta files for a specific
+ * version at a predictable path `_delta_log/$version.json`. With coordinated-commits, delta files
+ * may alternatively be located at `_delta_log/_commits/$version.$uuid.json`.
+ * DeltaCommitFileProvider attempts to locate the correct delta files from the Snapshot's
+ * LogSegment.
*
* @param logPath The path to the Delta table log directory.
* @param maxVersionInclusive The maximum version of the Delta table (inclusive).
diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/util/FileNames.scala b/spark/src/main/scala/org/apache/spark/sql/delta/util/FileNames.scala
index 4cedaecd1f6..b09dfa66039 100644
--- a/spark/src/main/scala/org/apache/spark/sql/delta/util/FileNames.scala
+++ b/spark/src/main/scala/org/apache/spark/sql/delta/util/FileNames.scala
@@ -37,7 +37,7 @@ object FileNames {
/**
* Returns the delta (json format) path for a given delta file.
* WARNING: This API is unsafe and can resolve to incorrect paths if the table has
- * Managed Commits.
+ * Coordinated Commits.
* Use DeltaCommitFileProvider(snapshot).deltaFile instead to guarantee accurate paths.
*/
def unsafeDeltaFile(path: Path, version: Long): Path = new Path(path, f"$version%020d.json")
diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/CheckpointsSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/CheckpointsSuite.scala
index ae6bec7df8a..b568e720ee1 100644
--- a/spark/src/test/scala/org/apache/spark/sql/delta/CheckpointsSuite.scala
+++ b/spark/src/test/scala/org/apache/spark/sql/delta/CheckpointsSuite.scala
@@ -25,7 +25,7 @@ import scala.concurrent.duration._
import com.databricks.spark.util.{Log4jUsageLogger, MetricDefinitions, UsageRecord}
import org.apache.spark.sql.delta.actions._
import org.apache.spark.sql.delta.deletionvectors.DeletionVectorsSuite
-import org.apache.spark.sql.delta.managedcommit.ManagedCommitBaseSuite
+import org.apache.spark.sql.delta.coordinatedcommits.CoordinatedCommitsBaseSuite
import org.apache.spark.sql.delta.sources.DeltaSQLConf
import org.apache.spark.sql.delta.storage.LocalLogStore
import org.apache.spark.sql.delta.test.DeltaSQLCommandTest
@@ -49,7 +49,7 @@ class CheckpointsSuite
with SharedSparkSession
with DeltaCheckpointTestUtils
with DeltaSQLCommandTest
- with ManagedCommitBaseSuite {
+ with CoordinatedCommitsBaseSuite {
def testDifferentV2Checkpoints(testName: String)(f: => Unit): Unit = {
for (checkpointFormat <- Seq(V2Checkpoint.Format.JSON.name, V2Checkpoint.Format.PARQUET.name)) {
@@ -1063,15 +1063,15 @@ class FakeGCSFileSystemValidatingCommits extends FakeGCSFileSystemValidatingChec
override protected def shouldValidateFilePattern(f: Path): Boolean = f.getName.contains(".json")
}
-class CheckpointsWithManagedCommitBatch1Suite extends CheckpointsSuite {
- override val managedCommitBackfillBatchSize: Option[Int] = Some(1)
+class CheckpointsWithCoordinatedCommitsBatch1Suite extends CheckpointsSuite {
+ override val coordinatedCommitsBackfillBatchSize: Option[Int] = Some(1)
}
-class CheckpointsWithManagedCommitBatch2Suite extends CheckpointsSuite {
- override val managedCommitBackfillBatchSize: Option[Int] = Some(2)
+class CheckpointsWithCoordinatedCommitsBatch2Suite extends CheckpointsSuite {
+ override val coordinatedCommitsBackfillBatchSize: Option[Int] = Some(2)
}
-class CheckpointsWithManagedCommitBatch100Suite extends CheckpointsSuite {
- override val managedCommitBackfillBatchSize: Option[Int] = Some(100)
+class CheckpointsWithCoordinatedCommitsBatch100Suite extends CheckpointsSuite {
+ override val coordinatedCommitsBackfillBatchSize: Option[Int] = Some(100)
}
diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaHistoryManagerSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaHistoryManagerSuite.scala
index c4a5cc32231..9495a1f9342 100644
--- a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaHistoryManagerSuite.scala
+++ b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaHistoryManagerSuite.scala
@@ -31,7 +31,7 @@ import org.apache.spark.sql.delta.DeltaHistoryManagerSuiteShims._
import org.apache.spark.sql.delta.DeltaTestUtils.createTestAddFile
import org.apache.spark.sql.delta.DeltaTestUtils.filterUsageRecords
import org.apache.spark.sql.delta.actions.{Action, CommitInfo}
-import org.apache.spark.sql.delta.managedcommit.ManagedCommitBaseSuite
+import org.apache.spark.sql.delta.coordinatedcommits.CoordinatedCommitsBaseSuite
import org.apache.spark.sql.delta.sources.DeltaSQLConf
import org.apache.spark.sql.delta.stats.StatsUtils
import org.apache.spark.sql.delta.test.DeltaSQLCommandTest
@@ -56,7 +56,7 @@ trait DeltaTimeTravelTests extends QueryTest
with GivenWhenThen
with DeltaSQLCommandTest
with StatsUtils
- with ManagedCommitBaseSuite {
+ with CoordinatedCommitsBaseSuite {
protected implicit def durationToLong(duration: FiniteDuration): Long = {
duration.toMillis
}
@@ -370,7 +370,7 @@ trait DeltaTimeTravelTests extends QueryTest
val e2 = intercept[AnalysisException] {
sql(s"select count(*) from ${versionAsOf(tblName, 0)}").collect()
}
- if (managedCommitBackfillBatchSize.exists(_ > 2)) {
+ if (coordinatedCommitsBackfillBatchSize.exists(_ > 2)) {
assert(e2.getMessage.contains("No commits found at"))
} else {
assert(e2.getMessage.contains("No recreatable commits found at"))
@@ -667,14 +667,14 @@ class DeltaHistoryManagerSuite extends DeltaHistoryManagerBase {
}
}
-class DeltaHistoryManagerWithManagedCommitBatch1Suite extends DeltaHistoryManagerSuite {
- override def managedCommitBackfillBatchSize: Option[Int] = Some(1)
+class DeltaHistoryManagerWithCoordinatedCommitsBatch1Suite extends DeltaHistoryManagerSuite {
+ override def coordinatedCommitsBackfillBatchSize: Option[Int] = Some(1)
}
-class DeltaHistoryManagerWithManagedCommitBatch2Suite extends DeltaHistoryManagerSuite {
- override def managedCommitBackfillBatchSize: Option[Int] = Some(2)
+class DeltaHistoryManagerWithCoordinatedCommitsBatch2Suite extends DeltaHistoryManagerSuite {
+ override def coordinatedCommitsBackfillBatchSize: Option[Int] = Some(2)
}
-class DeltaHistoryManagerWithManagedCommitBatch100Suite extends DeltaHistoryManagerSuite {
- override def managedCommitBackfillBatchSize: Option[Int] = Some(100)
+class DeltaHistoryManagerWithCoordinatedCommitsBatch100Suite extends DeltaHistoryManagerSuite {
+ override def coordinatedCommitsBackfillBatchSize: Option[Int] = Some(100)
}
diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaLogMinorCompactionSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaLogMinorCompactionSuite.scala
index e66b86af84d..a2096edcc9c 100644
--- a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaLogMinorCompactionSuite.scala
+++ b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaLogMinorCompactionSuite.scala
@@ -18,7 +18,7 @@ package org.apache.spark.sql.delta
import org.apache.spark.sql.delta.DeltaOperations.ManualUpdate
import org.apache.spark.sql.delta.actions._
-import org.apache.spark.sql.delta.managedcommit.ManagedCommitBaseSuite
+import org.apache.spark.sql.delta.coordinatedcommits.CoordinatedCommitsBaseSuite
import org.apache.spark.sql.delta.sources.DeltaSQLConf
import org.apache.spark.sql.delta.test.DeltaSQLCommandTest
import org.apache.spark.sql.delta.test.DeltaSQLTestUtils
@@ -34,7 +34,7 @@ class DeltaLogMinorCompactionSuite extends QueryTest
with SharedSparkSession
with DeltaSQLCommandTest
with DeltaSQLTestUtils
- with ManagedCommitBaseSuite {
+ with CoordinatedCommitsBaseSuite {
/** Helper method to do minor compaction of [[DeltaLog]] from [startVersion, endVersion] */
private def minorCompactDeltaLog(
@@ -42,8 +42,8 @@ class DeltaLogMinorCompactionSuite extends QueryTest
startVersion: Long,
endVersion: Long): Unit = {
val deltaLog = DeltaLog.forTable(spark, tablePath)
- deltaLog.update().tableCommitOwnerClientOpt.foreach { tableCommitOwnerClient =>
- tableCommitOwnerClient.backfillToVersion(endVersion)
+ deltaLog.update().tableCommitCoordinatorClientOpt.foreach { tableCommitCoordinatorClient =>
+ tableCommitCoordinatorClient.backfillToVersion(endVersion)
}
val logReplay = new InMemoryLogReplay(
minFileRetentionTimestamp = 0,
@@ -440,15 +440,17 @@ class DeltaLogMinorCompactionSuite extends QueryTest
}
}
-class DeltaLogMinorCompactionWithManagedCommitBatch1Suite extends DeltaLogMinorCompactionSuite {
- override val managedCommitBackfillBatchSize: Option[Int] = Some(1)
+class DeltaLogMinorCompactionWithCoordinatedCommitsBatch1Suite
+ extends DeltaLogMinorCompactionSuite {
+ override val coordinatedCommitsBackfillBatchSize: Option[Int] = Some(1)
}
-class DeltaLogMinorCompactionWithManagedCommitBatch2Suite extends DeltaLogMinorCompactionSuite {
- override val managedCommitBackfillBatchSize: Option[Int] = Some(2)
+class DeltaLogMinorCompactionWithCoordinatedCommitsBatch2Suite
+ extends DeltaLogMinorCompactionSuite {
+ override val coordinatedCommitsBackfillBatchSize: Option[Int] = Some(2)
}
-class DeltaLogMinorCompactionWithManagedCommitBatch100Suite
+class DeltaLogMinorCompactionWithCoordinatedCommitsBatch100Suite
extends DeltaLogMinorCompactionSuite {
- override val managedCommitBackfillBatchSize: Option[Int] = Some(100)
+ override val coordinatedCommitsBackfillBatchSize: Option[Int] = Some(100)
}
diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaLogSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaLogSuite.scala
index 3098ad58780..4e6786aacfe 100644
--- a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaLogSuite.scala
+++ b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaLogSuite.scala
@@ -25,7 +25,7 @@ import scala.language.postfixOps
import org.apache.spark.sql.delta.DeltaOperations.Truncate
import org.apache.spark.sql.delta.DeltaTestUtils.createTestAddFile
import org.apache.spark.sql.delta.actions._
-import org.apache.spark.sql.delta.managedcommit.{CommitOwnerProvider, ManagedCommitBaseSuite, TrackingCommitOwnerClient}
+import org.apache.spark.sql.delta.coordinatedcommits.{CommitCoordinatorProvider, CoordinatedCommitsBaseSuite, TrackingCommitCoordinatorClient}
import org.apache.spark.sql.delta.sources.DeltaSQLConf
import org.apache.spark.sql.delta.test.DeltaSQLCommandTest
import org.apache.spark.sql.delta.test.DeltaSQLTestUtils
@@ -49,7 +49,7 @@ import org.apache.spark.util.Utils
class DeltaLogSuite extends QueryTest
with SharedSparkSession
with DeltaSQLCommandTest
- with ManagedCommitBaseSuite
+ with CoordinatedCommitsBaseSuite
with DeltaCheckpointTestUtils
with DeltaSQLTestUtils {
@@ -499,10 +499,10 @@ class DeltaLogSuite extends QueryTest
deltaLog.newDeltaHadoopConf())
.filter(!_.getPath.getName.startsWith("_"))
.foreach(f => fs.delete(f.getPath, true))
- if (managedCommitEnabledInTests) {
- // For Managed Commit table with a commit that is not backfilled, we can't use
+ if (coordinatedCommitsEnabledInTests) {
+ // For Coordinated Commits table with a commit that is not backfilled, we can't use
// 00000000002.json yet. Contact commit store to get uuid file path to malform json file.
- val oc = CommitOwnerProvider.getCommitOwnerClient(
+ val oc = CommitCoordinatorProvider.getCommitCoordinatorClient(
"tracking-in-memory", Map.empty[String, String], spark)
val commitResponse = oc.getCommits(deltaLog.logPath, Map.empty, Some(2))
if (!commitResponse.getCommits.isEmpty) {
@@ -510,7 +510,7 @@ class DeltaLogSuite extends QueryTest
fs.delete(path, true)
}
// Also deletes it from in-memory commit store.
- oc.asInstanceOf[TrackingCommitOwnerClient].removeCommitTestOnly(deltaLog.logPath, 2)
+ oc.asInstanceOf[TrackingCommitCoordinatorClient].removeCommitTestOnly(deltaLog.logPath, 2)
}
// Should show up to 20
@@ -598,10 +598,10 @@ class DeltaLogSuite extends QueryTest
val log = DeltaLog.forTable(spark, path)
var commitFilePath = FileNames.unsafeDeltaFile(log.logPath, 1L)
- if (managedCommitEnabledInTests) {
- // For Managed Commit table with a commit that is not backfilled, we can't use
+ if (coordinatedCommitsEnabledInTests) {
+ // For Coordinated Commits table with a commit that is not backfilled, we can't use
// 00000000001.json yet. Contact commit store to get uuid file path to malform json file.
- val oc = CommitOwnerProvider.getCommitOwnerClient(
+ val oc = CommitCoordinatorProvider.getCommitCoordinatorClient(
"tracking-in-memory", Map.empty[String, String], spark)
val commitResponse = oc.getCommits(log.logPath, Map.empty, Some(1))
if (!commitResponse.getCommits.isEmpty) {
@@ -760,7 +760,7 @@ class DeltaLogSuite extends QueryTest
assert(FileNames.getFileVersion(fileV2) === 2)
assert(FileNames.getFileVersion(fileV3) === 3)
- val backfillInterval = managedCommitBackfillBatchSize.getOrElse(0L)
+ val backfillInterval = coordinatedCommitsBackfillBatchSize.getOrElse(0L)
if (backfillInterval == 0 || backfillInterval == 1) {
assert(filesAreUnbackfilledArray === Seq(false, false, false))
} else if (backfillInterval == 2) {
@@ -773,14 +773,14 @@ class DeltaLogSuite extends QueryTest
}
-class ManagedCommitBatchBackfill1DeltaLogSuite extends DeltaLogSuite {
- override def managedCommitBackfillBatchSize: Option[Int] = Some(1)
+class CoordinatedCommitsBatchBackfill1DeltaLogSuite extends DeltaLogSuite {
+ override def coordinatedCommitsBackfillBatchSize: Option[Int] = Some(1)
}
-class ManagedCommitBatchBackfill2DeltaLogSuite extends DeltaLogSuite {
- override def managedCommitBackfillBatchSize: Option[Int] = Some(2)
+class CoordinatedCommitsBatchBackfill2DeltaLogSuite extends DeltaLogSuite {
+ override def coordinatedCommitsBackfillBatchSize: Option[Int] = Some(2)
}
-class ManagedCommitBatchBackfill100DeltaLogSuite extends DeltaLogSuite {
- override def managedCommitBackfillBatchSize: Option[Int] = Some(100)
+class CoordinatedCommitsBatchBackfill100DeltaLogSuite extends DeltaLogSuite {
+ override def coordinatedCommitsBackfillBatchSize: Option[Int] = Some(100)
}
diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaProtocolVersionSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaProtocolVersionSuite.scala
index 8229a51160a..dff9b4dff04 100644
--- a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaProtocolVersionSuite.scala
+++ b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaProtocolVersionSuite.scala
@@ -29,7 +29,7 @@ import org.apache.spark.sql.delta.actions._
import org.apache.spark.sql.delta.actions.TableFeatureProtocolUtils._
import org.apache.spark.sql.delta.catalog.DeltaTableV2
import org.apache.spark.sql.delta.commands.{AlterTableDropFeatureDeltaCommand, AlterTableSetPropertiesDeltaCommand, AlterTableUnsetPropertiesDeltaCommand}
-import org.apache.spark.sql.delta.managedcommit._
+import org.apache.spark.sql.delta.coordinatedcommits._
import org.apache.spark.sql.delta.sources.DeltaSQLConf
import org.apache.spark.sql.delta.storage.LogStore
import org.apache.spark.sql.delta.test.DeltaSQLCommandTest
@@ -3701,11 +3701,11 @@ trait DeltaProtocolVersionSuiteBase extends QueryTest
}
test("Removing VacuumProtocolCheckTableFeature should fail when dependent feature " +
- "Managed Commit is enabled") {
+ "Coordinated Commits is enabled") {
testRemoveVacuumProtocolCheckTableFeature(
enableFeatureInitially = true,
additionalTableProperties = Seq(
- (s"$FEATURE_PROP_PREFIX${ManagedCommitTableFeature.name}", "supported")),
+ (s"$FEATURE_PROP_PREFIX${CoordinatedCommitsTableFeature.name}", "supported")),
downgradeFailsWithException = Some("DELTA_FEATURE_DROP_DEPENDENT_FEATURE"),
featureExpectedAtTheEnd = true)
}
@@ -3891,94 +3891,98 @@ trait DeltaProtocolVersionSuiteBase extends QueryTest
}
}
- // ---- Managed Commit Drop Feature Tests ----
- private def setUpManagedCommitTable(dir: File, mcBuilder: CommitOwnerBuilder): Unit = {
- CommitOwnerProvider.clearNonDefaultBuilders()
- CommitOwnerProvider.registerBuilder(mcBuilder)
+ // ---- Coordinated Commits Drop Feature Tests ----
+ private def setUpCoordinatedCommitsTable(dir: File, mcBuilder: CommitCoordinatorBuilder): Unit = {
+ CommitCoordinatorProvider.clearNonDefaultBuilders()
+ CommitCoordinatorProvider.registerBuilder(mcBuilder)
val tablePath = dir.getAbsolutePath
val log = DeltaLog.forTable(spark, tablePath)
val fs = log.logPath.getFileSystem(log.newDeltaHadoopConf())
- val commitOwnerConf = Map(DeltaConfigs.MANAGED_COMMIT_OWNER_NAME.key -> mcBuilder.getName)
- val newMetadata = Metadata().copy(configuration = commitOwnerConf)
+ val commitCoordinatorConf =
+ Map(DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_NAME.key -> mcBuilder.getName)
+ val newMetadata = Metadata().copy(configuration = commitCoordinatorConf)
log.startTransaction().commitManually(newMetadata)
assert(log.unsafeVolatileSnapshot.version === 0)
- assert(log.unsafeVolatileSnapshot.metadata.managedCommitOwnerName === Some(mcBuilder.getName))
- assert(log.unsafeVolatileSnapshot.tableCommitOwnerClientOpt.nonEmpty)
- assert(log.unsafeVolatileSnapshot.metadata.managedCommitTableConf === Map.empty)
+ assert(log.unsafeVolatileSnapshot.metadata.coordinatedCommitsCoordinatorName ===
+ Some(mcBuilder.getName))
+ assert(log.unsafeVolatileSnapshot.tableCommitCoordinatorClientOpt.nonEmpty)
+ assert(log.unsafeVolatileSnapshot.metadata.coordinatedCommitsTableConf === Map.empty)
// upgrade commit always filesystem based
assert(fs.exists(FileNames.unsafeDeltaFile(log.logPath, 0)))
- // Do a couple of commits on the managed-commit table
+ // Do a couple of commits on the coordinated-commits table
(1 to 2).foreach { version =>
log.startTransaction()
.commitManually(DeltaTestUtils.createTestAddFile(s"$version"))
assert(log.unsafeVolatileSnapshot.version === version)
- assert(log.unsafeVolatileSnapshot.tableCommitOwnerClientOpt.nonEmpty)
- assert(log.unsafeVolatileSnapshot.metadata.managedCommitOwnerName.nonEmpty)
- assert(log.unsafeVolatileSnapshot.metadata.managedCommitOwnerConf === Map.empty)
- assert(log.unsafeVolatileSnapshot.metadata.managedCommitTableConf === Map.empty)
+ assert(log.unsafeVolatileSnapshot.tableCommitCoordinatorClientOpt.nonEmpty)
+ assert(log.unsafeVolatileSnapshot.metadata.coordinatedCommitsCoordinatorName.nonEmpty)
+ assert(log.unsafeVolatileSnapshot.metadata.coordinatedCommitsCoordinatorConf === Map.empty)
+ assert(log.unsafeVolatileSnapshot.metadata.coordinatedCommitsTableConf === Map.empty)
}
}
- private def validateManagedCommitDropLogs(
+ private def validateCoordinatedCommitsDropLogs(
usageLogs: Seq[UsageRecord],
expectTablePropertiesPresent: Boolean,
expectUnbackfilledCommitsPresent: Boolean,
exceptionMessageOpt: Option[String] = None): Unit = {
val dropFeatureBlob = usageLogs
- .find(_.tags.get("opType").contains("delta.managedCommitFeatureRemovalMetrics"))
- .getOrElse(fail("Expected a log for managedCommitFeatureRemovalMetrics"))
+ .find(_.tags.get("opType").contains("delta.coordinatedCommitsFeatureRemovalMetrics"))
+ .getOrElse(fail("Expected a log for coordinatedCommitsFeatureRemovalMetrics"))
val blob = JsonUtils.fromJson[Map[String, String]](dropFeatureBlob.blob)
assert(blob.contains("downgradeTimeMs"))
val expectTraceRemovalNeeded = expectTablePropertiesPresent || expectUnbackfilledCommitsPresent
assert(blob.get("traceRemovalNeeded").contains(expectTraceRemovalNeeded.toString))
Seq(
- DeltaConfigs.MANAGED_COMMIT_OWNER_NAME.key,
- DeltaConfigs.MANAGED_COMMIT_TABLE_CONF.key).foreach { prop =>
+ DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_NAME.key,
+ DeltaConfigs.COORDINATED_COMMITS_TABLE_CONF.key).foreach { prop =>
assert(blob.get(prop).contains(expectTablePropertiesPresent.toString))
}
- // MANAGED_COMMIT_OWNER_CONF is not used by "in-memory" commit owner.
+ // COORDINATED_COMMITS_COORDINATOR_CONF is not used by "in-memory" commit coordinator.
assert(blob
.get("postDisablementUnbackfilledCommitsPresent")
.contains(expectUnbackfilledCommitsPresent.toString))
assert(
- blob.get(DeltaConfigs.MANAGED_COMMIT_OWNER_CONF.key).contains("false"))
+ blob.get(DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_CONF.key).contains("false"))
assert(blob.get("traceRemovalSuccess").contains(exceptionMessageOpt.isEmpty.toString))
exceptionMessageOpt.foreach { exceptionMessage =>
assert(blob.get("traceRemovalException").contains(exceptionMessage))
}
}
- test("basic managed commit feature drop") {
+ test("basic coordinated commits feature drop") {
withTempDir { dir =>
- val mcBuilder = TrackingInMemoryCommitOwnerBuilder(batchSize = 1000)
- setUpManagedCommitTable(dir, mcBuilder)
+ val mcBuilder = TrackingInMemoryCommitCoordinatorBuilder(batchSize = 1000)
+ setUpCoordinatedCommitsTable(dir, mcBuilder)
val log = DeltaLog.forTable(spark, dir)
val usageLogs = Log4jUsageLogger.track {
AlterTableDropFeatureDeltaCommand(
DeltaTableV2(spark, log.dataPath),
- ManagedCommitTableFeature.name)
+ CoordinatedCommitsTableFeature.name)
.run(spark)
}
val snapshot = log.update()
assert(
- !ManagedCommitUtils.TABLE_PROPERTY_KEYS.exists(snapshot.metadata.configuration.contains(_)))
- assert(!snapshot.protocol.writerFeatures.exists(_.contains(ManagedCommitTableFeature.name)))
- validateManagedCommitDropLogs(
+ !CoordinatedCommitsUtils.TABLE_PROPERTY_KEYS.exists(
+ snapshot.metadata.configuration.contains(_)))
+ assert(
+ !snapshot.protocol.writerFeatures.exists(_.contains(CoordinatedCommitsTableFeature.name)))
+ validateCoordinatedCommitsDropLogs(
usageLogs, expectTablePropertiesPresent = true, expectUnbackfilledCommitsPresent = false)
}
}
- test("backfill failure during managed commit feature drop") {
+ test("backfill failure during coordinated commits feature drop") {
withTempDir { dir =>
var shouldFailBackfill = true
val alternatingFailureBackfillClient =
- new TrackingCommitOwnerClient(new InMemoryCommitOwner(1000) {
+ new TrackingCommitCoordinatorClient(new InMemoryCommitCoordinator(1000) {
override def backfillToVersion(
logStore: LogStore,
hadoopConf: Configuration,
logPath: Path,
- managedCommitTableConf: Map[String, String],
+ coordinatedCommitsTableConf: Map[String, String],
startVersion: Long,
endVersionOpt: Option[Long]): Unit = {
// Backfill fails on every other attempt.
@@ -3987,25 +3991,30 @@ trait DeltaProtocolVersionSuiteBase extends QueryTest
throw new IllegalStateException("backfill failed")
} else {
super.backfillToVersion(
- logStore, hadoopConf, logPath, managedCommitTableConf, startVersion, endVersionOpt)
+ logStore,
+ hadoopConf,
+ logPath,
+ coordinatedCommitsTableConf,
+ startVersion,
+ endVersionOpt)
}
}
})
val mcBuilder =
- TrackingInMemoryCommitOwnerBuilder(100, Some(alternatingFailureBackfillClient))
- setUpManagedCommitTable(dir, mcBuilder)
+ TrackingInMemoryCommitCoordinatorBuilder(100, Some(alternatingFailureBackfillClient))
+ setUpCoordinatedCommitsTable(dir, mcBuilder)
val log = DeltaLog.forTable(spark, dir)
val usageLogs = Log4jUsageLogger.track {
val e = intercept[IllegalStateException] {
AlterTableDropFeatureDeltaCommand(
DeltaTableV2(spark, log.dataPath),
- ManagedCommitTableFeature.name)
+ CoordinatedCommitsTableFeature.name)
.run(spark)
}
assert(e.getMessage.contains("backfill failed"))
}
- validateManagedCommitDropLogs(
+ validateCoordinatedCommitsDropLogs(
usageLogs,
expectTablePropertiesPresent = true,
expectUnbackfilledCommitsPresent = false,
@@ -4014,20 +4023,20 @@ trait DeltaProtocolVersionSuiteBase extends QueryTest
val fs = log.logPath.getFileSystem(log.newDeltaHadoopConf())
fs.exists(FileNames.unsafeDeltaFile(log.logPath, v))
}
- // Backfill of the commit which disables managed commits failed.
+ // Backfill of the commit which disables coordinated commits failed.
assert(!backfilledCommitExists(3))
- // The commit owner still tracks the commit that disables it.
- val commitsFromCommitOwner =
- log.snapshot.tableCommitOwnerClientOpt.get.getCommits(Some(3))
- assert(commitsFromCommitOwner.getCommits.exists(_.getVersion == 3))
+ // The commit coordinator still tracks the commit that disables it.
+ val commitsFromCommitCoordinator =
+ log.snapshot.tableCommitCoordinatorClientOpt.get.getCommits(Some(3))
+ assert(commitsFromCommitCoordinator.getCommits.exists(_.getVersion == 3))
// The next drop attempt will also trigger an explicit backfill.
val usageLogs2 = Log4jUsageLogger.track {
AlterTableDropFeatureDeltaCommand(
DeltaTableV2(spark, log.dataPath),
- ManagedCommitTableFeature.name)
+ CoordinatedCommitsTableFeature.name)
.run(spark)
}
- validateManagedCommitDropLogs(
+ validateCoordinatedCommitsDropLogs(
usageLogs2, expectTablePropertiesPresent = false, expectUnbackfilledCommitsPresent = true)
val snapshot = log.update()
assert(snapshot.version === 4)
@@ -4035,11 +4044,13 @@ trait DeltaProtocolVersionSuiteBase extends QueryTest
// The protocol downgrade commit is performed through logstore directly.
assert(backfilledCommitExists(4))
assert(
- !ManagedCommitUtils.TABLE_PROPERTY_KEYS.exists(snapshot.metadata.configuration.contains(_)))
- assert(!snapshot.protocol.writerFeatures.exists(_.contains(ManagedCommitTableFeature.name)))
+ !CoordinatedCommitsUtils.TABLE_PROPERTY_KEYS.exists(
+ snapshot.metadata.configuration.contains(_)))
+ assert(
+ !snapshot.protocol.writerFeatures.exists(_.contains(CoordinatedCommitsTableFeature.name)))
}
}
- // ---- End Managed Commit Drop Feature Tests ----
+ // ---- End Coordinated Commits Drop Feature Tests ----
// Create a table for testing that has an unsupported feature.
private def withTestTableWithUnsupportedWriterFeature(
diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaRetentionSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaRetentionSuite.scala
index 8da623a0147..41978cc88e9 100644
--- a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaRetentionSuite.scala
+++ b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaRetentionSuite.scala
@@ -507,8 +507,8 @@ class DeltaRetentionSuite extends QueryTest
}
}
-class DeltaRetentionWithManagedCommitBatch1Suite extends DeltaRetentionSuite {
- override val managedCommitBackfillBatchSize: Option[Int] = Some(1)
+class DeltaRetentionWithCoordinatedCommitsBatch1Suite extends DeltaRetentionSuite {
+ override val coordinatedCommitsBackfillBatchSize: Option[Int] = Some(1)
}
/**
@@ -517,10 +517,10 @@ class DeltaRetentionWithManagedCommitBatch1Suite extends DeltaRetentionSuite {
* files. However, in this suite, delta files might be backfilled asynchronously, which means
* setting the modification time will not work as expected.
*/
-class DeltaRetentionWithManagedCommitBatch2Suite extends QueryTest
+class DeltaRetentionWithCoordinatedCommitsBatch2Suite extends QueryTest
with DeltaSQLCommandTest
with DeltaRetentionSuiteBase {
- override def managedCommitBackfillBatchSize: Option[Int] = Some(2)
+ override def coordinatedCommitsBackfillBatchSize: Option[Int] = Some(2)
override def getLogFiles(dir: File): Seq[File] =
getDeltaFiles(dir) ++ getUnbackfilledDeltaFiles(dir) ++ getCheckpointFiles(dir)
@@ -528,9 +528,9 @@ class DeltaRetentionWithManagedCommitBatch2Suite extends QueryTest
/**
* This test verifies that unbackfilled versions, i.e., versions for which backfilled deltas do
* not exist yet, are never considered for deletion, even if they fall outside the retention
- * window. The primary reason for not deleting these versions is that the CommitOwner might be
- * actively tracking those files, and currently, MetadataCleanup does not communicate with the
- * CommitOwner.
+ * window. The primary reason for not deleting these versions is that the CommitCoordinator might
+ * be actively tracking those files, and currently, MetadataCleanup does not communicate with the
+ * CommitCoordinator.
*
* Although the fact that they are unbackfilled is somewhat redundant since these versions are
* currently already protected due to two additional reasons:
diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaRetentionSuiteBase.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaRetentionSuiteBase.scala
index afd289082da..262755ca40b 100644
--- a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaRetentionSuiteBase.scala
+++ b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaRetentionSuiteBase.scala
@@ -24,7 +24,7 @@ import scala.collection.mutable
import org.apache.spark.sql.delta.DeltaOperations.Truncate
import org.apache.spark.sql.delta.DeltaTestUtils.createTestAddFile
import org.apache.spark.sql.delta.actions.{CheckpointMetadata, Metadata, SidecarFile}
-import org.apache.spark.sql.delta.managedcommit.ManagedCommitBaseSuite
+import org.apache.spark.sql.delta.coordinatedcommits.CoordinatedCommitsBaseSuite
import org.apache.spark.sql.delta.sources.DeltaSQLConf
import org.apache.spark.sql.delta.test.DeltaTestImplicits._
import org.apache.spark.sql.delta.util.FileNames
@@ -41,7 +41,7 @@ import org.apache.spark.util.ManualClock
trait DeltaRetentionSuiteBase extends QueryTest
with SharedSparkSession
- with ManagedCommitBaseSuite {
+ with CoordinatedCommitsBaseSuite {
protected val testOp = Truncate()
protected override def sparkConf: SparkConf = super.sparkConf
@@ -92,7 +92,7 @@ trait DeltaRetentionSuiteBase extends QueryTest
protected def getDeltaVersions(dir: File): Set[Long] = {
val backfilledDeltaVersions = getFileVersions(getDeltaFiles(dir))
val unbackfilledDeltaVersions = getUnbackfilledDeltaVersions(dir)
- if (managedCommitEnabledInTests) {
+ if (coordinatedCommitsEnabledInTests) {
// The unbackfilled commit files (except commit 0) should be a superset of the backfilled
// commit files since they're always deleted together in this suite.
assert(
diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaTableFeatureSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaTableFeatureSuite.scala
index e922e3acb8a..574c632ba9c 100644
--- a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaTableFeatureSuite.scala
+++ b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaTableFeatureSuite.scala
@@ -22,7 +22,7 @@ import scala.collection.mutable
import org.apache.spark.sql.delta.actions._
import org.apache.spark.sql.delta.actions.TableFeatureProtocolUtils._
-import org.apache.spark.sql.delta.managedcommit.{CommitOwnerProvider, InMemoryCommitOwnerBuilder}
+import org.apache.spark.sql.delta.coordinatedcommits.{CommitCoordinatorProvider, InMemoryCommitCoordinatorBuilder}
import org.apache.spark.sql.delta.sources.DeltaSQLConf
import org.apache.spark.sql.delta.test.DeltaSQLCommandTest
import org.apache.spark.sql.delta.test.DeltaTestImplicits._
@@ -472,7 +472,7 @@ class DeltaTableFeatureSuite
}
}
- test("drop table feature works with managed commits") {
+ test("drop table feature works with coordinated commits") {
val table = "tbl"
withTable(table) {
spark.range(0).write.format("delta").saveAsTable(table)
@@ -480,9 +480,10 @@ class DeltaTableFeatureSuite
val featureName = TestRemovableReaderWriterFeature.name
assert(!log.update().protocol.readerAndWriterFeatureNames.contains(featureName))
- // Add managed commit table feature to the table
- CommitOwnerProvider.registerBuilder(InMemoryCommitOwnerBuilder(batchSize = 100))
- val tblProperties1 = Seq(s"'${DeltaConfigs.MANAGED_COMMIT_OWNER_NAME.key}' = 'in-memory'")
+ // Add coordinated commits table feature to the table
+ CommitCoordinatorProvider.registerBuilder(InMemoryCommitCoordinatorBuilder(batchSize = 100))
+ val tblProperties1 =
+ Seq(s"'${DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_NAME.key}' = 'in-memory'")
sql(buildTablePropertyModifyingCommand(
"ALTER", targetTableName = table, sourceTableName = table, tblProperties1))
diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaTimeTravelSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaTimeTravelSuite.scala
index aec01fb1a9d..3ab857369b5 100644
--- a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaTimeTravelSuite.scala
+++ b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaTimeTravelSuite.scala
@@ -28,7 +28,7 @@ import scala.language.implicitConversions
import org.apache.spark.sql.delta.DeltaHistoryManager.BufferingLogDeletionIterator
import org.apache.spark.sql.delta.DeltaTestUtils.createTestAddFile
import org.apache.spark.sql.delta.actions.{Action, CommitInfo, SingleAction}
-import org.apache.spark.sql.delta.managedcommit.ManagedCommitBaseSuite
+import org.apache.spark.sql.delta.coordinatedcommits.CoordinatedCommitsBaseSuite
import org.apache.spark.sql.delta.test.DeltaSQLCommandTest
import org.apache.spark.sql.delta.test.DeltaSQLTestUtils
import org.apache.spark.sql.delta.test.DeltaTestImplicits._
@@ -44,7 +44,7 @@ class DeltaTimeTravelSuite extends QueryTest
with SharedSparkSession
with DeltaSQLTestUtils
with DeltaSQLCommandTest
- with ManagedCommitBaseSuite {
+ with CoordinatedCommitsBaseSuite {
import testImplicits._
@@ -795,6 +795,6 @@ class DeltaTimeTravelSuite extends QueryTest
}
}
-class DeltaTimeTravelWithManagedCommitBatch1Suite extends DeltaTimeTravelSuite {
- override def managedCommitBackfillBatchSize: Option[Int] = Some(1)
+class DeltaTimeTravelWithCoordinatedCommitsBatch1Suite extends DeltaTimeTravelSuite {
+ override def coordinatedCommitsBackfillBatchSize: Option[Int] = Some(1)
}
diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DescribeDeltaDetailSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DescribeDeltaDetailSuite.scala
index fccaea5d69b..b82d100cdb7 100644
--- a/spark/src/test/scala/org/apache/spark/sql/delta/DescribeDeltaDetailSuite.scala
+++ b/spark/src/test/scala/org/apache/spark/sql/delta/DescribeDeltaDetailSuite.scala
@@ -21,7 +21,7 @@ import java.io.FileNotFoundException
// scalastyle:off import.ordering.noEmptyLine
import org.apache.spark.sql.delta.actions.TableFeatureProtocolUtils.{TABLE_FEATURES_MIN_READER_VERSION, TABLE_FEATURES_MIN_WRITER_VERSION}
-import org.apache.spark.sql.delta.managedcommit.ManagedCommitTestUtils
+import org.apache.spark.sql.delta.coordinatedcommits.CoordinatedCommitsTestUtils
import org.apache.spark.sql.delta.sources.DeltaSQLConf
import org.apache.spark.sql.delta.test.DeltaSQLCommandTest
@@ -34,7 +34,7 @@ import org.apache.spark.util.Utils
trait DescribeDeltaDetailSuiteBase extends QueryTest
with SharedSparkSession
- with ManagedCommitTestUtils
+ with CoordinatedCommitsTestUtils
with DeltaTestUtilsForTempViews {
import testImplicits._
@@ -232,14 +232,15 @@ trait DescribeDeltaDetailSuiteBase extends QueryTest
metadata.configuration ++ Map("foo" -> "bar")
)
txn.commit(newMetadata :: Nil, DeltaOperations.ManualUpdate)
- val managedCommitProperties = batchSizeOpt.map(_ =>
- Map(DeltaConfigs.MANAGED_COMMIT_OWNER_NAME.key -> "tracking-in-memory",
- DeltaConfigs.MANAGED_COMMIT_OWNER_CONF.key -> "{\"randomConf\":\"randomConfValue\"}",
- DeltaConfigs.MANAGED_COMMIT_TABLE_CONF.key -> "{}",
+ val coordinatedCommitsProperties = batchSizeOpt.map(_ =>
+ Map(DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_NAME.key -> "tracking-in-memory",
+ DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_CONF.key ->
+ "{\"randomConf\":\"randomConfValue\"}",
+ DeltaConfigs.COORDINATED_COMMITS_TABLE_CONF.key -> "{}",
DeltaConfigs.IN_COMMIT_TIMESTAMPS_ENABLED.key -> "true"))
.getOrElse(Map.empty)
checkResult(sql(s"DESCRIBE DETAIL $tableName"),
- Seq(Map("foo" -> "bar") ++ managedCommitProperties),
+ Seq(Map("foo" -> "bar") ++ coordinatedCommitsProperties),
Seq("properties")
)
}
diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/FindLastCompleteCheckpointSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/FindLastCompleteCheckpointSuite.scala
index 61a44dceff2..d549f999697 100644
--- a/spark/src/test/scala/org/apache/spark/sql/delta/FindLastCompleteCheckpointSuite.scala
+++ b/spark/src/test/scala/org/apache/spark/sql/delta/FindLastCompleteCheckpointSuite.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.delta
import com.databricks.spark.util.Log4jUsageLogger
import org.apache.spark.sql.delta.CheckpointInstance.Format
import org.apache.spark.sql.delta.DeltaTestUtils.BOOLEAN_DOMAIN
-import org.apache.spark.sql.delta.managedcommit.ManagedCommitBaseSuite
+import org.apache.spark.sql.delta.coordinatedcommits.CoordinatedCommitsBaseSuite
import org.apache.spark.sql.delta.storage.LocalLogStore
import org.apache.spark.sql.delta.test.DeltaSQLCommandTest
import org.apache.spark.sql.delta.util.{FileNames, JsonUtils}
@@ -34,7 +34,7 @@ class FindLastCompleteCheckpointSuite
extends QueryTest
with SharedSparkSession
with DeltaSQLCommandTest
- with ManagedCommitBaseSuite {
+ with CoordinatedCommitsBaseSuite {
protected override def sparkConf: SparkConf = {
super.sparkConf
diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/InCommitTimestampSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/InCommitTimestampSuite.scala
index 7c293eeda85..b70de3e04da 100644
--- a/spark/src/test/scala/org/apache/spark/sql/delta/InCommitTimestampSuite.scala
+++ b/spark/src/test/scala/org/apache/spark/sql/delta/InCommitTimestampSuite.scala
@@ -25,7 +25,7 @@ import com.databricks.spark.util.{Log4jUsageLogger, UsageRecord}
import org.apache.spark.sql.delta.DeltaOperations.ManualUpdate
import org.apache.spark.sql.delta.DeltaTestUtils.createTestAddFile
import org.apache.spark.sql.delta.actions.{Action, CommitInfo}
-import org.apache.spark.sql.delta.managedcommit.{CommitOwnerProvider, ManagedCommitBaseSuite, ManagedCommitTestUtils, TrackingInMemoryCommitOwnerBuilder}
+import org.apache.spark.sql.delta.coordinatedcommits.{CommitCoordinatorProvider, CoordinatedCommitsBaseSuite, CoordinatedCommitsTestUtils, TrackingInMemoryCommitCoordinatorBuilder}
import org.apache.spark.sql.delta.sources.DeltaSQLConf
import org.apache.spark.sql.delta.test.DeltaSQLCommandTest
import org.apache.spark.sql.delta.util.{DeltaCommitFileProvider, FileNames, JsonUtils}
@@ -40,7 +40,7 @@ class InCommitTimestampSuite
with SharedSparkSession
with DeltaSQLCommandTest
with DeltaTestUtilsBase
- with ManagedCommitTestUtils {
+ with CoordinatedCommitsTestUtils {
override def beforeAll(): Unit = {
super.beforeAll()
@@ -67,8 +67,8 @@ class InCommitTimestampSuite
}
}
- // Managed Commits will also automatically enable ICT.
- testWithDefaultCommitOwnerUnset(
+ // Coordinated Commits will also automatically enable ICT.
+ testWithDefaultCommitCoordinatorUnset(
"Create a non-inCommitTimestamp table and then enable timestamp") {
withSQLConf(
DeltaConfigs.IN_COMMIT_TIMESTAMPS_ENABLED.defaultTablePropertyKey -> false.toString
@@ -298,8 +298,9 @@ class InCommitTimestampSuite
}
}
- // Managed Commits will also automatically enable ICT.
- testWithDefaultCommitOwnerUnset("Enablement tracking works when ICT is enabled post commit 0") {
+ // Coordinated Commits will also automatically enable ICT.
+ testWithDefaultCommitCoordinatorUnset("Enablement tracking works when ICT is enabled post " +
+ "commit 0") {
withSQLConf(
DeltaConfigs.IN_COMMIT_TIMESTAMPS_ENABLED.defaultTablePropertyKey -> false.toString
) {
@@ -325,8 +326,8 @@ class InCommitTimestampSuite
}
}
- // Managed Commits will also automatically enable ICT.
- testWithDefaultCommitOwnerUnset("Conflict resolution of enablement version") {
+ // Coordinated Commits will also automatically enable ICT.
+ testWithDefaultCommitCoordinatorUnset("Conflict resolution of enablement version") {
withSQLConf(
DeltaConfigs.IN_COMMIT_TIMESTAMPS_ENABLED.defaultTablePropertyKey -> false.toString
) {
@@ -361,8 +362,8 @@ class InCommitTimestampSuite
}
}
- // Managed Commits will also automatically enable ICT.
- testWithDefaultCommitOwnerUnset(
+ // Coordinated Commits will also automatically enable ICT.
+ testWithDefaultCommitCoordinatorUnset(
"commitLarge should correctly set the enablement tracking properties") {
withTempDir { tempDir =>
spark.range(2).write.format("delta").save(tempDir.getAbsolutePath)
@@ -593,8 +594,8 @@ class InCommitTimestampSuite
}
}
- // Managed Commits will also automatically enable ICT.
- testWithDefaultCommitOwnerUnset("DeltaHistoryManager.getActiveCommitAtTime: " +
+ // Coordinated Commits will also automatically enable ICT.
+ testWithDefaultCommitCoordinatorUnset("DeltaHistoryManager.getActiveCommitAtTime: " +
"works correctly when the history has both ICT and non-ICT commits") {
withSQLConf(
DeltaConfigs.IN_COMMIT_TIMESTAMPS_ENABLED.defaultTablePropertyKey -> false.toString) {
@@ -678,8 +679,8 @@ class InCommitTimestampSuite
}
}
- // Managed Commits will also automatically enable ICT.
- testWithDefaultCommitOwnerUnset("DeltaHistoryManager.getHistory --- " +
+ // Coordinated Commits will also automatically enable ICT.
+ testWithDefaultCommitCoordinatorUnset("DeltaHistoryManager.getHistory --- " +
"works correctly when the history has both ICT and non-ICT commits") {
withSQLConf(
DeltaConfigs.IN_COMMIT_TIMESTAMPS_ENABLED.defaultTablePropertyKey -> false.toString) {
@@ -987,25 +988,25 @@ class InCommitTimestampSuite
}
}
-class InCommitTimestampWithManagedCommitSuite
+class InCommitTimestampWithCoordinatedCommitsSuite
extends InCommitTimestampSuite
- with ManagedCommitBaseSuite {
+ with CoordinatedCommitsBaseSuite {
override def beforeAll(): Unit = {
super.beforeAll()
spark.conf.set(DeltaConfigs.IN_COMMIT_TIMESTAMPS_ENABLED.defaultTablePropertyKey, "true")
}
- override def managedCommitBackfillBatchSize: Option[Int] = Some(5)
+ override def coordinatedCommitsBackfillBatchSize: Option[Int] = Some(5)
- test("getActiveCommitAtTime works correctly within managed commit range") {
- CommitOwnerProvider.clearNonDefaultBuilders()
- val builder = TrackingInMemoryCommitOwnerBuilder(batchSize = 10)
- CommitOwnerProvider.registerBuilder(builder)
+ test("getActiveCommitAtTime works correctly within coordinated commits range") {
+ CommitCoordinatorProvider.clearNonDefaultBuilders()
+ val builder = TrackingInMemoryCommitCoordinatorBuilder(batchSize = 10)
+ CommitCoordinatorProvider.registerBuilder(builder)
withTempDir { tempDir =>
spark.range(10).write.format("delta").save(tempDir.getAbsolutePath)
val deltaLog = DeltaLog.forTable(spark, new Path(tempDir.getCanonicalPath))
val commit0 = DeltaHistoryManager.Commit(0, deltaLog.snapshot.timestamp)
- val tableCommitOwnerClient = deltaLog.snapshot.tableCommitOwnerClientOpt.get
+ val tableCommitCoordinatorClient = deltaLog.snapshot.tableCommitCoordinatorClientOpt.get
val numberAdditionalCommits = 4
// Create 4 unbackfilled commits.
for (i <- 1 to numberAdditionalCommits) {
@@ -1013,7 +1014,7 @@ class InCommitTimestampWithManagedCommitSuite
}
val commitFileProvider = DeltaCommitFileProvider(deltaLog.update())
val unbackfilledCommits =
- tableCommitOwnerClient
+ tableCommitCoordinatorClient
.getCommits(Some(1))
.getCommits
.map { commit => DeltaHistoryManager.Commit(commit.getVersion, commit.getCommitTimestamp)}
diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/OptimisticTransactionSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/OptimisticTransactionSuite.scala
index 9e668bb7cf2..646b8e87071 100644
--- a/spark/src/test/scala/org/apache/spark/sql/delta/OptimisticTransactionSuite.scala
+++ b/spark/src/test/scala/org/apache/spark/sql/delta/OptimisticTransactionSuite.scala
@@ -23,7 +23,7 @@ import com.databricks.spark.util.Log4jUsageLogger
import org.apache.spark.sql.delta.DeltaOperations.ManualUpdate
import org.apache.spark.sql.delta.DeltaTestUtils.createTestAddFile
import org.apache.spark.sql.delta.actions.{Action, AddFile, CommitInfo, Metadata, Protocol, RemoveFile, SetTransaction}
-import org.apache.spark.sql.delta.managedcommit._
+import org.apache.spark.sql.delta.coordinatedcommits._
import org.apache.spark.sql.delta.sources.DeltaSQLConf
import org.apache.spark.sql.delta.storage.LogStore
import org.apache.spark.sql.delta.test.DeltaTestImplicits._
@@ -269,7 +269,7 @@ class OptimisticTransactionSuite
override def beforeEach(): Unit = {
super.beforeEach()
- CommitOwnerProvider.clearNonDefaultBuilders()
+ CommitCoordinatorProvider.clearNonDefaultBuilders()
}
test("initial commit without metadata should fail") {
@@ -296,7 +296,7 @@ class OptimisticTransactionSuite
}
}
- test("enabling Managed Commits on an existing table should create commit dir") {
+ test("enabling Coordinated Commits on an existing table should create commit dir") {
withTempDir { tempDir =>
val log = DeltaLog.forTable(spark, new Path(tempDir.getAbsolutePath))
val metadata = Metadata()
@@ -306,18 +306,18 @@ class OptimisticTransactionSuite
// Delete commit directory.
fs.delete(commitDir)
assert(!fs.exists(commitDir))
- // With no Managed Commits conf, commit directory should not be created.
+ // With no Coordinated Commits conf, commit directory should not be created.
log.startTransaction().commit(Seq(metadata), ManualUpdate)
assert(!fs.exists(commitDir))
- // Enabling Managed Commits on an existing table should create the commit dir.
- CommitOwnerProvider.registerBuilder(InMemoryCommitOwnerBuilder(3))
+ // Enabling Coordinated Commits on an existing table should create the commit dir.
+ CommitCoordinatorProvider.registerBuilder(InMemoryCommitCoordinatorBuilder(3))
val newMetadata = metadata.copy(configuration =
(metadata.configuration ++
- Map(DeltaConfigs.MANAGED_COMMIT_OWNER_NAME.key -> "in-memory")).toMap)
+ Map(DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_NAME.key -> "in-memory")).toMap)
log.startTransaction().commit(Seq(newMetadata), ManualUpdate)
assert(fs.exists(commitDir))
log.update().ensureCommitFilesBackfilled()
- // With no new Managed Commits conf, commit directory should not be created and so the
+ // With no new Coordinated Commits conf, commit directory should not be created and so the
// transaction should fail because of corrupted dir.
fs.delete(commitDir)
assert(!fs.exists(commitDir))
@@ -484,19 +484,19 @@ class OptimisticTransactionSuite
}
test("Limited retries for non-conflict retryable CommitFailedExceptions") {
- val commitOwnerName = "retryable-non-conflict-commit-owner"
+ val commitCoordinatorName = "retryable-non-conflict-commit-coordinator"
var commitAttempts = 0
val numRetries = "100"
val numNonConflictRetries = "10"
val initialNonConflictErrors = 5
val initialConflictErrors = 5
- object RetryableNonConflictCommitOwnerBuilder$ extends CommitOwnerBuilder {
+ object RetryableNonConflictCommitCoordinatorBuilder$ extends CommitCoordinatorBuilder {
- override def getName: String = commitOwnerName
+ override def getName: String = commitCoordinatorName
- val commitOwnerClient: InMemoryCommitOwner = {
- new InMemoryCommitOwner(batchSize = 1000L) {
+ val commitCoordinatorClient: InMemoryCommitCoordinator = {
+ new InMemoryCommitCoordinator(batchSize = 1000L) {
override def commit(
logStore: LogStore,
hadoopConf: Configuration,
@@ -520,17 +520,19 @@ class OptimisticTransactionSuite
}
}
override def build(
- spark: SparkSession, conf: Map[String, String]): CommitOwnerClient = commitOwnerClient
+ spark: SparkSession,
+ conf: Map[String, String]): CommitCoordinatorClient = commitCoordinatorClient
}
- CommitOwnerProvider.registerBuilder(RetryableNonConflictCommitOwnerBuilder$)
+ CommitCoordinatorProvider.registerBuilder(RetryableNonConflictCommitCoordinatorBuilder$)
withSQLConf(
DeltaSQLConf.DELTA_MAX_RETRY_COMMIT_ATTEMPTS.key -> numRetries,
DeltaSQLConf.DELTA_MAX_NON_CONFLICT_RETRY_COMMIT_ATTEMPTS.key -> numNonConflictRetries) {
withTempDir { tempDir =>
val log = DeltaLog.forTable(spark, new Path(tempDir.getCanonicalPath))
- val conf = Map(DeltaConfigs.MANAGED_COMMIT_OWNER_NAME.key -> commitOwnerName)
+ val conf =
+ Map(DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_NAME.key -> commitCoordinatorName)
log.startTransaction().commit(Seq(Metadata(configuration = conf)), ManualUpdate)
val testTxn = log.startTransaction()
intercept[CommitFailedException] { testTxn.commit(Seq.empty, ManualUpdate) }
@@ -541,16 +543,16 @@ class OptimisticTransactionSuite
}
}
- test("No retries for FileAlreadyExistsException with commit-owner") {
- val commitOwnerName = "file-already-exists-commit-owner"
+ test("No retries for FileAlreadyExistsException with commit-coordinator") {
+ val commitCoordinatorName = "file-already-exists-commit-coordinator"
var commitAttempts = 0
- object FileAlreadyExistsCommitOwnerBuilder extends CommitOwnerBuilder {
+ object FileAlreadyExistsCommitCoordinatorBuilder extends CommitCoordinatorBuilder {
- override def getName: String = commitOwnerName
+ override def getName: String = commitCoordinatorName
- lazy val commitOwnerClient: CommitOwnerClient = {
- new InMemoryCommitOwner(batchSize = 1000L) {
+ lazy val commitCoordinatorClient: CommitCoordinatorClient = {
+ new InMemoryCommitCoordinator(batchSize = 1000L) {
override def commit(
logStore: LogStore,
hadoopConf: Configuration,
@@ -570,22 +572,24 @@ class OptimisticTransactionSuite
}
}
override def build(
- spark: SparkSession, conf: Map[String, String]): CommitOwnerClient = commitOwnerClient
+ spark: SparkSession,
+ conf: Map[String, String]): CommitCoordinatorClient = commitCoordinatorClient
}
- CommitOwnerProvider.registerBuilder(FileAlreadyExistsCommitOwnerBuilder)
+ CommitCoordinatorProvider.registerBuilder(FileAlreadyExistsCommitCoordinatorBuilder)
withSQLConf(
DeltaSQLConf.DELTA_MAX_RETRY_COMMIT_ATTEMPTS.key -> "100",
DeltaSQLConf.DELTA_MAX_NON_CONFLICT_RETRY_COMMIT_ATTEMPTS.key -> "10") {
withTempDir { tempDir =>
val log = DeltaLog.forTable(spark, new Path(tempDir.getCanonicalPath))
- val conf = Map(DeltaConfigs.MANAGED_COMMIT_OWNER_NAME.key -> commitOwnerName)
+ val conf =
+ Map(DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_NAME.key -> commitCoordinatorName)
log.startTransaction().commit(Seq(Metadata(configuration = conf)), ManualUpdate)
val testTxn = log.startTransaction()
intercept[FileAlreadyExistsException] { testTxn.commit(Seq.empty, ManualUpdate) }
// Test that there are no retries for the FileAlreadyExistsException in
- // CommitOwnerClient.commit()
+ // CommitCoordinatorClient.commit()
// num-attempts(1) = 1 + num-retries(0)
assert(commitAttempts == 1)
}
@@ -857,8 +861,9 @@ class OptimisticTransactionSuite
test(s"commitLarge should handle Commit Failed Exception with conflict: $conflict") {
withTempDir { tempDir =>
val deltaLog = DeltaLog.forTable(spark, tempDir.getAbsolutePath)
- val commitOwnerName = "retryable-conflict-commit-owner"
- class RetryableConflictCommitOwnerClient extends InMemoryCommitOwner(batchSize = 5) {
+ val commitCoordinatorName = "retryable-conflict-commit-coordinator"
+ class RetryableConflictCommitCoordinatorClient
+ extends InMemoryCommitCoordinator(batchSize = 5) {
override def commit(
logStore: LogStore,
hadoopConf: Configuration,
@@ -876,14 +881,16 @@ class OptimisticTransactionSuite
logStore, hadoopConf, tablePath, tableConf, commitVersion, actions, updatedActions)
}
}
- object RetryableConflictCommitOwnerBuilder$ extends CommitOwnerBuilder {
- lazy val commitOwnerClient = new RetryableConflictCommitOwnerClient()
- override def getName: String = commitOwnerName
+ object RetryableConflictCommitCoordinatorBuilder$ extends CommitCoordinatorBuilder {
+ lazy val commitCoordinatorClient = new RetryableConflictCommitCoordinatorClient()
+ override def getName: String = commitCoordinatorName
override def build(
- spark: SparkSession, conf: Map[String, String]): CommitOwnerClient = commitOwnerClient
+ spark: SparkSession,
+ conf: Map[String, String]): CommitCoordinatorClient = commitCoordinatorClient
}
- CommitOwnerProvider.registerBuilder(RetryableConflictCommitOwnerBuilder$)
- val conf = Map(DeltaConfigs.MANAGED_COMMIT_OWNER_NAME.key -> commitOwnerName)
+ CommitCoordinatorProvider.registerBuilder(RetryableConflictCommitCoordinatorBuilder$)
+ val conf =
+ Map(DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_NAME.key -> commitCoordinatorName)
deltaLog.startTransaction().commit(Seq(Metadata(configuration = conf)), ManualUpdate)
deltaLog.startTransaction().commit(addA :: Nil, ManualUpdate)
val records = Log4jUsageLogger.track {
@@ -911,9 +918,9 @@ class OptimisticTransactionSuite
val failureRecord = filterUsageRecords(records, "delta.commitLarge.failure")
assert(failureRecord.size == 1)
val data = JsonUtils.fromJson[Map[String, Any]](failureRecord.head.blob)
- assert(data("fromManagedCommit") == true)
- assert(data("fromManagedCommitConflict") == conflict)
- assert(data("fromManagedCommitRetryable") == true)
+ assert(data("fromCoordinatedCommits") == true)
+ assert(data("fromCoordinatedCommitsConflict") == conflict)
+ assert(data("fromCoordinatedCommitsRetryable") == true)
}
}
}
diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/SnapshotManagementSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/SnapshotManagementSuite.scala
index 375a3043346..4dcf10c6ba0 100644
--- a/spark/src/test/scala/org/apache/spark/sql/delta/SnapshotManagementSuite.scala
+++ b/spark/src/test/scala/org/apache/spark/sql/delta/SnapshotManagementSuite.scala
@@ -22,10 +22,10 @@ import java.util.concurrent.CountDownLatch
import scala.collection.mutable
import com.databricks.spark.util.{Log4jUsageLogger, UsageRecord}
-import org.apache.spark.sql.delta.DeltaConfigs.MANAGED_COMMIT_OWNER_NAME
+import org.apache.spark.sql.delta.DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_NAME
import org.apache.spark.sql.delta.DeltaTestUtils.{verifyBackfilled, verifyUnbackfilled, BOOLEAN_DOMAIN}
import org.apache.spark.sql.delta.SnapshotManagementSuiteShims._
-import org.apache.spark.sql.delta.managedcommit._
+import org.apache.spark.sql.delta.coordinatedcommits._
import org.apache.spark.sql.delta.sources.DeltaSQLConf
import org.apache.spark.sql.delta.storage.LocalLogStore
import org.apache.spark.sql.delta.storage.LogStore
@@ -46,7 +46,7 @@ import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.storage.StorageLevel
class SnapshotManagementSuite extends QueryTest with DeltaSQLTestUtils with SharedSparkSession
- with DeltaSQLCommandTest with ManagedCommitBaseSuite {
+ with DeltaSQLCommandTest with CoordinatedCommitsBaseSuite {
/**
@@ -305,7 +305,7 @@ class SnapshotManagementSuite extends QueryTest with DeltaSQLTestUtils with Shar
// Delete delta files
new File(tempDir, "_delta_log").listFiles().filter(_.getName.endsWith(".json"))
.foreach(_.delete())
- if (managedCommitEnabledInTests) {
+ if (coordinatedCommitsEnabledInTests) {
new File(new File(tempDir, "_delta_log"), "_commits")
.listFiles()
.filter(_.getName.endsWith(".json"))
@@ -467,7 +467,7 @@ class SnapshotManagementSuite extends QueryTest with DeltaSQLTestUtils with Shar
spark.range(10).write.format("delta").save(path)
val newLogSegment = log.snapshot.logSegment
assert(log.getLogSegmentAfterCommit(
- log.snapshot.tableCommitOwnerClientOpt,
+ log.snapshot.tableCommitCoordinatorClientOpt,
oldLogSegment.checkpointProvider) === newLogSegment)
spark.range(10).write.format("delta").mode("append").save(path)
val fs = log.logPath.getFileSystem(log.newDeltaHadoopConf())
@@ -492,7 +492,7 @@ class SnapshotManagementSuite extends QueryTest with DeltaSQLTestUtils with Shar
log.getLogSegmentAfterCommit(0, None, newLogSegment, commit, None, EmptyCheckpointProvider)
}
assert(log.getLogSegmentAfterCommit(
- log.snapshot.tableCommitOwnerClientOpt,
+ log.snapshot.tableCommitCoordinatorClientOpt,
oldLogSegment.checkpointProvider) === log.snapshot.logSegment)
}
}
@@ -519,23 +519,23 @@ class SnapshotManagementSuite extends QueryTest with DeltaSQLTestUtils with Shar
}
}
-class SnapshotManagementWithManagedCommitBatch1Suite extends SnapshotManagementSuite {
- override def managedCommitBackfillBatchSize: Option[Int] = Some(1)
+class SnapshotManagementWithCoordinatedCommitsBatch1Suite extends SnapshotManagementSuite {
+ override def coordinatedCommitsBackfillBatchSize: Option[Int] = Some(1)
}
-class SnapshotManagementWithManagedCommitBatch2Suite extends SnapshotManagementSuite {
- override def managedCommitBackfillBatchSize: Option[Int] = Some(2)
+class SnapshotManagementWithCoordinatedCommitsBatch2Suite extends SnapshotManagementSuite {
+ override def coordinatedCommitsBackfillBatchSize: Option[Int] = Some(2)
}
-class SnapshotManagementWithManagedCommitBatch100Suite extends SnapshotManagementSuite {
- override def managedCommitBackfillBatchSize: Option[Int] = Some(100)
+class SnapshotManagementWithCoordinatedCommitsBatch100Suite extends SnapshotManagementSuite {
+ override def coordinatedCommitsBackfillBatchSize: Option[Int] = Some(100)
}
class CountDownLatchLogStore(sparkConf: SparkConf, hadoopConf: Configuration)
extends LocalLogStore(sparkConf, hadoopConf) {
override def listFrom(path: Path, hadoopConf: Configuration): Iterator[FileStatus] = {
val files = super.listFrom(path, hadoopConf).toSeq
- if (ConcurrentBackfillCommitOwnerClient.beginConcurrentBackfills) {
+ if (ConcurrentBackfillCommitCoordinatorClient.beginConcurrentBackfills) {
CountDownLatchLogStore.listFromCalled.countDown()
}
files.iterator
@@ -545,23 +545,23 @@ object CountDownLatchLogStore {
val listFromCalled = new CountDownLatch(1)
}
-case class ConcurrentBackfillCommitOwnerClient(
+case class ConcurrentBackfillCommitCoordinatorClient(
synchronousBackfillThreshold: Long,
override val batchSize: Long
-) extends InMemoryCommitOwner(batchSize) {
+) extends InMemoryCommitCoordinator(batchSize) {
private val deferredBackfills: mutable.Map[Long, () => Unit] = mutable.Map.empty
override def getCommits(
logPath: Path,
- managedCommitTableConf: Map[String, String],
+ coordinatedCommitsTableConf: Map[String, String],
startVersion: Option[Long],
endVersion: Option[Long]): GetCommitsResponse = {
- if (ConcurrentBackfillCommitOwnerClient.beginConcurrentBackfills) {
+ if (ConcurrentBackfillCommitCoordinatorClient.beginConcurrentBackfills) {
CountDownLatchLogStore.listFromCalled.await()
logInfo(s"Finishing pending backfills concurrently: ${deferredBackfills.keySet}")
deferredBackfills.keys.toSeq.sorted.foreach((version: Long) => deferredBackfills(version)())
deferredBackfills.clear()
}
- super.getCommits(logPath, managedCommitTableConf, startVersion, endVersion)
+ super.getCommits(logPath, coordinatedCommitsTableConf, startVersion, endVersion)
}
override def backfill(
logStore: LogStore,
@@ -570,7 +570,7 @@ case class ConcurrentBackfillCommitOwnerClient(
version: Long,
fileStatus: FileStatus): Unit = {
if (version > synchronousBackfillThreshold &&
- ConcurrentBackfillCommitOwnerClient.deferBackfills) {
+ ConcurrentBackfillCommitCoordinatorClient.deferBackfills) {
deferredBackfills(version) = () =>
super.backfill(logStore, hadoopConf, logPath, version, fileStatus)
} else {
@@ -578,36 +578,37 @@ case class ConcurrentBackfillCommitOwnerClient(
}
}
}
-object ConcurrentBackfillCommitOwnerClient {
+object ConcurrentBackfillCommitCoordinatorClient {
var deferBackfills = false
var beginConcurrentBackfills = false
}
-object ConcurrentBackfillCommitOwnerBuilder extends CommitOwnerBuilder {
+object ConcurrentBackfillCommitCoordinatorBuilder extends CommitCoordinatorBuilder {
val batchSize = 5
- private lazy val concurrentBackfillCommitOwnerClient =
- ConcurrentBackfillCommitOwnerClient(synchronousBackfillThreshold = 2, batchSize)
- override def getName: String = "awaiting-commit-owner"
- override def build(spark: SparkSession, conf: Map[String, String]): CommitOwnerClient = {
- concurrentBackfillCommitOwnerClient
+ private lazy val concurrentBackfillCommitCoordinatorClient =
+ ConcurrentBackfillCommitCoordinatorClient(synchronousBackfillThreshold = 2, batchSize)
+ override def getName: String = "awaiting-commit-coordinator"
+ override def build(spark: SparkSession, conf: Map[String, String]): CommitCoordinatorClient = {
+ concurrentBackfillCommitCoordinatorClient
}
}
/**
* Setup (Assuming batch size = 5 & synchronousBackfillThreshold = 2):
* - LogStore contains backfilled commits [0, 2]
- * - CommitOwnerClient contains unbackfilled commits [3, ...]
+ * - CommitCoordinatorClient contains unbackfilled commits [3, ...]
* - Backfills are pending for versions [3, 5]
*
- * Goal: Create a gap for versions [3, 5] in the LogStore and CommitOwnerClient listings.
+ * Goal: Create a gap for versions [3, 5] in the LogStore and CommitCoordinatorClient listings.
*
* Step 1: LogStore retrieves delta files for versions [0, 2] from the file system.
* Step 2: Wait on the latch to ensure step (1) is completed before step (3) begins.
- * Step 3: Backfill commits [3, 5] from CommitOwnerClient to LogStore using deferredBackfills map.
- * Step 4: CommitOwnerClient returns commits [6, ...] (if valid).
+ * Step 3: Backfill commits [3, 5] from CommitCoordinatorClient to LogStore using deferredBackfills
+ * map.
+ * Step 4: CommitCoordinatorClient returns commits [6, ...] (if valid).
*
- * Test that the code correctly handles the gap in the LogStore and CommitOwnerClient listings by
- * making an additional call to LogStore to fetch versions [3, 5].
+ * Test that the code correctly handles the gap in the LogStore and CommitCoordinatorClient listings
+ * by making an additional call to LogStore to fetch versions [3, 5].
*/
class SnapshotManagementParallelListingSuite extends QueryTest
with SharedSparkSession
@@ -618,10 +619,10 @@ class SnapshotManagementParallelListingSuite extends QueryTest
override protected def beforeEach(): Unit = {
super.beforeEach()
- CommitOwnerProvider.clearNonDefaultBuilders()
- CommitOwnerProvider.registerBuilder(ConcurrentBackfillCommitOwnerBuilder)
- ConcurrentBackfillCommitOwnerClient.beginConcurrentBackfills = false
- ConcurrentBackfillCommitOwnerClient.deferBackfills = false
+ CommitCoordinatorProvider.clearNonDefaultBuilders()
+ CommitCoordinatorProvider.registerBuilder(ConcurrentBackfillCommitCoordinatorBuilder)
+ ConcurrentBackfillCommitCoordinatorClient.beginConcurrentBackfills = false
+ ConcurrentBackfillCommitCoordinatorClient.deferBackfills = false
}
private def writeDeltaData(path: String, endVersion: Long): Unit = {
@@ -667,12 +668,12 @@ class SnapshotManagementParallelListingSuite extends QueryTest
test(
s"Backfills are properly reconciled with concurrentBackfills: $concurrentBackfills, " +
s"tryIncludeGapAtTheEnd: $tryIncludeGapAtTheEnd") {
- ConcurrentBackfillCommitOwnerClient.deferBackfills = concurrentBackfills
- val batchSize = ConcurrentBackfillCommitOwnerBuilder.batchSize
+ ConcurrentBackfillCommitCoordinatorClient.deferBackfills = concurrentBackfills
+ val batchSize = ConcurrentBackfillCommitCoordinatorBuilder.batchSize
val endVersion = if (tryIncludeGapAtTheEnd) { batchSize } else { batchSize + 3 }
withSQLConf(
- MANAGED_COMMIT_OWNER_NAME.defaultTablePropertyKey ->
- ConcurrentBackfillCommitOwnerBuilder.getName,
+ COORDINATED_COMMITS_COORDINATOR_NAME.defaultTablePropertyKey ->
+ ConcurrentBackfillCommitCoordinatorBuilder.getName,
DeltaSQLConf.DELTALOG_MINOR_COMPACTION_USE_FOR_READS.key -> "false") {
withTempDir { tempDir =>
val path = tempDir.getCanonicalPath
@@ -682,7 +683,7 @@ class SnapshotManagementParallelListingSuite extends QueryTest
// Invalidate cache to ensure re-listing.
DeltaLog.invalidateCache(spark, dataPath)
- ConcurrentBackfillCommitOwnerClient.beginConcurrentBackfills = true
+ ConcurrentBackfillCommitCoordinatorClient.beginConcurrentBackfills = true
val (snapshot, records) = captureUsageRecordsAndGetSnapshot(dataPath)
val expectedNeedAdditionalFsListingCount = if (concurrentBackfills) { 1 } else { 0 }
diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/managedcommit/CommitOwnerClientImplSuiteBase.scala b/spark/src/test/scala/org/apache/spark/sql/delta/coordinatedcommits/CommitCoordinatorClientImplSuiteBase.scala
similarity index 73%
rename from spark/src/test/scala/org/apache/spark/sql/delta/managedcommit/CommitOwnerClientImplSuiteBase.scala
rename to spark/src/test/scala/org/apache/spark/sql/delta/coordinatedcommits/CommitCoordinatorClientImplSuiteBase.scala
index f8b621067eb..7747c7da721 100644
--- a/spark/src/test/scala/org/apache/spark/sql/delta/managedcommit/CommitOwnerClientImplSuiteBase.scala
+++ b/spark/src/test/scala/org/apache/spark/sql/delta/coordinatedcommits/CommitCoordinatorClientImplSuiteBase.scala
@@ -14,7 +14,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.delta.managedcommit
+package org.apache.spark.sql.delta.coordinatedcommits
import java.io.File
import java.util.concurrent.{Executors, TimeUnit}
@@ -28,7 +28,7 @@ import org.apache.spark.sql.delta.storage.{LogStore, LogStoreProvider}
import org.apache.spark.sql.delta.test.{DeltaSQLCommandTest, DeltaSQLTestUtils}
import org.apache.spark.sql.delta.util.FileNames
import org.apache.spark.sql.delta.util.threads.DeltaThreadPool
-import io.delta.dynamodbcommitstore.DynamoDBCommitOwnerClient
+import io.delta.dynamodbcommitcoordinator.DynamoDBCommitCoordinatorClient
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
@@ -36,35 +36,35 @@ import org.apache.spark.sql.QueryTest
import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.util.{ThreadUtils, Utils}
-trait CommitOwnerClientImplSuiteBase extends QueryTest
+trait CommitCoordinatorClientImplSuiteBase extends QueryTest
with SharedSparkSession
with LogStoreProvider
- with ManagedCommitTestUtils
+ with CoordinatedCommitsTestUtils
with DeltaSQLTestUtils
with DeltaSQLCommandTest {
/**
- * Needs to be overwritten by implementing classes to provide a [[TableCommitOwnerClient]]
- * wrapping the commit owner client that should be tested.
+ * Needs to be overwritten by implementing classes to provide a [[TableCommitCoordinatorClient]]
+ * wrapping the commit coordinator client that should be tested.
*/
- protected def createTableCommitOwnerClient(deltaLog: DeltaLog): TableCommitOwnerClient
+ protected def createTableCommitCoordinatorClient(deltaLog: DeltaLog): TableCommitCoordinatorClient
/**
* Needs to be overwritten by implementing classes to provide an implementation
* of backfill registration.
*/
protected def registerBackfillOp(
- tableCommitOwnerClient: TableCommitOwnerClient,
+ tableCommitCoordinatorClient: TableCommitCoordinatorClient,
deltaLog: DeltaLog,
version: Long): Unit
/**
* Needs to be overwritten by implementing classes to provide a way of validating
- * that the commit owner client under test performs backfilling as expected at
+ * that the commit coordinator client under test performs backfilling as expected at
* the specified version.
*/
protected def validateBackfillStrategy(
- tableCommitOwnerClient: TableCommitOwnerClient,
+ tableCommitCoordinatorClient: TableCommitCoordinatorClient,
logPath: Path,
version: Long): Unit
@@ -80,8 +80,8 @@ trait CommitOwnerClientImplSuiteBase extends QueryTest
maxVersion: Long): Unit
/**
- * Checks that the commit owner state is correct in terms of
- * - The latest table version in the commit owner is correct
+ * Checks that the commit coordinator state is correct in terms of
+ * - The latest table version in the commit coordinator is correct
* - All supposedly backfilled commits are indeed backfilled
* - The contents of the backfilled commits are correct (verified
* if commitTimestampOpt is provided)
@@ -91,17 +91,17 @@ trait CommitOwnerClientImplSuiteBase extends QueryTest
*/
protected def assertInvariants(
logPath: Path,
- tableCommitOwnerClient: TableCommitOwnerClient,
+ tableCommitCoordinatorClient: TableCommitCoordinatorClient,
commitTimestampsOpt: Option[Array[Long]] = None): Unit = {
val maxUntrackedVersion: Int = {
- val commitResponse = tableCommitOwnerClient.getCommits()
+ val commitResponse = tableCommitCoordinatorClient.getCommits()
if (commitResponse.getCommits.isEmpty) {
commitResponse.getLatestTableVersion.toInt
} else {
assert(
commitResponse.getCommits.last.getVersion == commitResponse.getLatestTableVersion,
- s"Max commit tracked by the commit owner ${commitResponse.getCommits.last} must " +
- s"match latestTableVersion tracked by the commit owner " +
+ s"Max commit tracked by the commit coordinator ${commitResponse.getCommits.last} must " +
+ s"match latestTableVersion tracked by the commit coordinator " +
s"${commitResponse.getLatestTableVersion}."
)
val minVersion = commitResponse.getCommits.head.getVersion
@@ -149,7 +149,7 @@ trait CommitOwnerClientImplSuiteBase extends QueryTest
protected def commit(
version: Long,
timestamp: Long,
- tableCommitOwnerClient: TableCommitOwnerClient): Commit = {
+ tableCommitCoordinatorClient: TableCommitCoordinatorClient): Commit = {
val commitInfo = CommitInfo.empty(version = Some(version)).withTimestamp(timestamp)
.copy(inCommitTimestamp = Some(timestamp))
val updatedActions = if (version == 0) {
@@ -157,7 +157,7 @@ trait CommitOwnerClientImplSuiteBase extends QueryTest
} else {
getUpdatedActionsForNonZerothCommit(commitInfo)
}
- tableCommitOwnerClient.commit(
+ tableCommitCoordinatorClient.commit(
version,
Iterator(s"$version", s"$timestamp"),
updatedActions).getCommit
@@ -197,34 +197,34 @@ trait CommitOwnerClientImplSuiteBase extends QueryTest
withTempTableDir { tempDir =>
val log = DeltaLog.forTable(spark, tempDir.toString)
val logPath = log.logPath
- val tableCommitOwnerClient = createTableCommitOwnerClient(log)
+ val tableCommitCoordinatorClient = createTableCommitCoordinatorClient(log)
val e = intercept[CommitFailedException] {
- commit(version = 0, timestamp = 0, tableCommitOwnerClient)
+ commit(version = 0, timestamp = 0, tableCommitCoordinatorClient)
}
assert(e.getMessage === "Commit version 0 must go via filesystem.")
writeCommitZero(logPath)
- assert(tableCommitOwnerClient.getCommits() == GetCommitsResponse(Seq.empty, -1))
+ assert(tableCommitCoordinatorClient.getCommits() == GetCommitsResponse(Seq.empty, -1))
assertBackfilled(version = 0, logPath, Some(0L))
// Test backfilling functionality for commits 1 - 8
(1 to 8).foreach { version =>
- commit(version, version, tableCommitOwnerClient)
- validateBackfillStrategy(tableCommitOwnerClient, logPath, version)
- assert(tableCommitOwnerClient.getCommits().getLatestTableVersion == version)
+ commit(version, version, tableCommitCoordinatorClient)
+ validateBackfillStrategy(tableCommitCoordinatorClient, logPath, version)
+ assert(tableCommitCoordinatorClient.getCommits().getLatestTableVersion == version)
}
// Test that out-of-order backfill is rejected
intercept[IllegalArgumentException] {
- registerBackfillOp(tableCommitOwnerClient, log, 10)
+ registerBackfillOp(tableCommitCoordinatorClient, log, 10)
}
- assertInvariants(logPath, tableCommitOwnerClient)
+ assertInvariants(logPath, tableCommitCoordinatorClient)
}
}
test("startVersion and endVersion are respected in getCommits") {
def runGetCommitsAndValidate(
- client: TableCommitOwnerClient,
+ client: TableCommitCoordinatorClient,
startVersion: Option[Long],
endVersion: Option[Long],
maxVersion: Long): Unit = {
@@ -236,18 +236,18 @@ trait CommitOwnerClientImplSuiteBase extends QueryTest
// prepare a table with 15 commits
val log = DeltaLog.forTable(spark, tempDir.toString)
val logPath = log.logPath
- val tableCommitOwnerClient = createTableCommitOwnerClient(log)
+ val tableCommitCoordinatorClient = createTableCommitCoordinatorClient(log)
writeCommitZero(logPath)
val maxVersion = 15
(1 to maxVersion).foreach { version =>
- commit(version, version, tableCommitOwnerClient)
+ commit(version, version, tableCommitCoordinatorClient)
}
- runGetCommitsAndValidate(tableCommitOwnerClient, None, None, maxVersion)
- runGetCommitsAndValidate(tableCommitOwnerClient, Some(9), None, maxVersion)
- runGetCommitsAndValidate(tableCommitOwnerClient, Some(11), Some(14), maxVersion)
- runGetCommitsAndValidate(tableCommitOwnerClient, Some(12), Some(12), maxVersion)
- runGetCommitsAndValidate(tableCommitOwnerClient, None, Some(14), maxVersion)
+ runGetCommitsAndValidate(tableCommitCoordinatorClient, None, None, maxVersion)
+ runGetCommitsAndValidate(tableCommitCoordinatorClient, Some(9), None, maxVersion)
+ runGetCommitsAndValidate(tableCommitCoordinatorClient, Some(11), Some(14), maxVersion)
+ runGetCommitsAndValidate(tableCommitCoordinatorClient, Some(12), Some(12), maxVersion)
+ runGetCommitsAndValidate(tableCommitCoordinatorClient, None, Some(14), maxVersion)
}
}
@@ -255,18 +255,18 @@ trait CommitOwnerClientImplSuiteBase extends QueryTest
withTempTableDir { tempDir =>
val log = DeltaLog.forTable(spark, tempDir.getPath)
val logPath = log.logPath
- val tableCommitOwnerClient = createTableCommitOwnerClient(log)
+ val tableCommitCoordinatorClient = createTableCommitCoordinatorClient(log)
// commit-0 must be file system based
writeCommitZero(logPath)
- (1 to 3).foreach(i => commit(i, i, tableCommitOwnerClient))
+ (1 to 3).foreach(i => commit(i, i, tableCommitCoordinatorClient))
// Test that backfilling is idempotent for already-backfilled commits.
- registerBackfillOp(tableCommitOwnerClient, log, 2)
- registerBackfillOp(tableCommitOwnerClient, log, 2)
+ registerBackfillOp(tableCommitCoordinatorClient, log, 2)
+ registerBackfillOp(tableCommitCoordinatorClient, log, 2)
// Test that backfilling uncommited commits fail.
intercept[IllegalArgumentException] {
- registerBackfillOp(tableCommitOwnerClient, log, 4)
+ registerBackfillOp(tableCommitCoordinatorClient, log, 4)
}
}
}
@@ -275,23 +275,23 @@ trait CommitOwnerClientImplSuiteBase extends QueryTest
withTempTableDir { tempDir =>
val log = DeltaLog.forTable(spark, tempDir.toString)
val logPath = log.logPath
- val tableCommitOwnerClient = createTableCommitOwnerClient(log)
+ val tableCommitCoordinatorClient = createTableCommitCoordinatorClient(log)
// commit-0 must be file system based
writeCommitZero(logPath)
// Verify that conflict-checker rejects out-of-order commits.
- (1 to 4).foreach(i => commit(i, i, tableCommitOwnerClient))
- // A retry of commit 0 fails from commit owner client with a conflict and it can't be
- // retried as commit 0 is upgrading the commit owner client.
- assertCommitFail(0, 5, retryable = false, commit(0, 5, tableCommitOwnerClient))
- assertCommitFail(4, 5, retryable = true, commit(4, 6, tableCommitOwnerClient))
-
- commit(5, 5, tableCommitOwnerClient)
- validateGetCommitsResult(tableCommitOwnerClient.getCommits(), None, None, 5)
- assertCommitFail(5, 6, retryable = true, commit(5, 5, tableCommitOwnerClient))
- assertCommitFail(7, 6, retryable = false, commit(7, 7, tableCommitOwnerClient))
-
- assertInvariants(logPath, tableCommitOwnerClient)
+ (1 to 4).foreach(i => commit(i, i, tableCommitCoordinatorClient))
+ // A retry of commit 0 fails from commit coordinator client with a conflict and it can't be
+ // retried as commit 0 is upgrading the commit coordinator client.
+ assertCommitFail(0, 5, retryable = false, commit(0, 5, tableCommitCoordinatorClient))
+ assertCommitFail(4, 5, retryable = true, commit(4, 6, tableCommitCoordinatorClient))
+
+ commit(5, 5, tableCommitCoordinatorClient)
+ validateGetCommitsResult(tableCommitCoordinatorClient.getCommits(), None, None, 5)
+ assertCommitFail(5, 6, retryable = true, commit(5, 5, tableCommitCoordinatorClient))
+ assertCommitFail(7, 6, retryable = false, commit(7, 7, tableCommitCoordinatorClient))
+
+ assertInvariants(logPath, tableCommitCoordinatorClient)
}
}
@@ -299,12 +299,12 @@ trait CommitOwnerClientImplSuiteBase extends QueryTest
withTempTableDir { tempDir =>
val tablePath = new Path(tempDir.getCanonicalPath)
val logPath = new Path(tablePath, DeltaLog.LOG_DIR_NAME)
- val tcs = createTableCommitOwnerClient(DeltaLog.forTable(spark, tablePath))
+ val tcs = createTableCommitCoordinatorClient(DeltaLog.forTable(spark, tablePath))
val numberOfWriters = 11
val numberOfCommitsPerWriter = 11
// scalastyle:off sparkThreadPools
- val executor = DeltaThreadPool("commitOwnerSuite", numberOfWriters)
+ val executor = DeltaThreadPool("commitCoordinatorSuite", numberOfWriters)
// scalastyle:on sparkThreadPools
val runningTimestamp = new AtomicInteger(0)
val commitFailedExceptions = new AtomicInteger(0)
diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/coordinatedcommits/CommitCoordinatorClientSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/coordinatedcommits/CommitCoordinatorClientSuite.scala
new file mode 100644
index 00000000000..2d0879a4cd8
--- /dev/null
+++ b/spark/src/test/scala/org/apache/spark/sql/delta/coordinatedcommits/CommitCoordinatorClientSuite.scala
@@ -0,0 +1,288 @@
+/*
+ * 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.coordinatedcommits
+
+import scala.reflect.runtime.universe._
+
+import org.apache.spark.sql.delta.{DeltaConfigs, DeltaLog, DeltaOperations, CoordinatedCommitsTableFeature}
+import org.apache.spark.sql.delta.actions._
+import org.apache.spark.sql.delta.storage.LogStore
+import org.apache.spark.sql.delta.test.DeltaSQLCommandTest
+import org.apache.spark.sql.delta.test.DeltaSQLTestUtils
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{QueryTest, SparkSession}
+import org.apache.spark.sql.test.SharedSparkSession
+
+class CommitCoordinatorClientSuite extends QueryTest with DeltaSQLTestUtils with SharedSparkSession
+ with DeltaSQLCommandTest {
+
+ private trait TestCommitCoordinatorClientBase extends CommitCoordinatorClient {
+ override def commit(
+ logStore: LogStore,
+ hadoopConf: Configuration,
+ logPath: Path,
+ coordinatedCommitsTableConf: Map[String, String],
+ commitVersion: Long,
+ actions: Iterator[String],
+ updatedActions: UpdatedActions): CommitResponse = {
+ throw new UnsupportedOperationException("Not implemented")
+ }
+
+ override def getCommits(
+ logPath: Path,
+ coordinatedCommitsTableConf: Map[String, String],
+ startVersion: Option[Long],
+ endVersion: Option[Long] = None): GetCommitsResponse = GetCommitsResponse(Seq.empty, -1)
+
+ override def backfillToVersion(
+ logStore: LogStore,
+ hadoopConf: Configuration,
+ logPath: Path,
+ coordinatedCommitsTableConf: Map[String, String],
+ version: Long,
+ lastKnownBackfilledVersion: Option[Long]): Unit = {}
+
+ override def semanticEquals(other: CommitCoordinatorClient): Boolean = this == other
+ }
+
+ private class TestCommitCoordinatorClient1 extends TestCommitCoordinatorClientBase
+ private class TestCommitCoordinatorClient2 extends TestCommitCoordinatorClientBase
+
+ override def beforeEach(): Unit = {
+ super.beforeEach()
+ CommitCoordinatorProvider.clearNonDefaultBuilders()
+ CommitCoordinatorProvider.registerBuilder(InMemoryCommitCoordinatorBuilder(batchSize = 1))
+ }
+
+ test("registering multiple commit-coordinator builders with same name") {
+ object Builder1 extends CommitCoordinatorBuilder {
+ override def build(
+ spark: SparkSession, conf: Map[String, String]): CommitCoordinatorClient = null
+ override def getName: String = "builder-1"
+ }
+ object BuilderWithSameName extends CommitCoordinatorBuilder {
+ override def build(
+ spark: SparkSession, conf: Map[String, String]): CommitCoordinatorClient = null
+ override def getName: String = "builder-1"
+ }
+ object Builder3 extends CommitCoordinatorBuilder {
+ override def build(
+ spark: SparkSession, conf: Map[String, String]): CommitCoordinatorClient = null
+ override def getName: String = "builder-3"
+ }
+ CommitCoordinatorProvider.registerBuilder(Builder1)
+ intercept[Exception] {
+ CommitCoordinatorProvider.registerBuilder(BuilderWithSameName)
+ }
+ CommitCoordinatorProvider.registerBuilder(Builder3)
+ }
+
+ test("getCommitCoordinator - builder returns same object") {
+ object Builder1 extends CommitCoordinatorBuilder {
+ val cs1 = new TestCommitCoordinatorClient1()
+ val cs2 = new TestCommitCoordinatorClient2()
+ override def build(
+ spark: SparkSession, conf: Map[String, String]): CommitCoordinatorClient = {
+ conf.getOrElse("url", "") match {
+ case "url1" => cs1
+ case "url2" => cs2
+ case _ => throw new IllegalArgumentException("Invalid url")
+ }
+ }
+ override def getName: String = "cs-x"
+ }
+ CommitCoordinatorProvider.registerBuilder(Builder1)
+ val cs1 =
+ CommitCoordinatorProvider.getCommitCoordinatorClient("cs-x", Map("url" -> "url1"), spark)
+ assert(cs1.isInstanceOf[TestCommitCoordinatorClient1])
+ val cs1_again =
+ CommitCoordinatorProvider.getCommitCoordinatorClient("cs-x", Map("url" -> "url1"), spark)
+ assert(cs1 eq cs1_again)
+ val cs2 = CommitCoordinatorProvider.getCommitCoordinatorClient(
+ "cs-x", Map("url" -> "url2", "a" -> "b"), spark)
+ assert(cs2.isInstanceOf[TestCommitCoordinatorClient2])
+ // If builder receives a config which doesn't have expected params, then it can throw exception.
+ intercept[IllegalArgumentException] {
+ CommitCoordinatorProvider.getCommitCoordinatorClient("cs-x", Map("url" -> "url3"), spark)
+ }
+ }
+
+ test("getCommitCoordinatorClient - builder returns new object each time") {
+ object Builder1 extends CommitCoordinatorBuilder {
+ override def build(
+ spark: SparkSession, conf: Map[String, String]): CommitCoordinatorClient = {
+ conf.getOrElse("url", "") match {
+ case "url1" => new TestCommitCoordinatorClient1()
+ case _ => throw new IllegalArgumentException("Invalid url")
+ }
+ }
+ override def getName: String = "cs-name"
+ }
+ CommitCoordinatorProvider.registerBuilder(Builder1)
+ val cs1 =
+ CommitCoordinatorProvider.getCommitCoordinatorClient("cs-name", Map("url" -> "url1"), spark)
+ assert(cs1.isInstanceOf[TestCommitCoordinatorClient1])
+ val cs1_again =
+ CommitCoordinatorProvider.getCommitCoordinatorClient("cs-name", Map("url" -> "url1"), spark)
+ assert(cs1 ne cs1_again)
+ }
+
+ test("COORDINATED_COMMITS_PROVIDER_CONF") {
+ val m1 = Metadata(
+ configuration = Map(
+ DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_CONF.key ->
+ """{"key1": "string_value", "key2Int": 2, "key3ComplexStr": "\"hello\""}""")
+ )
+ assert(DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_CONF.fromMetaData(m1) ===
+ Map("key1" -> "string_value", "key2Int" -> "2", "key3ComplexStr" -> "\"hello\""))
+
+ val m2 = Metadata(
+ configuration = Map(
+ DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_CONF.key ->
+ """{"key1": "string_value", "key2Int": "2""")
+ )
+ intercept[com.fasterxml.jackson.core.JsonParseException] {
+ DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_CONF.fromMetaData(m2)
+ }
+ }
+
+ test("Commit fails if we try to put bad value for COORDINATED_COMMITS_PROVIDER_CONF") {
+ withTempDir { dir =>
+ val path = dir.getCanonicalPath
+ spark.range(10).write.format("delta").mode("append").save(path)
+ val deltaLog = DeltaLog.forTable(spark, path)
+
+ val metadataWithCorrectConf = Metadata(
+ configuration = Map(
+ DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_CONF.key ->
+ """{"key1": "string_value", "key2Int": 2, "key3ComplexStr": "\"hello\""}""")
+ )
+ val metadataWithIncorrectConf = Metadata(
+ configuration = Map(
+ DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_CONF.key ->
+ """{"key1": "string_value", "key2Int": "2""")
+ )
+
+ intercept[com.fasterxml.jackson.core.JsonParseException] {
+ deltaLog.startTransaction().commit(
+ Seq(metadataWithIncorrectConf), DeltaOperations.ManualUpdate)
+ }
+ DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_CONF.fromMetaData(metadataWithCorrectConf)
+ }
+ }
+
+ test("Adding COORDINATED_COMMITS_PROVIDER_NAME table property " +
+ "automatically upgrades the Protocol") {
+ withTempDir { dir =>
+ val path = dir.getCanonicalPath
+ spark.range(10).write.format("delta").mode("append").save(path)
+ val metadata = Metadata(
+ configuration = Map(DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_NAME.key -> "in-memory"))
+ val deltaLog = DeltaLog.forTable(spark, path)
+
+ def getWriterFeatures(log: DeltaLog): Set[String] = {
+ log.update().protocol.writerFeatures.getOrElse(Set.empty)
+ }
+
+ assert(!getWriterFeatures(deltaLog).contains(CoordinatedCommitsTableFeature.name))
+ deltaLog.startTransaction().commit(Seq(metadata), DeltaOperations.ManualUpdate)
+ assert(getWriterFeatures(deltaLog).contains(CoordinatedCommitsTableFeature.name))
+ }
+ }
+
+ test("Semantic Equality works as expected on CommitCoordinatorClients") {
+ class TestCommitCoordinatorClient(val key: String) extends TestCommitCoordinatorClientBase {
+ override def semanticEquals(other: CommitCoordinatorClient): Boolean =
+ other.isInstanceOf[TestCommitCoordinatorClient] &&
+ other.asInstanceOf[TestCommitCoordinatorClient].key == key
+ }
+ object Builder1 extends CommitCoordinatorBuilder {
+ override def build(
+ spark: SparkSession, conf: Map[String, String]): CommitCoordinatorClient = {
+ new TestCommitCoordinatorClient(conf("key"))
+ }
+ override def getName: String = "cs-name"
+ }
+ CommitCoordinatorProvider.registerBuilder(Builder1)
+
+ // Different CommitCoordinator with same keys should be semantically equal.
+ val obj1 =
+ CommitCoordinatorProvider.getCommitCoordinatorClient("cs-name", Map("key" -> "url1"), spark)
+ val obj2 =
+ CommitCoordinatorProvider.getCommitCoordinatorClient("cs-name", Map("key" -> "url1"), spark)
+ assert(obj1 != obj2)
+ assert(obj1.semanticEquals(obj2))
+
+ // Different CommitCoordinator with different keys should be semantically unequal.
+ val obj3 =
+ CommitCoordinatorProvider.getCommitCoordinatorClient("cs-name", Map("key" -> "url2"), spark)
+ assert(obj1 != obj3)
+ assert(!obj1.semanticEquals(obj3))
+ }
+
+ private def checkMissing[Interface: TypeTag, Class: TypeTag](): Set[String] = {
+ val fields = typeOf[Class].decls.collect {
+ case m: MethodSymbol if m.isCaseAccessor => m.name.toString
+ }
+
+ val getters = typeOf[Interface].decls.collect {
+ case m: MethodSymbol if m.isAbstract => m.name.toString
+ }.toSet
+
+ fields.filterNot { field =>
+ getters.contains(s"get${field.capitalize}")
+ }.toSet
+ }
+
+ /**
+ * We expect the Protocol action to have the same fields as AbstractProtocol (part of the
+ * CommitCoordinatorClient interface). With this if any change has happened in the Protocol of the
+ * table, the same change is propagated to the CommitCoordinatorClient as AbstractProtocol. The
+ * CommitCoordinatorClient can access the changes using getters and decide to act on the changes
+ * based on the spec of the commit coordinator.
+ *
+ * This test case ensures that any new field added in the Protocol action is also accessible in
+ * the CommitCoordinatorClient via the getter. If the new field is something which we do not
+ * expect to be passed to the CommitCoordinatorClient, the test needs to be modified accordingly.
+ */
+ test("AbstractProtocol should have getter methods for all fields in Protocol") {
+ val missingFields = checkMissing[AbstractProtocol, Protocol]()
+ val expectedMissingFields = Set.empty[String]
+ assert(missingFields == expectedMissingFields,
+ s"Missing getter methods in AbstractProtocol")
+ }
+
+ /**
+ * We expect the Metadata action to have the same fields as AbstractMetadata (part of the
+ * CommitCoordinatorClient interface). With this if any change has happened in the Metadata of the
+ * table, the same change is propagated to the CommitCoordinatorClient as AbstractMetadata. The
+ * CommitCoordinatorClient can access the changes using getters and decide to act on the changes
+ * based on the spec of the commit coordinator.
+ *
+ * This test case ensures that any new field added in the Metadata action is also accessible in
+ * the CommitCoordinatorClient via the getter. If the new field is something which we do not
+ * expect to be passed to the CommitCoordinatorClient, the test needs to be modified accordingly.
+ */
+ test("BaseMetadata should have getter methods for all fields in Metadata") {
+ val missingFields = checkMissing[AbstractMetadata, Metadata]()
+ val expectedMissingFields = Set("format")
+ assert(missingFields == expectedMissingFields,
+ s"Missing getter methods in AbstractMetadata")
+ }
+}
diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/managedcommit/ManagedCommitEnablementSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/coordinatedcommits/CoordinatedCommitsEnablementSuite.scala
similarity index 70%
rename from spark/src/test/scala/org/apache/spark/sql/delta/managedcommit/ManagedCommitEnablementSuite.scala
rename to spark/src/test/scala/org/apache/spark/sql/delta/coordinatedcommits/CoordinatedCommitsEnablementSuite.scala
index cab5e4743d7..c1bc5557b6d 100644
--- a/spark/src/test/scala/org/apache/spark/sql/delta/managedcommit/ManagedCommitEnablementSuite.scala
+++ b/spark/src/test/scala/org/apache/spark/sql/delta/coordinatedcommits/CoordinatedCommitsEnablementSuite.scala
@@ -14,26 +14,30 @@
* limitations under the License.
*/
-package org.apache.spark.sql.delta.managedcommit
+package org.apache.spark.sql.delta.coordinatedcommits
import org.apache.spark.sql.delta._
import org.apache.spark.sql.delta.test.{DeltaSQLCommandTest, DeltaSQLTestUtils}
-class ManagedCommitEnablementSuite
- extends ManagedCommitBaseSuite
+class CoordinatedCommitsEnablementSuite
+ extends CoordinatedCommitsBaseSuite
with DeltaSQLTestUtils
with DeltaSQLCommandTest
- with ManagedCommitTestUtils {
+ with CoordinatedCommitsTestUtils {
- override def managedCommitBackfillBatchSize: Option[Int] = Some(3)
+ override def coordinatedCommitsBackfillBatchSize: Option[Int] = Some(3)
import testImplicits._
- private def validateManagedCommitCompleteEnablement(
+ private def validateCoordinatedCommitsCompleteEnablement(
snapshot: Snapshot, expectEnabled: Boolean): Unit = {
- assert(DeltaConfigs.MANAGED_COMMIT_OWNER_NAME.fromMetaData(snapshot.metadata).isDefined
- == expectEnabled)
- Seq(ManagedCommitTableFeature, VacuumProtocolCheckTableFeature, InCommitTimestampTableFeature)
+ assert(
+ DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_NAME.fromMetaData(snapshot.metadata).isDefined
+ == expectEnabled)
+ Seq(
+ CoordinatedCommitsTableFeature,
+ VacuumProtocolCheckTableFeature,
+ InCommitTimestampTableFeature)
.foreach { feature =>
assert(snapshot.protocol.writerFeatures.exists(_.contains(feature.name)) == expectEnabled)
}
@@ -51,7 +55,7 @@ class ManagedCommitEnablementSuite
val tablePath = tempDir.getAbsolutePath
Seq(1).toDF().write.format("delta").mode("overwrite").save(tablePath)
val log = DeltaLog.forTable(spark, tablePath)
- validateManagedCommitCompleteEnablement(log.snapshot, expectEnabled = true)
+ validateCoordinatedCommitsCompleteEnablement(log.snapshot, expectEnabled = true)
}
}
@@ -61,7 +65,7 @@ class ManagedCommitEnablementSuite
val tablePath = tempDir.getAbsolutePath
sql(s"CREATE TABLE delta.`$tablePath` (id LONG) USING delta")
val log = DeltaLog.forTable(spark, tablePath)
- validateManagedCommitCompleteEnablement(log.snapshot, expectEnabled = true)
+ validateCoordinatedCommitsCompleteEnablement(log.snapshot, expectEnabled = true)
}
}
@@ -72,13 +76,13 @@ class ManagedCommitEnablementSuite
Seq(1).toDF().write.format("delta").mode("overwrite").save(tablePath)
Seq(1).toDF().write.format("delta").mode("overwrite").save(tablePath)
val log = DeltaLog.forTable(spark, tablePath)
- validateManagedCommitCompleteEnablement(log.snapshot, expectEnabled = true)
+ validateCoordinatedCommitsCompleteEnablement(log.snapshot, expectEnabled = true)
}
}
// ---- Tests END: Enablement at commit 0 ----
// ---- Tests START: Enablement after commit 0 ----
- testWithDefaultCommitOwnerUnset(
+ testWithDefaultCommitCoordinatorUnset(
"enablement after commit 0: MC should enable ICT and VacuumProtocolCheck" +
" --- update tblproperty") {
withTempDir { tempDir =>
@@ -86,15 +90,16 @@ class ManagedCommitEnablementSuite
Seq(1).toDF().write.format("delta").mode("overwrite").save(tablePath) // commit 0
Seq(1).toDF().write.format("delta").mode("append").save(tablePath) // commit 1
val log = DeltaLog.forTable(spark, tablePath)
- validateManagedCommitCompleteEnablement(log.snapshot, expectEnabled = false)
+ validateCoordinatedCommitsCompleteEnablement(log.snapshot, expectEnabled = false)
sql(s"ALTER TABLE delta.`$tablePath` SET " + // Enable MC
- s"TBLPROPERTIES ('${DeltaConfigs.MANAGED_COMMIT_OWNER_NAME.key}' = 'tracking-in-memory')")
+ s"TBLPROPERTIES ('${DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_NAME.key}' " +
+ s"= 'tracking-in-memory')")
Seq(1).toDF().write.format("delta").mode("overwrite").save(tablePath) // commit 3
- validateManagedCommitCompleteEnablement(log.update(), expectEnabled = true)
+ validateCoordinatedCommitsCompleteEnablement(log.update(), expectEnabled = true)
}
}
- testWithDefaultCommitOwnerUnset(
+ testWithDefaultCommitCoordinatorUnset(
"enablement after commit 0: MC should enable ICT and VacuumProtocolCheck" +
" --- replace table") {
withTempDir { tempDir =>
@@ -102,11 +107,12 @@ class ManagedCommitEnablementSuite
Seq(1).toDF().write.format("delta").mode("overwrite").save(tablePath) // commit 0
Seq(1).toDF().write.format("delta").mode("append").save(tablePath) // commit 1
val log = DeltaLog.forTable(spark, tablePath)
- validateManagedCommitCompleteEnablement(log.snapshot, expectEnabled = false)
+ validateCoordinatedCommitsCompleteEnablement(log.snapshot, expectEnabled = false)
sql(s"REPLACE TABLE delta.`$tablePath` (value int) USING delta " + // Enable MC
- s"TBLPROPERTIES ('${DeltaConfigs.MANAGED_COMMIT_OWNER_NAME.key}' = 'tracking-in-memory')")
+ s"TBLPROPERTIES ('${DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_NAME.key}' " +
+ s"= 'tracking-in-memory')")
Seq(1).toDF().write.format("delta").mode("overwrite").save(tablePath) // commit 3
- validateManagedCommitCompleteEnablement(log.update(), expectEnabled = true)
+ validateCoordinatedCommitsCompleteEnablement(log.update(), expectEnabled = true)
}
}
// ---- Tests END: Enablement after commit 0 ----
diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/managedcommit/ManagedCommitSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/coordinatedcommits/CoordinatedCommitsSuite.scala
similarity index 72%
rename from spark/src/test/scala/org/apache/spark/sql/delta/managedcommit/ManagedCommitSuite.scala
rename to spark/src/test/scala/org/apache/spark/sql/delta/coordinatedcommits/CoordinatedCommitsSuite.scala
index 98ff00433db..3f6d15116b4 100644
--- a/spark/src/test/scala/org/apache/spark/sql/delta/managedcommit/ManagedCommitSuite.scala
+++ b/spark/src/test/scala/org/apache/spark/sql/delta/coordinatedcommits/CoordinatedCommitsSuite.scala
@@ -14,16 +14,16 @@
* limitations under the License.
*/
-package org.apache.spark.sql.delta.managedcommit
+package org.apache.spark.sql.delta.coordinatedcommits
import java.io.File
import scala.collection.mutable.ArrayBuffer
import com.databricks.spark.util.Log4jUsageLogger
-import org.apache.spark.sql.delta.{DeltaOperations, ManagedCommitTableFeature, V2CheckpointTableFeature}
-import org.apache.spark.sql.delta.CommitOwnerGetCommitsFailedException
-import org.apache.spark.sql.delta.DeltaConfigs.{CHECKPOINT_INTERVAL, MANAGED_COMMIT_OWNER_CONF, MANAGED_COMMIT_OWNER_NAME, MANAGED_COMMIT_TABLE_CONF}
+import org.apache.spark.sql.delta.{DeltaOperations, CoordinatedCommitsTableFeature, V2CheckpointTableFeature}
+import org.apache.spark.sql.delta.CommitCoordinatorGetCommitsFailedException
+import org.apache.spark.sql.delta.DeltaConfigs.{CHECKPOINT_INTERVAL, COORDINATED_COMMITS_COORDINATOR_CONF, COORDINATED_COMMITS_COORDINATOR_NAME, COORDINATED_COMMITS_TABLE_CONF}
import org.apache.spark.sql.delta.DeltaLog
import org.apache.spark.sql.delta.DeltaTestUtils.createTestAddFile
import org.apache.spark.sql.delta.InitialSnapshot
@@ -44,40 +44,40 @@ import org.apache.spark.sql.{QueryTest, Row, SparkSession}
import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.util.ManualClock
-class ManagedCommitSuite
+class CoordinatedCommitsSuite
extends QueryTest
with DeltaSQLTestUtils
with SharedSparkSession
with DeltaSQLCommandTest
- with ManagedCommitTestUtils {
+ with CoordinatedCommitsTestUtils {
import testImplicits._
override def sparkConf: SparkConf = {
- // Make sure all new tables in tests use tracking-in-memory commit-owner by default.
+ // Make sure all new tables in tests use tracking-in-memory commit-coordinator by default.
super.sparkConf
- .set(MANAGED_COMMIT_OWNER_NAME.defaultTablePropertyKey, "tracking-in-memory")
- .set(MANAGED_COMMIT_OWNER_CONF.defaultTablePropertyKey, JsonUtils.toJson(Map()))
+ .set(COORDINATED_COMMITS_COORDINATOR_NAME.defaultTablePropertyKey, "tracking-in-memory")
+ .set(COORDINATED_COMMITS_COORDINATOR_CONF.defaultTablePropertyKey, JsonUtils.toJson(Map()))
}
override def beforeEach(): Unit = {
super.beforeEach()
- CommitOwnerProvider.clearNonDefaultBuilders()
+ CommitCoordinatorProvider.clearNonDefaultBuilders()
}
test("0th commit happens via filesystem") {
- val commitOwnerName = "nobackfilling-commit-owner"
- object NoBackfillingCommitOwnerBuilder$ extends CommitOwnerBuilder {
+ val commitCoordinatorName = "nobackfilling-commit-coordinator"
+ object NoBackfillingCommitCoordinatorBuilder$ extends CommitCoordinatorBuilder {
- override def getName: String = commitOwnerName
+ override def getName: String = commitCoordinatorName
- override def build(spark: SparkSession, conf: Map[String, String]): CommitOwnerClient =
- new InMemoryCommitOwner(batchSize = 5) {
+ override def build(spark: SparkSession, conf: Map[String, String]): CommitCoordinatorClient =
+ new InMemoryCommitCoordinator(batchSize = 5) {
override def commit(
logStore: LogStore,
hadoopConf: Configuration,
logPath: Path,
- managedCommitTableConf: Map[String, String],
+ coordinatedCommitsTableConf: Map[String, String],
commitVersion: Long,
actions: Iterator[String],
updatedActions: UpdatedActions): CommitResponse = {
@@ -86,8 +86,9 @@ class ManagedCommitSuite
}
}
- CommitOwnerProvider.registerBuilder(NoBackfillingCommitOwnerBuilder$)
- withSQLConf(MANAGED_COMMIT_OWNER_NAME.defaultTablePropertyKey -> commitOwnerName) {
+ CommitCoordinatorProvider.registerBuilder(NoBackfillingCommitCoordinatorBuilder$)
+ withSQLConf(
+ COORDINATED_COMMITS_COORDINATOR_NAME.defaultTablePropertyKey -> commitCoordinatorName) {
withTempDir { tempDir =>
val tablePath = tempDir.getAbsolutePath
Seq(1).toDF.write.format("delta").save(tablePath)
@@ -100,7 +101,8 @@ class ManagedCommitSuite
}
test("basic write") {
- CommitOwnerProvider.registerBuilder(TrackingInMemoryCommitOwnerBuilder(batchSize = 2))
+ CommitCoordinatorProvider.registerBuilder(
+ TrackingInMemoryCommitCoordinatorBuilder(batchSize = 2))
withTempDir { tempDir =>
val tablePath = tempDir.getAbsolutePath
Seq(1).toDF.write.format("delta").mode("overwrite").save(tablePath) // version 0
@@ -124,9 +126,10 @@ class ManagedCommitSuite
}
test("cold snapshot initialization") {
- val builder = TrackingInMemoryCommitOwnerBuilder(batchSize = 10)
- val commitOwnerClient = builder.build(spark, Map.empty).asInstanceOf[TrackingCommitOwnerClient]
- CommitOwnerProvider.registerBuilder(builder)
+ val builder = TrackingInMemoryCommitCoordinatorBuilder(batchSize = 10)
+ val commitCoordinatorClient =
+ builder.build(spark, Map.empty).asInstanceOf[TrackingCommitCoordinatorClient]
+ CommitCoordinatorProvider.registerBuilder(builder)
withTempDir { tempDir =>
val tablePath = tempDir.getAbsolutePath
Seq(1).toDF.write.format("delta").mode("overwrite").save(tablePath) // version 0
@@ -136,21 +139,22 @@ class ManagedCommitSuite
Seq(2).toDF.write.format("delta").mode("overwrite").save(tablePath) // version 1
Seq(3).toDF.write.format("delta").mode("append").save(tablePath) // version 2
DeltaLog.clearCache()
- commitOwnerClient.numGetCommitsCalled.set(0)
+ commitCoordinatorClient.numGetCommitsCalled.set(0)
import testImplicits._
val result1 = sql(s"SELECT * FROM delta.`$tablePath`").collect()
assert(result1.length === 2 && result1.toSet === Set(Row(2), Row(3)))
- assert(commitOwnerClient.numGetCommitsCalled.get === 2)
+ assert(commitCoordinatorClient.numGetCommitsCalled.get === 2)
}
}
- // Test commit-owner changed on concurrent cluster
- testWithDefaultCommitOwnerUnset("snapshot is updated recursively when FS table is converted" +
- " to commit-owner table on a concurrent cluster") {
- val commitOwnerClient =
- new TrackingCommitOwnerClient(new InMemoryCommitOwner(batchSize = 10))
- val builder = TrackingInMemoryCommitOwnerBuilder(batchSize = 10, Some(commitOwnerClient))
- CommitOwnerProvider.registerBuilder(builder)
+ // Test commit-coordinator changed on concurrent cluster
+ testWithDefaultCommitCoordinatorUnset("snapshot is updated recursively when FS table is" +
+ " converted to commit-coordinator table on a concurrent cluster") {
+ val commitCoordinatorClient =
+ new TrackingCommitCoordinatorClient(new InMemoryCommitCoordinator(batchSize = 10))
+ val builder =
+ TrackingInMemoryCommitCoordinatorBuilder(batchSize = 10, Some(commitCoordinatorClient))
+ CommitCoordinatorProvider.registerBuilder(builder)
withTempDir { tempDir =>
val tablePath = tempDir.getAbsolutePath
@@ -160,17 +164,17 @@ class ManagedCommitSuite
deltaLog1.startTransaction().commitManually()
val snapshotV2 = deltaLog1.update()
assert(snapshotV2.version === 2)
- assert(snapshotV2.tableCommitOwnerClientOpt.isEmpty)
+ assert(snapshotV2.tableCommitCoordinatorClientOpt.isEmpty)
DeltaLog.clearCache()
- // Add new commit to convert FS table to managed-commit table
+ // Add new commit to convert FS table to coordinated-commits table
val deltaLog2 = DeltaLog.forTable(spark, tablePath)
- enableManagedCommit(deltaLog2, commitOwner = "tracking-in-memory")
+ enableCoordinatedCommits(deltaLog2, commitCoordinator = "tracking-in-memory")
deltaLog2.startTransaction().commitManually(createTestAddFile("f2"))
deltaLog2.startTransaction().commitManually()
val snapshotV5 = deltaLog2.unsafeVolatileSnapshot
assert(snapshotV5.version === 5)
- assert(snapshotV5.tableCommitOwnerClientOpt.nonEmpty)
+ assert(snapshotV5.tableCommitCoordinatorClientOpt.nonEmpty)
// only delta 4/5 will be un-backfilled and should have two dots in filename (x.uuid.json)
assert(snapshotV5.logSegment.deltas.count(_.getPath.getName.count(_ == '.') == 2) === 2)
@@ -184,13 +188,14 @@ class ManagedCommitSuite
}
test("update works correctly with InitialSnapshot") {
- CommitOwnerProvider.registerBuilder(TrackingInMemoryCommitOwnerBuilder(batchSize = 2))
+ CommitCoordinatorProvider.registerBuilder(
+ TrackingInMemoryCommitCoordinatorBuilder(batchSize = 2))
withTempDir { tempDir =>
val tablePath = tempDir.getAbsolutePath
val clock = new ManualClock(System.currentTimeMillis())
val log = DeltaLog.forTable(spark, new Path(tablePath), clock)
assert(log.unsafeVolatileSnapshot.isInstanceOf[InitialSnapshot])
- assert(log.unsafeVolatileSnapshot.tableCommitOwnerClientOpt.isEmpty)
+ assert(log.unsafeVolatileSnapshot.tableCommitCoordinatorClientOpt.isEmpty)
assert(log.getCapturedSnapshot().updateTimestamp == clock.getTimeMillis())
clock.advance(500)
log.update()
@@ -206,7 +211,7 @@ class ManagedCommitSuite
// 3. Do cold read from table and confirm we can construct snapshot v3 automatically. This will
// need multiple snapshot update internally and both CS1 and CS2 will be contacted one
// after the other.
- // 4. Write commit 4/5 using new commit-owner.
+ // 4. Write commit 4/5 using new commit-coordinator.
// 5. Read the table again and make sure right APIs are called:
// a) If read query is run in scala, we do listing 2 times. So CS2.getCommits will be called
// twice. We should not be contacting CS1 anymore.
@@ -214,23 +219,24 @@ class ManagedCommitSuite
// only once.
test("snapshot is updated properly when owner changes multiple times") {
val batchSize = 10
- val cs1 = new TrackingCommitOwnerClient(new InMemoryCommitOwner(batchSize))
- val cs2 = new TrackingCommitOwnerClient(new InMemoryCommitOwner(batchSize))
+ val cs1 = new TrackingCommitCoordinatorClient(new InMemoryCommitCoordinator(batchSize))
+ val cs2 = new TrackingCommitCoordinatorClient(new InMemoryCommitCoordinator(batchSize))
- case class TrackingInMemoryCommitOwnerBuilder(
+ case class TrackingInMemoryCommitCoordinatorBuilder(
name: String,
- commitOwnerClient: CommitOwnerClient) extends CommitOwnerBuilder {
+ commitCoordinatorClient: CommitCoordinatorClient) extends CommitCoordinatorBuilder {
var numBuildCalled = 0
- override def build(spark: SparkSession, conf: Map[String, String]): CommitOwnerClient = {
+ override def build(
+ spark: SparkSession, conf: Map[String, String]): CommitCoordinatorClient = {
numBuildCalled += 1
- commitOwnerClient
+ commitCoordinatorClient
}
override def getName: String = name
}
- val builder1 = TrackingInMemoryCommitOwnerBuilder(name = "tracking-in-memory-1", cs1)
- val builder2 = TrackingInMemoryCommitOwnerBuilder(name = "tracking-in-memory-2", cs2)
- Seq(builder1, builder2).foreach(CommitOwnerProvider.registerBuilder)
+ val builder1 = TrackingInMemoryCommitCoordinatorBuilder(name = "tracking-in-memory-1", cs1)
+ val builder2 = TrackingInMemoryCommitCoordinatorBuilder(name = "tracking-in-memory-2", cs2)
+ Seq(builder1, builder2).foreach(CommitCoordinatorProvider.registerBuilder)
def resetMetrics(): Unit = {
Seq(builder1, builder2).foreach { b => b.numBuildCalled = 0 }
@@ -238,7 +244,7 @@ class ManagedCommitSuite
}
withSQLConf(
- MANAGED_COMMIT_OWNER_NAME.defaultTablePropertyKey -> builder1.name) {
+ COORDINATED_COMMITS_COORDINATOR_NAME.defaultTablePropertyKey -> builder1.name) {
withTempDir { tempDir =>
val tablePath = tempDir.getAbsolutePath
// Step-1: Make 2 commits on the table with CS1 as owner.
@@ -250,7 +256,7 @@ class ManagedCommitSuite
// Step-2: Add commit 2: change the table owner from "tracking-in-memory-1" to FS.
// Add commit 3: change the table owner from FS to "tracking-in-memory-2".
// Both of these commits should be FS based as the spec mandates an atomic backfill when
- // the commit-owner changes.
+ // the commit-coordinator changes.
{
val log = DeltaLog.forTable(spark, tablePath)
val conf = log.newDeltaHadoopConf()
@@ -261,13 +267,15 @@ class ManagedCommitSuite
val oldMetadata = log.unsafeVolatileMetadata
val oldMetadataConf = oldMetadata.configuration
val newMetadata1 = oldMetadata.copy(
- configuration = oldMetadataConf - MANAGED_COMMIT_OWNER_NAME.key)
+ configuration = oldMetadataConf - COORDINATED_COMMITS_COORDINATOR_NAME.key)
val newMetadata2 = oldMetadata.copy(
- configuration = oldMetadataConf + (MANAGED_COMMIT_OWNER_NAME.key -> builder2.name))
+ configuration = oldMetadataConf + (
+ COORDINATED_COMMITS_COORDINATOR_NAME.key -> builder2.name))
log.startTransaction().commitManually(newMetadata1)
log.startTransaction().commitManually(newMetadata2)
- // Also backfill commit 0, 1 -- which the spec mandates when the commit-owner changes.
+ // Also backfill commit 0, 1 -- which the spec mandates when the commit-coordinator
+ // changes.
// commit 0 should already be backfilled
assert(segment.deltas(0).getPath.getName === "00000000000000000000.json")
log.store.write(
@@ -286,7 +294,8 @@ class ManagedCommitSuite
assert(builder1.numBuildCalled == 0)
assert(builder2.numBuildCalled == 1)
val snapshotV3 = DeltaLog.forTable(spark, tablePath).unsafeVolatileSnapshot
- assert(snapshotV3.tableCommitOwnerClientOpt.map(_.commitOwnerClient) === Some(cs2))
+ assert(
+ snapshotV3.tableCommitCoordinatorClientOpt.map(_.commitCoordinatorClient) === Some(cs2))
assert(snapshotV3.version === 3)
// Step-4: Write more commits using new owner
@@ -301,7 +310,7 @@ class ManagedCommitSuite
assert(
sql(s"SELECT * FROM delta.`$tablePath`").collect().toSet === (0 to 3).map(Row(_)).toSet)
// since this was hot query, so no new snapshot was created as part of this
- // deltaLog.update() and so commit-owner is not initialized again.
+ // deltaLog.update() and so commit-coordinator is not initialized again.
assert((builder1.numBuildCalled, builder2.numBuildCalled) === (0, 0))
// Since this is dataframe read, so we invoke deltaLog.update() twice and so GetCommits API
// is called twice.
@@ -323,13 +332,13 @@ class ManagedCommitSuite
}
// This test has the following setup:
- // 1. Table is created with CS1 as commit-owner.
+ // 1. Table is created with CS1 as commit-coordinator.
// 2. Do another commit (v1) on table.
// 3. Take a reference to current DeltaLog and clear the cache. This deltaLog object currently
// points to the latest table snapshot i.e. v1.
// 4. Do commit v2 on the table.
- // 5. Do commit v3 on table. As part of this, change commit-owner to FS. Do v4 on table and change
- // owner to CS2.
+ // 5. Do commit v3 on table. As part of this, change commit-coordinator to FS. Do v4 on table and
+ // change owner to CS2.
// 6. Do commit v5 on table. This will happen via CS2.
// 7. Invoke deltaLog.update() on the old deltaLog object which is still pointing to v1.
// - While doing this, we will inject failure in CS2 so that it fails twice when cs2.getCommits
@@ -342,32 +351,33 @@ class ManagedCommitSuite
// - the recorded timestamp for this should be clock timestamp.
test("failures inside getCommits, correct timestamp is added in CapturedSnapshot") {
val batchSize = 10
- val cs1 = new TrackingCommitOwnerClient(new InMemoryCommitOwner(batchSize))
- val cs2 = new TrackingCommitOwnerClient(new InMemoryCommitOwner(batchSize)) {
+ val cs1 = new TrackingCommitCoordinatorClient(new InMemoryCommitCoordinator(batchSize))
+ val cs2 = new TrackingCommitCoordinatorClient(new InMemoryCommitCoordinator(batchSize)) {
var failAttempts = Set[Int]()
override def getCommits(
logPath: Path,
- managedCommitTableConf: Map[String, String],
+ coordinatedCommitsTableConf: Map[String, String],
startVersion: Option[Long],
endVersion: Option[Long]): GetCommitsResponse = {
if (failAttempts.contains(numGetCommitsCalled.get + 1)) {
numGetCommitsCalled.incrementAndGet()
throw new IllegalStateException("Injected failure")
}
- super.getCommits(logPath, managedCommitTableConf, startVersion, endVersion)
+ super.getCommits(logPath, coordinatedCommitsTableConf, startVersion, endVersion)
}
}
- case class TrackingInMemoryCommitOwnerClientBuilder(
+ case class TrackingInMemoryCommitCoordinatorClientBuilder(
name: String,
- commitOwnerClient: CommitOwnerClient) extends CommitOwnerBuilder {
+ commitCoordinatorClient: CommitCoordinatorClient) extends CommitCoordinatorBuilder {
override def build(
- spark: SparkSession, conf: Map[String, String]): CommitOwnerClient = commitOwnerClient
+ spark: SparkSession,
+ conf: Map[String, String]): CommitCoordinatorClient = commitCoordinatorClient
override def getName: String = name
}
- val builder1 = TrackingInMemoryCommitOwnerClientBuilder(name = "in-memory-1", cs1)
- val builder2 = TrackingInMemoryCommitOwnerClientBuilder(name = "in-memory-2", cs2)
- Seq(builder1, builder2).foreach(CommitOwnerProvider.registerBuilder)
+ val builder1 = TrackingInMemoryCommitCoordinatorClientBuilder(name = "in-memory-1", cs1)
+ val builder2 = TrackingInMemoryCommitCoordinatorClientBuilder(name = "in-memory-2", cs2)
+ Seq(builder1, builder2).foreach(CommitCoordinatorProvider.registerBuilder)
def resetMetrics(): Unit = {
cs1.reset()
@@ -375,7 +385,7 @@ class ManagedCommitSuite
cs2.failAttempts = Set()
}
- withSQLConf(MANAGED_COMMIT_OWNER_NAME.defaultTablePropertyKey -> "in-memory-1") {
+ withSQLConf(COORDINATED_COMMITS_COORDINATOR_NAME.defaultTablePropertyKey -> "in-memory-1") {
withTempDir { tempDir =>
val tablePath = tempDir.getAbsolutePath
// Step-1
@@ -400,10 +410,11 @@ class ManagedCommitSuite
assert(log.unsafeVolatileSnapshot.version === 2)
val oldMetadataConf = log.update().metadata.configuration
val newMetadata1 = oldMetadata.copy(
- configuration = oldMetadataConf - MANAGED_COMMIT_OWNER_NAME.key)
+ configuration = oldMetadataConf - COORDINATED_COMMITS_COORDINATOR_NAME.key)
val newMetadata2 = oldMetadata.copy(
- configuration = oldMetadataConf + (MANAGED_COMMIT_OWNER_NAME.key -> "in-memory-2"))
- assert(log.update().tableCommitOwnerClientOpt.get.commitOwnerClient === cs1)
+ configuration = oldMetadataConf + (
+ COORDINATED_COMMITS_COORDINATOR_NAME.key -> "in-memory-2"))
+ assert(log.update().tableCommitCoordinatorClientOpt.get.commitCoordinatorClient === cs1)
log.startTransaction().commitManually(newMetadata1) // version 3
(1 to 3).foreach { v =>
// backfill commit 1 and 2 also as 3/4 are written directly to FS.
@@ -413,9 +424,9 @@ class ManagedCommitSuite
actions = log.store.read(segment.deltas(v).getPath).toIterator,
overwrite = true)
}
- assert(log.update().tableCommitOwnerClientOpt === None)
+ assert(log.update().tableCommitCoordinatorClientOpt === None)
log.startTransaction().commitManually(newMetadata2) // version 4
- assert(log.update().tableCommitOwnerClientOpt.get.commitOwnerClient === cs2)
+ assert(log.update().tableCommitCoordinatorClientOpt.get.commitCoordinatorClient === cs2)
// Step-6
Seq(4).toDF.write.format("delta").mode("append").save(tablePath) // version 5
@@ -429,7 +440,7 @@ class ManagedCommitSuite
clock.setTime(System.currentTimeMillis())
resetMetrics()
cs2.failAttempts = Set(1, 2) // fail 0th and 1st attempt, 2nd attempt will succeed.
- val ex1 = intercept[CommitOwnerGetCommitsFailedException] { oldDeltaLog.update() }
+ val ex1 = intercept[CommitCoordinatorGetCommitsFailedException] { oldDeltaLog.update() }
assert((cs1.numGetCommitsCalled.get, cs2.numGetCommitsCalled.get) === (1, 1))
assert(ex1.getMessage.contains("Injected failure"))
assert(oldDeltaLog.unsafeVolatileSnapshot.version == 1)
@@ -437,7 +448,7 @@ class ManagedCommitSuite
// Attempt-2
// 2nd update also fails
- val ex2 = intercept[CommitOwnerGetCommitsFailedException] { oldDeltaLog.update() }
+ val ex2 = intercept[CommitCoordinatorGetCommitsFailedException] { oldDeltaLog.update() }
assert((cs1.numGetCommitsCalled.get, cs2.numGetCommitsCalled.get) === (2, 2))
assert(ex2.getMessage.contains("Injected failure"))
assert(oldDeltaLog.unsafeVolatileSnapshot.version == 1)
@@ -525,11 +536,12 @@ class ManagedCommitSuite
}
}
- testWithDefaultCommitOwnerUnset("DeltaLog.getSnapshotAt") {
- val commitOwnerClient =
- new TrackingCommitOwnerClient(new InMemoryCommitOwner(batchSize = 10))
- val builder = TrackingInMemoryCommitOwnerBuilder(batchSize = 10, Some(commitOwnerClient))
- CommitOwnerProvider.registerBuilder(builder)
+ testWithDefaultCommitCoordinatorUnset("DeltaLog.getSnapshotAt") {
+ val commitCoordinatorClient =
+ new TrackingCommitCoordinatorClient(new InMemoryCommitCoordinator(batchSize = 10))
+ val builder =
+ TrackingInMemoryCommitCoordinatorBuilder(batchSize = 10, Some(commitCoordinatorClient))
+ CommitCoordinatorProvider.registerBuilder(builder)
def checkGetSnapshotAt(
deltaLog: DeltaLog,
version: Long,
@@ -559,7 +571,7 @@ class ManagedCommitSuite
withTempDir { dir =>
val tablePath = dir.getAbsolutePath
- // Part-1: Validate getSnapshotAt API works as expected for non-managed commit tables
+ // Part-1: Validate getSnapshotAt API works as expected for non-coordinated commits tables
// commit 0, 1, 2 on FS table
Seq(1).toDF.write.format("delta").mode("overwrite").save(tablePath) // v0
Seq(1).toDF.write.format("delta").mode("overwrite").save(tablePath) // v1
@@ -579,10 +591,10 @@ class ManagedCommitSuite
Seq(deltaLog1, deltaLog2).foreach { log => assert(log.unsafeVolatileSnapshot.version === 2) }
DeltaLog.clearCache()
- // Part-2: Validate getSnapshotAt API works as expected for managed commit tables when the
- // switch is made
+ // Part-2: Validate getSnapshotAt API works as expected for coordinated commits tables when
+ // the switch is made
// commit 3
- enableManagedCommit(DeltaLog.forTable(spark, tablePath), "tracking-in-memory")
+ enableCoordinatedCommits(DeltaLog.forTable(spark, tablePath), "tracking-in-memory")
// commit 4
Seq(1).toDF.write.format("delta").mode("overwrite").save(tablePath)
// the old deltaLog objects still points to version 2
@@ -594,7 +606,7 @@ class ManagedCommitSuite
checkGetSnapshotAt(deltaLog1, version = 2, expectedUpdateCount = 0, expectedListingCount = 0)
// We are asking for v3 although the deltaLog1.unsafeVolatileSnapshot is for v2. So this will
// need deltaLog.update() to get the latest snapshot first - this update itself internally
- // will do 2 round of listing as we are discovering a commit-owner after first round of
+ // will do 2 round of listing as we are discovering a commit-coordinator after first round of
// listing. Once the update finishes, deltaLog1 will point to v4. So we need another round of
// listing to get just v3.
checkGetSnapshotAt(deltaLog1, version = 3, expectedUpdateCount = 1, expectedListingCount = 3)
@@ -606,11 +618,11 @@ class ManagedCommitSuite
checkGetSnapshotAt(deltaLog1, version = 4, expectedUpdateCount = 0, expectedListingCount = 0)
// We are asking for v3 although the deltaLog2.unsafeVolatileSnapshot is for v2. So this will
// need deltaLog.update() to get the latest snapshot first - this update itself internally
- // will do 2 round of listing as we are discovering a commit-owner after first round of
+ // will do 2 round of listing as we are discovering a commit-coordinator after first round of
// listing. Once the update finishes, deltaLog2 will point to v4. It can be returned directly.
checkGetSnapshotAt(deltaLog2, version = 4, expectedUpdateCount = 1, expectedListingCount = 2)
- // Part-2: Validate getSnapshotAt API works as expected for managed commit tables
+ // Part-2: Validate getSnapshotAt API works as expected for coordinated commits tables
Seq(1).toDF.write.format("delta").mode("overwrite").save(tablePath) // v5
deltaLog2 = DeltaLog.forTable(spark, tablePath)
DeltaLog.clearCache()
@@ -625,18 +637,19 @@ class ManagedCommitSuite
}
}
- private def enableManagedCommit(deltaLog: DeltaLog, commitOwner: String): Unit = {
+ private def enableCoordinatedCommits(deltaLog: DeltaLog, commitCoordinator: String): Unit = {
val oldMetadata = deltaLog.update().metadata
- val commitOwnerConf = (MANAGED_COMMIT_OWNER_NAME.key -> commitOwner)
- val newMetadata = oldMetadata.copy(configuration = oldMetadata.configuration + commitOwnerConf)
+ val commitCoordinatorConf = (COORDINATED_COMMITS_COORDINATOR_NAME.key -> commitCoordinator)
+ val newMetadata =
+ oldMetadata.copy(configuration = oldMetadata.configuration + commitCoordinatorConf)
deltaLog.startTransaction().commitManually(newMetadata)
}
test("tableConf returned from registration API is recorded in deltaLog and passed " +
- "to CommitOwnerClient in future for all the APIs") {
+ "to CommitCoordinatorClient in future for all the APIs") {
val tableConf = Map("tableID" -> "random-u-u-i-d", "1" -> "2")
- val trackingCommitOwnerClient = new TrackingCommitOwnerClient(
- new InMemoryCommitOwner(batchSize = 10) {
+ val trackingCommitCoordinatorClient = new TrackingCommitCoordinatorClient(
+ new InMemoryCommitCoordinator(batchSize = 10) {
override def registerTable(
logPath: Path,
currentVersion: Long,
@@ -648,23 +661,23 @@ class ManagedCommitSuite
override def getCommits(
logPath: Path,
- managedCommitTableConf: Map[String, String],
+ coordinatedCommitsTableConf: Map[String, String],
startVersion: Option[Long],
endVersion: Option[Long]): GetCommitsResponse = {
- assert(managedCommitTableConf === tableConf)
- super.getCommits(logPath, managedCommitTableConf, startVersion, endVersion)
+ assert(coordinatedCommitsTableConf === tableConf)
+ super.getCommits(logPath, coordinatedCommitsTableConf, startVersion, endVersion)
}
override def commit(
logStore: LogStore,
hadoopConf: Configuration,
logPath: Path,
- managedCommitTableConf: Map[String, String],
+ coordinatedCommitsTableConf: Map[String, String],
commitVersion: Long,
actions: Iterator[String],
updatedActions: UpdatedActions): CommitResponse = {
- assert(managedCommitTableConf === tableConf)
- super.commit(logStore, hadoopConf, logPath, managedCommitTableConf,
+ assert(coordinatedCommitsTableConf === tableConf)
+ super.commit(logStore, hadoopConf, logPath, coordinatedCommitsTableConf,
commitVersion, actions, updatedActions)
}
@@ -672,56 +685,60 @@ class ManagedCommitSuite
logStore: LogStore,
hadoopConf: Configuration,
logPath: Path,
- managedCommitTableConf: Map[String, String],
+ coordinatedCommitsTableConf: Map[String, String],
version: Long,
lastKnownBackfilledVersionOpt: Option[Long]): Unit = {
- assert(managedCommitTableConf === tableConf)
+ assert(coordinatedCommitsTableConf === tableConf)
super.backfillToVersion(
logStore,
hadoopConf,
logPath,
- managedCommitTableConf,
+ coordinatedCommitsTableConf,
version,
lastKnownBackfilledVersionOpt)
}
}
)
- val builder =
- TrackingInMemoryCommitOwnerBuilder(batchSize = 10, Some(trackingCommitOwnerClient))
- CommitOwnerProvider.registerBuilder(builder)
+ val builder = TrackingInMemoryCommitCoordinatorBuilder(
+ batchSize = 10, Some(trackingCommitCoordinatorClient))
+ CommitCoordinatorProvider.registerBuilder(builder)
withTempDir { tempDir =>
val tablePath = tempDir.getAbsolutePath
val log = DeltaLog.forTable(spark, tablePath)
- val commitOwnerConf = Map(MANAGED_COMMIT_OWNER_NAME.key -> builder.getName)
- val newMetadata = Metadata().copy(configuration = commitOwnerConf)
+ val commitCoordinatorConf = Map(COORDINATED_COMMITS_COORDINATOR_NAME.key -> builder.getName)
+ val newMetadata = Metadata().copy(configuration = commitCoordinatorConf)
log.startTransaction().commitManually(newMetadata)
assert(log.unsafeVolatileSnapshot.version === 0)
- assert(log.unsafeVolatileSnapshot.metadata.managedCommitTableConf === tableConf)
+ assert(log.unsafeVolatileSnapshot.metadata.coordinatedCommitsTableConf === tableConf)
log.startTransaction().commitManually(createTestAddFile("f1"))
log.startTransaction().commitManually(createTestAddFile("f2"))
log.checkpoint()
log.startTransaction().commitManually(createTestAddFile("f2"))
- assert(trackingCommitOwnerClient.numCommitsCalled.get > 0)
- assert(trackingCommitOwnerClient.numGetCommitsCalled.get > 0)
- assert(trackingCommitOwnerClient.numBackfillToVersionCalled.get > 0)
+ assert(trackingCommitCoordinatorClient.numCommitsCalled.get > 0)
+ assert(trackingCommitCoordinatorClient.numGetCommitsCalled.get > 0)
+ assert(trackingCommitCoordinatorClient.numBackfillToVersionCalled.get > 0)
}
}
for (upgradeExistingTable <- BOOLEAN_DOMAIN)
testWithDifferentBackfillInterval("upgrade + downgrade [FS -> MC1 -> FS -> MC2]," +
s" upgradeExistingTable = $upgradeExistingTable") { backfillInterval =>
- withoutManagedCommitDefaultTableProperties {
- CommitOwnerProvider.clearNonDefaultBuilders()
- val builder1 = TrackingInMemoryCommitOwnerBuilder(batchSize = backfillInterval)
- val builder2 = new TrackingInMemoryCommitOwnerBuilder(batchSize = backfillInterval) {
+ withoutCoordinatedCommitsDefaultTableProperties {
+ CommitCoordinatorProvider.clearNonDefaultBuilders()
+ val builder1 = TrackingInMemoryCommitCoordinatorBuilder(batchSize = backfillInterval)
+ val builder2 = new TrackingInMemoryCommitCoordinatorBuilder(batchSize = backfillInterval) {
override def getName: String = "tracking-in-memory-2"
}
- Seq(builder1, builder2).foreach(CommitOwnerProvider.registerBuilder(_))
- val cs1 = builder1.trackingInMemoryCommitOwnerClient.asInstanceOf[TrackingCommitOwnerClient]
- val cs2 = builder2.trackingInMemoryCommitOwnerClient.asInstanceOf[TrackingCommitOwnerClient]
+ Seq(builder1, builder2).foreach(CommitCoordinatorProvider.registerBuilder(_))
+ val cs1 = builder1
+ .trackingInMemoryCommitCoordinatorClient
+ .asInstanceOf[TrackingCommitCoordinatorClient]
+ val cs2 = builder2
+ .trackingInMemoryCommitCoordinatorClient
+ .asInstanceOf[TrackingCommitCoordinatorClient]
withTempDir { tempDir =>
val tablePath = tempDir.getAbsolutePath
@@ -729,43 +746,45 @@ class ManagedCommitSuite
val fs = log.logPath.getFileSystem(log.newDeltaHadoopConf())
var upgradeStartVersion = 0L
- // Create a non-managed commit table if we are testing upgrade for existing tables
+ // Create a non-coordinated commits table if we are testing upgrade for existing tables
if (upgradeExistingTable) {
log.startTransaction().commitManually(Metadata())
assert(log.unsafeVolatileSnapshot.version === 0)
log.startTransaction().commitManually(createTestAddFile("1"))
assert(log.unsafeVolatileSnapshot.version === 1)
- assert(log.unsafeVolatileSnapshot.tableCommitOwnerClientOpt.isEmpty)
+ assert(log.unsafeVolatileSnapshot.tableCommitCoordinatorClientOpt.isEmpty)
upgradeStartVersion = 2L
}
// Upgrade the table
// [upgradeExistingTable = false] Commit-0
// [upgradeExistingTable = true] Commit-2
- val commitOwnerConf = Map(MANAGED_COMMIT_OWNER_NAME.key -> builder1.getName)
- val newMetadata = Metadata().copy(configuration = commitOwnerConf)
+ val commitCoordinatorConf =
+ Map(COORDINATED_COMMITS_COORDINATOR_NAME.key -> builder1.getName)
+ val newMetadata = Metadata().copy(configuration = commitCoordinatorConf)
log.startTransaction().commitManually(newMetadata)
assert(log.unsafeVolatileSnapshot.version === upgradeStartVersion)
- assert(log.unsafeVolatileSnapshot.metadata.managedCommitOwnerName ===
+ assert(log.unsafeVolatileSnapshot.metadata.coordinatedCommitsCoordinatorName ===
Some(builder1.getName))
- assert(log.unsafeVolatileSnapshot.tableCommitOwnerClientOpt.nonEmpty)
- assert(log.unsafeVolatileSnapshot.metadata.managedCommitTableConf === Map.empty)
+ assert(log.unsafeVolatileSnapshot.tableCommitCoordinatorClientOpt.nonEmpty)
+ assert(log.unsafeVolatileSnapshot.metadata.coordinatedCommitsTableConf === Map.empty)
// upgrade commit always filesystem based
assert(fs.exists(FileNames.unsafeDeltaFile(log.logPath, upgradeStartVersion)))
assert(Seq(cs1, cs2).map(_.numCommitsCalled.get) == Seq(0, 0))
assert(Seq(cs1, cs2).map(_.numRegisterTableCalled.get) == Seq(1, 0))
- // Do couple of commits on the managed-commit table
+ // Do couple of commits on the coordinated-commits table
// [upgradeExistingTable = false] Commit-1/2
// [upgradeExistingTable = true] Commit-3/4
(1 to 2).foreach { versionOffset =>
val version = upgradeStartVersion + versionOffset
log.startTransaction().commitManually(createTestAddFile(s"$versionOffset"))
assert(log.unsafeVolatileSnapshot.version === version)
- assert(log.unsafeVolatileSnapshot.tableCommitOwnerClientOpt.nonEmpty)
- assert(log.unsafeVolatileSnapshot.metadata.managedCommitOwnerName.nonEmpty)
- assert(log.unsafeVolatileSnapshot.metadata.managedCommitOwnerConf === Map.empty)
- assert(log.unsafeVolatileSnapshot.metadata.managedCommitTableConf === Map.empty)
+ assert(log.unsafeVolatileSnapshot.tableCommitCoordinatorClientOpt.nonEmpty)
+ assert(log.unsafeVolatileSnapshot.metadata.coordinatedCommitsCoordinatorName.nonEmpty)
+ assert(
+ log.unsafeVolatileSnapshot.metadata.coordinatedCommitsCoordinatorConf === Map.empty)
+ assert(log.unsafeVolatileSnapshot.metadata.coordinatedCommitsTableConf === Map.empty)
assert(cs1.numCommitsCalled.get === versionOffset)
val backfillExpected = if (version % backfillInterval == 0) true else false
assert(fs.exists(FileNames.unsafeDeltaFile(log.logPath, version)) == backfillExpected)
@@ -774,22 +793,22 @@ class ManagedCommitSuite
// Downgrade the table
// [upgradeExistingTable = false] Commit-3
// [upgradeExistingTable = true] Commit-5
- val commitOwnerConfKeys = Seq(
- MANAGED_COMMIT_OWNER_NAME.key,
- MANAGED_COMMIT_OWNER_CONF.key,
- MANAGED_COMMIT_TABLE_CONF.key
+ val commitCoordinatorConfKeys = Seq(
+ COORDINATED_COMMITS_COORDINATOR_NAME.key,
+ COORDINATED_COMMITS_COORDINATOR_CONF.key,
+ COORDINATED_COMMITS_TABLE_CONF.key
)
val newConfig = log.snapshot.metadata.configuration
- .filterKeys(!commitOwnerConfKeys.contains(_)) ++ Map("downgraded_at" -> "v2")
+ .filterKeys(!commitCoordinatorConfKeys.contains(_)) ++ Map("downgraded_at" -> "v2")
val newMetadata2 = log.snapshot.metadata.copy(configuration = newConfig.toMap)
log.startTransaction().commitManually(newMetadata2)
assert(log.unsafeVolatileSnapshot.version === upgradeStartVersion + 3)
- assert(log.unsafeVolatileSnapshot.tableCommitOwnerClientOpt.isEmpty)
- assert(log.unsafeVolatileSnapshot.metadata.managedCommitOwnerName.isEmpty)
- assert(log.unsafeVolatileSnapshot.metadata.managedCommitOwnerConf === Map.empty)
- assert(log.unsafeVolatileSnapshot.metadata.managedCommitTableConf === Map.empty)
+ assert(log.unsafeVolatileSnapshot.tableCommitCoordinatorClientOpt.isEmpty)
+ assert(log.unsafeVolatileSnapshot.metadata.coordinatedCommitsCoordinatorName.isEmpty)
+ assert(log.unsafeVolatileSnapshot.metadata.coordinatedCommitsCoordinatorConf === Map.empty)
+ assert(log.unsafeVolatileSnapshot.metadata.coordinatedCommitsTableConf === Map.empty)
assert(log.unsafeVolatileSnapshot.metadata === newMetadata2)
- // This must have increased by 1 as downgrade commit happens via CommitOwnerClient.
+ // This must have increased by 1 as downgrade commit happens via CommitCoordinatorClient.
assert(Seq(cs1, cs2).map(_.numCommitsCalled.get) == Seq(3, 0))
assert(Seq(cs1, cs2).map(_.numRegisterTableCalled.get) == Seq(1, 0))
(0 to 3).foreach { version =>
@@ -801,36 +820,37 @@ class ManagedCommitSuite
// [upgradeExistingTable = true] Commit-6
log.startTransaction().commitManually(createTestAddFile("post-upgrade-file"))
assert(log.unsafeVolatileSnapshot.version === upgradeStartVersion + 4)
- // no commit-owner after downgrade
- assert(log.unsafeVolatileSnapshot.tableCommitOwnerClientOpt.isEmpty)
- assert(log.unsafeVolatileSnapshot.metadata.managedCommitTableConf === Map.empty)
+ // no commit-coordinator after downgrade
+ assert(log.unsafeVolatileSnapshot.tableCommitCoordinatorClientOpt.isEmpty)
+ assert(log.unsafeVolatileSnapshot.metadata.coordinatedCommitsTableConf === Map.empty)
// Metadata is same as what we added at time of downgrade
assert(log.unsafeVolatileSnapshot.metadata === newMetadata2)
// State reconstruction should give correct results
var expectedFileNames = Set("1", "2", "post-upgrade-file")
assert(log.unsafeVolatileSnapshot.allFiles.collect().toSet ===
expectedFileNames.map(name => createTestAddFile(name, dataChange = false)))
- // commit-owner should not be invoked for commit API.
+ // commit-coordinator should not be invoked for commit API.
// Register table API should not be called until the end
assert(Seq(cs1, cs2).map(_.numCommitsCalled.get) == Seq(3, 0))
assert(Seq(cs1, cs2).map(_.numRegisterTableCalled.get) == Seq(1, 0))
// 4th file is directly written to FS in backfilled way.
assert(fs.exists(FileNames.unsafeDeltaFile(log.logPath, upgradeStartVersion + 4)))
- // Now transfer the table to another commit-owner
+ // Now transfer the table to another commit-coordinator
// [upgradeExistingTable = false] Commit-5
// [upgradeExistingTable = true] Commit-7
- val commitOwnerConf2 = Map(MANAGED_COMMIT_OWNER_NAME.key -> builder2.getName)
+ val commitCoordinatorConf2 =
+ Map(COORDINATED_COMMITS_COORDINATOR_NAME.key -> builder2.getName)
val oldMetadata3 = log.unsafeVolatileSnapshot.metadata
val newMetadata3 = oldMetadata3.copy(
- configuration = oldMetadata3.configuration ++ commitOwnerConf2)
+ configuration = oldMetadata3.configuration ++ commitCoordinatorConf2)
log.startTransaction().commitManually(newMetadata3, createTestAddFile("upgrade-2-file"))
assert(log.unsafeVolatileSnapshot.version === upgradeStartVersion + 5)
- assert(log.unsafeVolatileSnapshot.tableCommitOwnerClientOpt.nonEmpty)
- assert(log.unsafeVolatileSnapshot.metadata.managedCommitOwnerName ===
+ assert(log.unsafeVolatileSnapshot.tableCommitCoordinatorClientOpt.nonEmpty)
+ assert(log.unsafeVolatileSnapshot.metadata.coordinatedCommitsCoordinatorName ===
Some(builder2.getName))
- assert(log.unsafeVolatileSnapshot.metadata.managedCommitOwnerConf === Map.empty)
- assert(log.unsafeVolatileSnapshot.metadata.managedCommitTableConf === Map.empty)
+ assert(log.unsafeVolatileSnapshot.metadata.coordinatedCommitsCoordinatorConf === Map.empty)
+ assert(log.unsafeVolatileSnapshot.metadata.coordinatedCommitsTableConf === Map.empty)
expectedFileNames = Set("1", "2", "post-upgrade-file", "upgrade-2-file")
assert(log.unsafeVolatileSnapshot.allFiles.collect().toSet ===
expectedFileNames.map(name => createTestAddFile(name, dataChange = false)))
@@ -849,44 +869,48 @@ class ManagedCommitSuite
}
}
- test("transfer from one commit-owner to another commit-owner fails [MC-1 -> MC-2 fails]") {
- CommitOwnerProvider.clearNonDefaultBuilders()
- val builder1 = TrackingInMemoryCommitOwnerBuilder(batchSize = 10)
- val builder2 = new TrackingInMemoryCommitOwnerBuilder(batchSize = 10) {
+ test("transfer from one commit-coordinator to another commit-coordinator fails " +
+ "[MC-1 -> MC-2 fails]") {
+ CommitCoordinatorProvider.clearNonDefaultBuilders()
+ val builder1 = TrackingInMemoryCommitCoordinatorBuilder(batchSize = 10)
+ val builder2 = new TrackingInMemoryCommitCoordinatorBuilder(batchSize = 10) {
override def getName: String = "tracking-in-memory-2"
}
- Seq(builder1, builder2).foreach(CommitOwnerProvider.registerBuilder(_))
+ Seq(builder1, builder2).foreach(CommitCoordinatorProvider.registerBuilder(_))
withTempDir { tempDir =>
val tablePath = tempDir.getAbsolutePath
val log = DeltaLog.forTable(spark, tablePath)
// A new table will automatically get `tracking-in-memory` as the whole suite is configured to
- // use it as default commit-owner via [[MANAGED_COMMIT_OWNER_NAME.defaultTablePropertyKey]].
+ // use it as default commit-coordinator via
+ // [[COORDINATED_COMMITS_COORDINATOR_NAME.defaultTablePropertyKey]].
log.startTransaction().commitManually(Metadata())
assert(log.unsafeVolatileSnapshot.version === 0L)
- assert(log.unsafeVolatileSnapshot.tableCommitOwnerClientOpt.nonEmpty)
+ assert(log.unsafeVolatileSnapshot.tableCommitCoordinatorClientOpt.nonEmpty)
- // Change commit-owner
- val newCommitOwnerConf = Map(MANAGED_COMMIT_OWNER_NAME.key -> builder2.getName)
+ // Change commit-coordinator
+ val newCommitCoordinatorConf =
+ Map(COORDINATED_COMMITS_COORDINATOR_NAME.key -> builder2.getName)
val oldMetadata = log.unsafeVolatileSnapshot.metadata
val newMetadata = oldMetadata.copy(
- configuration = oldMetadata.configuration ++ newCommitOwnerConf)
+ configuration = oldMetadata.configuration ++ newCommitCoordinatorConf)
val ex = intercept[IllegalStateException] {
log.startTransaction().commitManually(newMetadata)
}
- assert(ex.getMessage.contains("from one commit-owner to another commit-owner is not allowed"))
+ assert(ex.getMessage.contains("from one commit-coordinator to another commit-coordinator " +
+ "is not allowed"))
}
}
- testWithDefaultCommitOwnerUnset("FS -> MC upgrade is not retried on a conflict") {
- val builder = TrackingInMemoryCommitOwnerBuilder(batchSize = 10)
- CommitOwnerProvider.registerBuilder(builder)
+ testWithDefaultCommitCoordinatorUnset("FS -> MC upgrade is not retried on a conflict") {
+ val builder = TrackingInMemoryCommitCoordinatorBuilder(batchSize = 10)
+ CommitCoordinatorProvider.registerBuilder(builder)
withTempDir { tempDir =>
val tablePath = tempDir.getAbsolutePath
val log = DeltaLog.forTable(spark, tablePath)
- val commitOwnerConf = Map(MANAGED_COMMIT_OWNER_NAME.key -> builder.getName)
- val newMetadata = Metadata().copy(configuration = commitOwnerConf)
+ val commitCoordinatorConf = Map(COORDINATED_COMMITS_COORDINATOR_NAME.key -> builder.getName)
+ val newMetadata = Metadata().copy(configuration = commitCoordinatorConf)
val txn = log.startTransaction() // upgrade txn started
log.startTransaction().commitManually(createTestAddFile("f1"))
intercept[io.delta.exceptions.ConcurrentWriteException] {
@@ -895,22 +919,23 @@ class ManagedCommitSuite
}
}
- testWithDefaultCommitOwnerUnset("FS -> MC upgrade with commitLarge API") {
- val builder = TrackingInMemoryCommitOwnerBuilder(batchSize = 10)
- val cs = builder.trackingInMemoryCommitOwnerClient.asInstanceOf[TrackingCommitOwnerClient]
- CommitOwnerProvider.registerBuilder(builder)
+ testWithDefaultCommitCoordinatorUnset("FS -> MC upgrade with commitLarge API") {
+ val builder = TrackingInMemoryCommitCoordinatorBuilder(batchSize = 10)
+ val cs =
+ builder.trackingInMemoryCommitCoordinatorClient.asInstanceOf[TrackingCommitCoordinatorClient]
+ CommitCoordinatorProvider.registerBuilder(builder)
withTempDir { tempDir =>
val tablePath = tempDir.getAbsolutePath
Seq(1).toDF.write.format("delta").save(tablePath)
Seq(1).toDF.write.mode("overwrite").format("delta").save(tablePath)
var log = DeltaLog.forTable(spark, tablePath)
assert(log.unsafeVolatileSnapshot.version === 1L)
- assert(log.unsafeVolatileSnapshot.tableCommitOwnerClientOpt.isEmpty)
+ assert(log.unsafeVolatileSnapshot.tableCommitCoordinatorClientOpt.isEmpty)
- val commitOwnerConf = Map(MANAGED_COMMIT_OWNER_NAME.key -> builder.getName)
+ val commitCoordinatorConf = Map(COORDINATED_COMMITS_COORDINATOR_NAME.key -> builder.getName)
val oldMetadata = log.unsafeVolatileSnapshot.metadata
val newMetadata = oldMetadata.copy(
- configuration = oldMetadata.configuration ++ commitOwnerConf)
+ configuration = oldMetadata.configuration ++ commitCoordinatorConf)
val oldProtocol = log.unsafeVolatileSnapshot.protocol
assert(!oldProtocol.readerAndWriterFeatures.contains(V2CheckpointTableFeature))
val newProtocol =
@@ -920,7 +945,9 @@ class ManagedCommitSuite
readerFeatures =
Some(oldProtocol.readerFeatures.getOrElse(Set.empty) + V2CheckpointTableFeature.name),
writerFeatures =
- Some(oldProtocol.writerFeatures.getOrElse(Set.empty) + ManagedCommitTableFeature.name))
+ Some(
+ oldProtocol.writerFeatures.getOrElse(Set.empty) + CoordinatedCommitsTableFeature.name)
+ )
assert(cs.numRegisterTableCalled.get === 0)
assert(cs.numCommitsCalled.get === 0)
@@ -938,40 +965,41 @@ class ManagedCommitSuite
assert(cs.numCommitsCalled.get === 0)
assert(log.unsafeVolatileSnapshot.version === 2L)
- Seq(V2CheckpointTableFeature, ManagedCommitTableFeature).foreach { feature =>
+ Seq(V2CheckpointTableFeature, CoordinatedCommitsTableFeature).foreach { feature =>
assert(log.unsafeVolatileSnapshot.protocol.isFeatureSupported(feature))
}
- assert(log.unsafeVolatileSnapshot.tableCommitOwnerClientOpt.nonEmpty)
- assert(log.unsafeVolatileSnapshot.metadata.managedCommitOwnerName === Some(builder.getName))
- assert(log.unsafeVolatileSnapshot.metadata.managedCommitOwnerConf === Map.empty)
- assert(log.unsafeVolatileSnapshot.metadata.managedCommitTableConf === Map.empty)
+ assert(log.unsafeVolatileSnapshot.tableCommitCoordinatorClientOpt.nonEmpty)
+ assert(log.unsafeVolatileSnapshot.metadata.coordinatedCommitsCoordinatorName ===
+ Some(builder.getName))
+ assert(log.unsafeVolatileSnapshot.metadata.coordinatedCommitsCoordinatorConf === Map.empty)
+ assert(log.unsafeVolatileSnapshot.metadata.coordinatedCommitsTableConf === Map.empty)
Seq(3).toDF.write.mode("append").format("delta").save(tablePath)
assert(cs.numRegisterTableCalled.get === 1)
assert(cs.numCommitsCalled.get === 1)
assert(log.unsafeVolatileSnapshot.version === 3L)
- assert(log.unsafeVolatileSnapshot.tableCommitOwnerClientOpt.nonEmpty)
+ assert(log.unsafeVolatileSnapshot.tableCommitCoordinatorClientOpt.nonEmpty)
}
}
test("Incomplete backfills are handled properly by next commit after MC to FS conversion") {
val batchSize = 10
- val neverBackfillingCommitOwner =
- new TrackingCommitOwnerClient(new InMemoryCommitOwner(batchSize) {
+ val neverBackfillingCommitCoordinator =
+ new TrackingCommitCoordinatorClient(new InMemoryCommitCoordinator(batchSize) {
override def backfillToVersion(
logStore: LogStore,
hadoopConf: Configuration,
logPath: Path,
- managedCommitTableConf: Map[String, String],
+ coordinatedCommitsTableConf: Map[String, String],
version: Long,
lastKnownBackfilledVersionOpt: Option[Long]): Unit = { }
})
- CommitOwnerProvider.clearNonDefaultBuilders()
+ CommitCoordinatorProvider.clearNonDefaultBuilders()
val builder =
- TrackingInMemoryCommitOwnerBuilder(batchSize, Some(neverBackfillingCommitOwner))
- CommitOwnerProvider.registerBuilder(builder)
+ TrackingInMemoryCommitCoordinatorBuilder(batchSize, Some(neverBackfillingCommitCoordinator))
+ CommitCoordinatorProvider.registerBuilder(builder)
withTempDir { tempDir =>
val tablePath = tempDir.getAbsolutePath
@@ -980,19 +1008,19 @@ class ManagedCommitSuite
Seq(1).toDF.write.format("delta").mode("overwrite").save(tablePath) // v2
val log = DeltaLog.forTable(spark, tablePath)
- assert(log.unsafeVolatileSnapshot.tableCommitOwnerClientOpt.nonEmpty)
+ assert(log.unsafeVolatileSnapshot.tableCommitCoordinatorClientOpt.nonEmpty)
assert(log.unsafeVolatileSnapshot.version === 2)
assert(
log.unsafeVolatileSnapshot.logSegment.deltas.count(FileNames.isUnbackfilledDeltaFile) == 2)
val oldMetadata = log.unsafeVolatileSnapshot.metadata
val downgradeMetadata = oldMetadata.copy(
- configuration = oldMetadata.configuration - MANAGED_COMMIT_OWNER_NAME.key)
+ configuration = oldMetadata.configuration - COORDINATED_COMMITS_COORDINATOR_NAME.key)
log.startTransaction().commitManually(downgradeMetadata)
log.update()
val snapshotAfterDowngrade = log.unsafeVolatileSnapshot
assert(snapshotAfterDowngrade.version === 3)
- assert(snapshotAfterDowngrade.tableCommitOwnerClientOpt.isEmpty)
+ assert(snapshotAfterDowngrade.tableCommitCoordinatorClientOpt.isEmpty)
assert(snapshotAfterDowngrade.logSegment.deltas.count(FileNames.isUnbackfilledDeltaFile) == 3)
val records = Log4jUsageLogger.track {
@@ -1000,7 +1028,7 @@ class ManagedCommitSuite
Seq(1).toDF.write.format("delta").mode("overwrite").save(tablePath)
}
val filteredUsageLogs = filterUsageRecords(
- records, "delta.managedCommit.backfillWhenManagedCommitSupportedAndDisabled")
+ records, "delta.coordinatedCommits.backfillWhenCoordinatedCommitsSupportedAndDisabled")
assert(filteredUsageLogs.size === 1)
val usageObj = JsonUtils.fromJson[Map[String, Any]](filteredUsageLogs.head.blob)
assert(usageObj("numUnbackfilledFiles").asInstanceOf[Int] === 3)
@@ -1009,13 +1037,13 @@ class ManagedCommitSuite
}
/////////////////////////////////////////////////////////////////////////////////////////////
- // Test managed-commits with DeltaLog.getChangeLogFile API starts //
+ // Test coordinated-commits with DeltaLog.getChangeLogFile API starts //
/////////////////////////////////////////////////////////////////////////////////////////////
/**
* Helper method which generates a delta table with `totalCommits`.
- * The `upgradeToManagedCommitVersion`th commit version upgrades this table to managed commit
- * and it uses `backfillInterval` for backfilling.
+ * The `upgradeToCoordinatedCommitsVersion`th commit version upgrades this table to coordinated
+ * commits and it uses `backfillInterval` for backfilling.
* This method returns a mapping of version to DeltaLog for the versions in
* `requiredDeltaLogVersions`. Each of this deltaLog object has a Snapshot as per what is
* mentioned in the `requiredDeltaLogVersions`.
@@ -1023,24 +1051,26 @@ class ManagedCommitSuite
private def generateDataForGetChangeLogFilesTest(
dir: File,
totalCommits: Int,
- upgradeToManagedCommitVersion: Int,
+ upgradeToCoordinatedCommitsVersion: Int,
backfillInterval: Int,
requiredDeltaLogVersions: Set[Int]): Map[Int, DeltaLog] = {
- val commitOwnerClient = new TrackingCommitOwnerClient(new InMemoryCommitOwner(backfillInterval))
- val builder = TrackingInMemoryCommitOwnerBuilder(backfillInterval, Some(commitOwnerClient))
- CommitOwnerProvider.registerBuilder(builder)
+ val commitCoordinatorClient =
+ new TrackingCommitCoordinatorClient(new InMemoryCommitCoordinator(backfillInterval))
+ val builder =
+ TrackingInMemoryCommitCoordinatorBuilder(backfillInterval, Some(commitCoordinatorClient))
+ CommitCoordinatorProvider.registerBuilder(builder)
val versionToDeltaLogMapping = collection.mutable.Map.empty[Int, DeltaLog]
withSQLConf(
CHECKPOINT_INTERVAL.defaultTablePropertyKey -> "100") {
val tablePath = dir.getAbsolutePath
(0 to totalCommits).foreach { v =>
- if (v === upgradeToManagedCommitVersion) {
+ if (v === upgradeToCoordinatedCommitsVersion) {
val deltaLog = DeltaLog.forTable(spark, tablePath)
val oldMetadata = deltaLog.unsafeVolatileSnapshot.metadata
- val commitOwner = (MANAGED_COMMIT_OWNER_NAME.key -> "tracking-in-memory")
+ val commitCoordinator = (COORDINATED_COMMITS_COORDINATOR_NAME.key -> "tracking-in-memory")
val newMetadata =
- oldMetadata.copy(configuration = oldMetadata.configuration + commitOwner)
+ oldMetadata.copy(configuration = oldMetadata.configuration + commitCoordinator)
deltaLog.startTransaction().commitManually(newMetadata)
} else {
Seq(v).toDF().write.format("delta").mode("append").save(tablePath)
@@ -1083,13 +1113,13 @@ class ManagedCommitSuite
assert(filterUsageRecords(usageRecords, "delta.log.update").size === updateCountEvents)
}
- testWithDefaultCommitOwnerUnset("DeltaLog.getChangeLogFile with and" +
- " without endVersion [No Managed Commits]") {
+ testWithDefaultCommitCoordinatorUnset("DeltaLog.getChangeLogFile with and" +
+ " without endVersion [No Coordinated Commits]") {
withTempDir { dir =>
val versionsToDeltaLogMapping = generateDataForGetChangeLogFilesTest(
dir,
totalCommits = 4,
- upgradeToManagedCommitVersion = -1,
+ upgradeToCoordinatedCommitsVersion = -1,
backfillInterval = -1,
requiredDeltaLogVersions = Set(2, 4))
@@ -1105,8 +1135,8 @@ class ManagedCommitSuite
// We are asking for changes between 0 to `end` to a DeltaLog(unsafeVolatileSnapshot = 2).
// Since the commits in filesystem are more than what unsafeVolatileSnapshot has, we should
- // need an update() to get the latest snapshot and see if managed commit was enabled on the
- // table concurrently.
+ // need an update() to get the latest snapshot and see if coordinated commits was enabled on
+ // the table concurrently.
runGetChangeLogFiles(
versionsToDeltaLogMapping(2),
totalCommitsOnTable = 4,
@@ -1116,7 +1146,7 @@ class ManagedCommitSuite
// We are asking for changes between 0 to `end` to a DeltaLog(unsafeVolatileSnapshot = 4).
// The latest commit from filesystem listing is 4 -- same as unsafeVolatileSnapshot and this
- // unsafeVolatileSnapshot doesn't have managed commit enabled. So we should not need an
+ // unsafeVolatileSnapshot doesn't have coordinated commits enabled. So we should not need an
// update().
runGetChangeLogFiles(
versionsToDeltaLogMapping(4),
@@ -1127,13 +1157,13 @@ class ManagedCommitSuite
}
}
- testWithDefaultCommitOwnerUnset("DeltaLog.getChangeLogFile with and" +
- " without endVersion [Managed Commits backfill size 1]") {
+ testWithDefaultCommitCoordinatorUnset("DeltaLog.getChangeLogFile with and" +
+ " without endVersion [Coordinated Commits backfill size 1]") {
withTempDir { dir =>
val versionsToDeltaLogMapping = generateDataForGetChangeLogFilesTest(
dir,
totalCommits = 4,
- upgradeToManagedCommitVersion = 2,
+ upgradeToCoordinatedCommitsVersion = 2,
backfillInterval = 1,
requiredDeltaLogVersions = Set(0, 2, 4))
@@ -1149,8 +1179,8 @@ class ManagedCommitSuite
// We are asking for changes between 0 to `end` to a DeltaLog(unsafeVolatileSnapshot = 2).
// Since the commits in filesystem are more than what unsafeVolatileSnapshot has, we should
- // need an update() to get the latest snapshot and see if managed commit was enabled on the
- // table concurrently.
+ // need an update() to get the latest snapshot and see if coordinated commits was enabled on
+ // the table concurrently.
runGetChangeLogFiles(
versionsToDeltaLogMapping(2),
totalCommitsOnTable = 4,
@@ -1160,7 +1190,7 @@ class ManagedCommitSuite
// We are asking for changes between 0 to 4 to a DeltaLog(unsafeVolatileSnapshot = 4).
// Since the commits in filesystem are between 0 to 4, so we don't need to update() to get
- // the latest snapshot and see if managed commit was enabled on the table concurrently.
+ // the latest snapshot and see if coordinated commits was enabled on the table concurrently.
runGetChangeLogFiles(
versionsToDeltaLogMapping(4),
totalCommitsOnTable = 4,
@@ -1171,8 +1201,8 @@ class ManagedCommitSuite
// We are asking for changes between 0 to `end` to a DeltaLog(unsafeVolatileSnapshot = 4).
// The latest commit from filesystem listing is 4 -- same as unsafeVolatileSnapshot and this
- // unsafeVolatileSnapshot has managed commit enabled. So we should need an update() to find
- // out latest commits from Commit Owner.
+ // unsafeVolatileSnapshot has coordinated commits enabled. So we should need an update() to
+ // find out latest commits from Commit Coordinator.
runGetChangeLogFiles(
versionsToDeltaLogMapping(4),
totalCommitsOnTable = 4,
@@ -1182,13 +1212,13 @@ class ManagedCommitSuite
}
}
- testWithDefaultCommitOwnerUnset("DeltaLog.getChangeLogFile with and" +
- " without endVersion [Managed Commits backfill size 10]") {
+ testWithDefaultCommitCoordinatorUnset("DeltaLog.getChangeLogFile with and" +
+ " without endVersion [Coordinated Commits backfill size 10]") {
withTempDir { dir =>
val versionsToDeltaLogMapping = generateDataForGetChangeLogFilesTest(
dir,
totalCommits = 8,
- upgradeToManagedCommitVersion = 2,
+ upgradeToCoordinatedCommitsVersion = 2,
backfillInterval = 10,
requiredDeltaLogVersions = Set(2, 3, 4, 8))
@@ -1205,8 +1235,8 @@ class ManagedCommitSuite
}
// We are asking for changes between 0 to `end` to a DeltaLog(unsafeVolatileSnapshot = 2/4).
- // Since the unsafeVolatileSnapshot has managed-commits enabled, so we need to trigger an
- // update to find the latest commits from Commit Owner.
+ // Since the unsafeVolatileSnapshot has coordinated-commits enabled, so we need to trigger an
+ // update to find the latest commits from Commit Coordinator.
Seq(2, 3, 4).foreach { version =>
runGetChangeLogFiles(
versionsToDeltaLogMapping(version),
@@ -1233,6 +1263,6 @@ class ManagedCommitSuite
}
/////////////////////////////////////////////////////////////////////////////////////////////
- // Test managed-commits with DeltaLog.getChangeLogFile API ENDS //
+ // Test coordinated-commits with DeltaLog.getChangeLogFile API ENDS //
/////////////////////////////////////////////////////////////////////////////////////////////
}
diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/coordinatedcommits/CoordinatedCommitsTestUtils.scala b/spark/src/test/scala/org/apache/spark/sql/delta/coordinatedcommits/CoordinatedCommitsTestUtils.scala
new file mode 100644
index 00000000000..d8c592d73ab
--- /dev/null
+++ b/spark/src/test/scala/org/apache/spark/sql/delta/coordinatedcommits/CoordinatedCommitsTestUtils.scala
@@ -0,0 +1,303 @@
+/*
+ * 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.coordinatedcommits
+
+import java.util.concurrent.atomic.AtomicInteger
+
+import org.apache.spark.sql.delta.{DeltaConfigs, DeltaLog, DeltaTestUtilsBase}
+import org.apache.spark.sql.delta.DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_NAME
+import org.apache.spark.sql.delta.actions.{Action, CommitInfo, Metadata, Protocol}
+import org.apache.spark.sql.delta.storage.LogStore
+import org.apache.spark.sql.delta.util.JsonUtils
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.{SparkConf, SparkFunSuite}
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.test.SharedSparkSession
+
+trait CoordinatedCommitsTestUtils
+ extends DeltaTestUtilsBase { self: SparkFunSuite with SharedSparkSession =>
+
+ /**
+ * Runs a specific test with coordinated commits default properties unset.
+ * Any table created in this test won't have coordinated commits enabled by default.
+ */
+ def testWithDefaultCommitCoordinatorUnset(testName: String)(f: => Unit): Unit = {
+ test(testName) {
+ withoutCoordinatedCommitsDefaultTableProperties {
+ f
+ }
+ }
+ }
+
+ /**
+ * Runs the function `f` with coordinated commits default properties unset.
+ * Any table created in function `f`` won't have coordinated commits enabled by default.
+ */
+ def withoutCoordinatedCommitsDefaultTableProperties[T](f: => T): T = {
+ val commitCoordinatorKey = COORDINATED_COMMITS_COORDINATOR_NAME.defaultTablePropertyKey
+ val oldCommitCoordinatorValue = spark.conf.getOption(commitCoordinatorKey)
+ spark.conf.unset(commitCoordinatorKey)
+ try { f } finally {
+ oldCommitCoordinatorValue.foreach {
+ spark.conf.set(commitCoordinatorKey, _)
+ }
+ }
+ }
+
+ /** Run the test with different backfill batch sizes: 1, 2, 10 */
+ def testWithDifferentBackfillInterval(testName: String)(f: Int => Unit): Unit = {
+ Seq(1, 2, 10).foreach { backfillBatchSize =>
+ test(s"$testName [Backfill batch size: $backfillBatchSize]") {
+ CommitCoordinatorProvider.clearNonDefaultBuilders()
+ CommitCoordinatorProvider.registerBuilder(
+ TrackingInMemoryCommitCoordinatorBuilder(backfillBatchSize))
+ CommitCoordinatorProvider.registerBuilder(
+ InMemoryCommitCoordinatorBuilder(backfillBatchSize))
+ f(backfillBatchSize)
+ }
+ }
+ }
+
+ /**
+ * Run the test against a [[TrackingCommitCoordinatorClient]] with backfill batch size =
+ * `batchBackfillSize`
+ */
+ def testWithCoordinatedCommits(backfillBatchSize: Int)(testName: String)(f: => Unit): Unit = {
+ test(s"$testName [Backfill batch size: $backfillBatchSize]") {
+ CommitCoordinatorProvider.clearNonDefaultBuilders()
+ CommitCoordinatorProvider.registerBuilder(
+ TrackingInMemoryCommitCoordinatorBuilder(backfillBatchSize))
+ val coordinatedCommitsCoordinatorConf = Map("randomConf" -> "randomConfValue")
+ val coordinatedCommitsCoordinatorJson = JsonUtils.toJson(coordinatedCommitsCoordinatorConf)
+ withSQLConf(
+ DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_NAME.defaultTablePropertyKey ->
+ "tracking-in-memory",
+ DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_CONF.defaultTablePropertyKey ->
+ coordinatedCommitsCoordinatorJson) {
+ f
+ }
+ }
+ }
+
+ /** Run the test with:
+ * 1. Without coordinated-commits
+ * 2. With coordinated-commits with different backfill batch sizes
+ */
+ def testWithDifferentBackfillIntervalOptional(testName: String)(f: Option[Int] => Unit): Unit = {
+ test(s"$testName [Backfill batch size: None]") {
+ f(None)
+ }
+ testWithDifferentBackfillInterval(testName) { backfillBatchSize =>
+ val coordinatedCommitsCoordinatorConf = Map("randomConf" -> "randomConfValue")
+ val coordinatedCommitsCoordinatorJson = JsonUtils.toJson(coordinatedCommitsCoordinatorConf)
+ withSQLConf(
+ DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_NAME.defaultTablePropertyKey ->
+ "tracking-in-memory",
+ DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_CONF.defaultTablePropertyKey ->
+ coordinatedCommitsCoordinatorJson) {
+ f(Some(backfillBatchSize))
+ }
+ }
+ }
+
+ def getUpdatedActionsForZerothCommit(
+ commitInfo: CommitInfo,
+ oldMetadata: Metadata = Metadata()): UpdatedActions = {
+ val newMetadataConfiguration =
+ oldMetadata.configuration +
+ (DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_NAME.key -> "tracking-in-memory")
+ val newMetadata = oldMetadata.copy(configuration = newMetadataConfiguration)
+ UpdatedActions(commitInfo, newMetadata, Protocol(), oldMetadata, Protocol())
+ }
+
+ def getUpdatedActionsForNonZerothCommit(commitInfo: CommitInfo): UpdatedActions = {
+ val updatedActions = getUpdatedActionsForZerothCommit(commitInfo)
+ updatedActions.copy(oldMetadata = updatedActions.getNewMetadata)
+ }
+}
+
+case class TrackingInMemoryCommitCoordinatorBuilder(
+ batchSize: Long,
+ defaultCommitCoordinatorClientOpt: Option[CommitCoordinatorClient] = None)
+ extends CommitCoordinatorBuilder {
+ lazy val trackingInMemoryCommitCoordinatorClient =
+ defaultCommitCoordinatorClientOpt.getOrElse {
+ new TrackingCommitCoordinatorClient(
+ new PredictableUuidInMemoryCommitCoordinatorClient(batchSize))
+ }
+
+ override def getName: String = "tracking-in-memory"
+ override def build(spark: SparkSession, conf: Map[String, String]): CommitCoordinatorClient = {
+ trackingInMemoryCommitCoordinatorClient
+ }
+}
+
+class PredictableUuidInMemoryCommitCoordinatorClient(batchSize: Long)
+ extends InMemoryCommitCoordinator(batchSize) {
+
+ var nextUuidSuffix = 1L
+ override def generateUUID(): String = {
+ nextUuidSuffix += 1
+ s"uuid-${nextUuidSuffix - 1}"
+ }
+}
+
+object TrackingCommitCoordinatorClient {
+ private val insideOperation = new ThreadLocal[Boolean] {
+ override def initialValue(): Boolean = false
+ }
+}
+
+class TrackingCommitCoordinatorClient(delegatingCommitCoordinatorClient: InMemoryCommitCoordinator)
+ extends CommitCoordinatorClient {
+
+ val numCommitsCalled = new AtomicInteger(0)
+ val numGetCommitsCalled = new AtomicInteger(0)
+ val numBackfillToVersionCalled = new AtomicInteger(0)
+ val numRegisterTableCalled = new AtomicInteger(0)
+
+ def recordOperation[T](op: String)(f: => T): T = {
+ val oldInsideOperation = TrackingCommitCoordinatorClient.insideOperation.get()
+ try {
+ if (!TrackingCommitCoordinatorClient.insideOperation.get()) {
+ op match {
+ case "commit" => numCommitsCalled.incrementAndGet()
+ case "getCommits" => numGetCommitsCalled.incrementAndGet()
+ case "backfillToVersion" => numBackfillToVersionCalled.incrementAndGet()
+ case "registerTable" => numRegisterTableCalled.incrementAndGet()
+ case _ => ()
+ }
+ }
+ TrackingCommitCoordinatorClient.insideOperation.set(true)
+ f
+ } finally {
+ TrackingCommitCoordinatorClient.insideOperation.set(oldInsideOperation)
+ }
+ }
+
+ override def commit(
+ logStore: LogStore,
+ hadoopConf: Configuration,
+ logPath: Path,
+ coordinatedCommitsTableConf: Map[String, String],
+ commitVersion: Long,
+ actions: Iterator[String],
+ updatedActions: UpdatedActions): CommitResponse = recordOperation("commit") {
+ delegatingCommitCoordinatorClient.commit(
+ logStore,
+ hadoopConf,
+ logPath,
+ coordinatedCommitsTableConf,
+ commitVersion,
+ actions,
+ updatedActions)
+ }
+
+ override def getCommits(
+ logPath: Path,
+ coordinatedCommitsTableConf: Map[String, String],
+ startVersion: Option[Long],
+ endVersion: Option[Long] = None): GetCommitsResponse = recordOperation("getCommits") {
+ delegatingCommitCoordinatorClient.getCommits(
+ logPath, coordinatedCommitsTableConf, startVersion, endVersion)
+ }
+
+ def removeCommitTestOnly(
+ logPath: Path,
+ commitVersion: Long
+ ): Unit = {
+ delegatingCommitCoordinatorClient.perTableMap.get(logPath).commitsMap.remove(commitVersion)
+ }
+
+ override def backfillToVersion(
+ logStore: LogStore,
+ hadoopConf: Configuration,
+ logPath: Path,
+ coordinatedCommitsTableConf: Map[String, String],
+ version: Long,
+ lastKnownBackfilledVersion: Option[Long]): Unit = recordOperation("backfillToVersion") {
+ delegatingCommitCoordinatorClient.backfillToVersion(
+ logStore,
+ hadoopConf,
+ logPath,
+ coordinatedCommitsTableConf,
+ version,
+ lastKnownBackfilledVersion)
+ }
+
+ override def semanticEquals(other: CommitCoordinatorClient): Boolean = this == other
+
+ def reset(): Unit = {
+ numCommitsCalled.set(0)
+ numGetCommitsCalled.set(0)
+ numBackfillToVersionCalled.set(0)
+ }
+
+ override def registerTable(
+ logPath: Path,
+ currentVersion: Long,
+ currentMetadata: AbstractMetadata,
+ currentProtocol: AbstractProtocol): Map[String, String] = recordOperation("registerTable") {
+ delegatingCommitCoordinatorClient.registerTable(
+ logPath, currentVersion, currentMetadata, currentProtocol)
+ }
+}
+
+/**
+ * A helper class which enables coordinated-commits for the test suite based on the given
+ * `coordinatedCommitsBackfillBatchSize` conf.
+ */
+trait CoordinatedCommitsBaseSuite extends SparkFunSuite with SharedSparkSession {
+
+ // If this config is not overridden, coordinated commits are disabled.
+ def coordinatedCommitsBackfillBatchSize: Option[Int] = None
+
+ final def coordinatedCommitsEnabledInTests: Boolean = coordinatedCommitsBackfillBatchSize.nonEmpty
+
+ override protected def sparkConf: SparkConf = {
+ if (coordinatedCommitsBackfillBatchSize.nonEmpty) {
+ val coordinatedCommitsCoordinatorConf = Map("randomConf" -> "randomConfValue")
+ val coordinatedCommitsCoordinatorJson = JsonUtils.toJson(coordinatedCommitsCoordinatorConf)
+ super.sparkConf
+ .set(
+ DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_NAME.defaultTablePropertyKey,
+ "tracking-in-memory")
+ .set(
+ DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_CONF.defaultTablePropertyKey,
+ coordinatedCommitsCoordinatorJson)
+ } else {
+ super.sparkConf
+ }
+ }
+
+ override def beforeEach(): Unit = {
+ super.beforeEach()
+ CommitCoordinatorProvider.clearNonDefaultBuilders()
+ coordinatedCommitsBackfillBatchSize.foreach { batchSize =>
+ CommitCoordinatorProvider.registerBuilder(TrackingInMemoryCommitCoordinatorBuilder(batchSize))
+ }
+ }
+
+ protected def isICTEnabledForNewTables: Boolean = {
+ spark.conf.getOption(
+ DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_NAME.defaultTablePropertyKey).nonEmpty ||
+ spark.conf.getOption(
+ DeltaConfigs.IN_COMMIT_TIMESTAMPS_ENABLED.defaultTablePropertyKey).contains("true")
+ }
+}
diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/managedcommit/DynamoDBCommitOwnerClientSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/coordinatedcommits/DynamoDBCommitCoordinatorClientSuite.scala
similarity index 74%
rename from spark/src/test/scala/org/apache/spark/sql/delta/managedcommit/DynamoDBCommitOwnerClientSuite.scala
rename to spark/src/test/scala/org/apache/spark/sql/delta/coordinatedcommits/DynamoDBCommitCoordinatorClientSuite.scala
index 6300f6ff56f..02414395901 100644
--- a/spark/src/test/scala/org/apache/spark/sql/delta/managedcommit/DynamoDBCommitOwnerClientSuite.scala
+++ b/spark/src/test/scala/org/apache/spark/sql/delta/coordinatedcommits/DynamoDBCommitCoordinatorClientSuite.scala
@@ -14,7 +14,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.delta.managedcommit
+package org.apache.spark.sql.delta.coordinatedcommits
import java.util.concurrent.locks.ReentrantReadWriteLock
@@ -27,7 +27,7 @@ import org.apache.spark.sql.delta.{DeltaConfigs, DeltaLog}
import org.apache.spark.sql.delta.actions.{Metadata, Protocol}
import org.apache.spark.sql.delta.sources.DeltaSQLConf
import org.apache.spark.sql.delta.util.{FileNames, JsonUtils}
-import io.delta.dynamodbcommitstore.{DynamoDBCommitOwnerClient, DynamoDBCommitOwnerClientBuilder}
+import io.delta.dynamodbcommitcoordinator.{DynamoDBCommitCoordinatorClient, DynamoDBCommitCoordinatorClientBuilder}
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
@@ -35,7 +35,7 @@ import org.apache.spark.sql.SparkSession
/**
* An in-memory implementation of DynamoDB client for testing. Only the methods used by
- * `DynamoDBCommitOwnerClient` are implemented.
+ * `DynamoDBCommitCoordinatorClient` are implemented.
*/
class InMemoryDynamoDBClient extends AbstractAmazonDynamoDB {
/**
@@ -139,45 +139,46 @@ class InMemoryDynamoDBClient extends AbstractAmazonDynamoDB {
}
}
-case class TestDynamoDBCommitOwnerBuilder(batchSize: Long) extends CommitOwnerBuilder {
+case class TestDynamoDBCommitCoordinatorBuilder(batchSize: Long) extends CommitCoordinatorBuilder {
override def getName: String = "test-dynamodb"
- override def build(spark: SparkSession, config: Map[String, String]): CommitOwnerClient = {
- new DynamoDBCommitOwnerClient(
- "testTable",
- "test-endpoint",
- new InMemoryDynamoDBClient(),
- batchSize)
+ override def build(
+ spark: SparkSession, config: Map[String, String]): CommitCoordinatorClient = {
+ new DynamoDBCommitCoordinatorClient(
+ "testTable",
+ "test-endpoint",
+ new InMemoryDynamoDBClient(),
+ batchSize)
}
}
-abstract class DynamoDBCommitOwnerClientSuite(batchSize: Long)
- extends CommitOwnerClientImplSuiteBase {
+abstract class DynamoDBCommitCoordinatorClientSuite(batchSize: Long)
+ extends CommitCoordinatorClientImplSuiteBase {
- override protected def createTableCommitOwnerClient(
+ override protected def createTableCommitCoordinatorClient(
deltaLog: DeltaLog)
- : TableCommitOwnerClient = {
- val cs = TestDynamoDBCommitOwnerBuilder(batchSize = batchSize).build(spark, Map.empty)
+ : TableCommitCoordinatorClient = {
+ val cs = TestDynamoDBCommitCoordinatorBuilder(batchSize = batchSize).build(spark, Map.empty)
val tableConf = cs.registerTable(
deltaLog.logPath,
currentVersion = -1L,
Metadata(),
Protocol(1, 1))
- TableCommitOwnerClient(cs, deltaLog, tableConf)
+ TableCommitCoordinatorClient(cs, deltaLog, tableConf)
}
override protected def registerBackfillOp(
- tableCommitOwnerClient: TableCommitOwnerClient,
+ tableCommitCoordinatorClient: TableCommitCoordinatorClient,
deltaLog: DeltaLog,
version: Long): Unit = {
- tableCommitOwnerClient.backfillToVersion(version)
+ tableCommitCoordinatorClient.backfillToVersion(version)
}
override protected def validateBackfillStrategy(
- tableCommitOwnerClient: TableCommitOwnerClient,
+ tableCommitCoordinatorClient: TableCommitCoordinatorClient,
logPath: Path,
version: Long): Unit = {
val lastExpectedBackfilledVersion = (version - (version % batchSize)).toInt
- val unbackfilledCommitVersionsAll = tableCommitOwnerClient
+ val unbackfilledCommitVersionsAll = tableCommitCoordinatorClient
.getCommits().getCommits.map(_.getVersion)
val expectedVersions = lastExpectedBackfilledVersion + 1 to version.toInt
@@ -206,7 +207,7 @@ abstract class DynamoDBCommitOwnerClientSuite(batchSize: Long)
val logPath = log.logPath
writeCommitZero(logPath)
val dynamoDB = new InMemoryDynamoDBClient();
- val commitOwner = new DynamoDBCommitOwnerClient(
+ val commitCoordinator = new DynamoDBCommitCoordinatorClient(
"testTable",
"test-endpoint",
dynamoDB,
@@ -214,7 +215,7 @@ abstract class DynamoDBCommitOwnerClientSuite(batchSize: Long)
1, // readCapacityUnits
1, // writeCapacityUnits
skipPathCheck)
- val tableConf = commitOwner.registerTable(
+ val tableConf = commitCoordinator.registerTable(
logPath,
-1L,
Metadata(),
@@ -224,15 +225,15 @@ abstract class DynamoDBCommitOwnerClientSuite(batchSize: Long)
val fs = wrongLogPath.getFileSystem(log.newDeltaHadoopConf())
fs.mkdirs(wrongTablePath)
fs.mkdirs(FileNames.commitDirPath(wrongLogPath))
- val wrongTablePathTableCommitOwner = new TableCommitOwnerClient(
- commitOwner, wrongLogPath, tableConf, log.newDeltaHadoopConf(), log.store)
+ val wrongTablePathTableCommitCoordinator = new TableCommitCoordinatorClient(
+ commitCoordinator, wrongLogPath, tableConf, log.newDeltaHadoopConf(), log.store)
if (skipPathCheck) {
// This should succeed because we are skipping the path check.
- val resp = commit(1L, 1L, wrongTablePathTableCommitOwner)
+ val resp = commit(1L, 1L, wrongTablePathTableCommitCoordinator)
assert(resp.getVersion == 1L)
} else {
val e = intercept[CommitFailedException] {
- commit(1L, 1L, wrongTablePathTableCommitOwner)
+ commit(1L, 1L, wrongTablePathTableCommitCoordinator)
}
assert(e.getMessage.contains("while the table is registered at"))
}
@@ -240,7 +241,7 @@ abstract class DynamoDBCommitOwnerClientSuite(batchSize: Long)
}
test("builder should read dynamic configs from sparkSession") {
- class TestDynamoDBCommitOwnerBuilder extends DynamoDBCommitOwnerClientBuilder {
+ class TestDynamoDBCommitCoordinatorBuilder extends DynamoDBCommitCoordinatorClientBuilder {
override def getName: String = "dynamodb-test"
override def createAmazonDDBClient(
endpoint: String,
@@ -251,22 +252,22 @@ abstract class DynamoDBCommitOwnerClientSuite(batchSize: Long)
new InMemoryDynamoDBClient()
}
- override def getDynamoDBCommitOwnerClient(
- managedCommitTableName: String,
+ override def getDynamoDBCommitCoordinatorClient(
+ coordinatedCommitsTableName: String,
dynamoDBEndpoint: String,
ddbClient: AmazonDynamoDB,
backfillBatchSize: Long,
readCapacityUnits: Int,
writeCapacityUnits: Int,
- skipPathCheck: Boolean): DynamoDBCommitOwnerClient = {
- assert(managedCommitTableName == "tableName-1223")
+ skipPathCheck: Boolean): DynamoDBCommitCoordinatorClient = {
+ assert(coordinatedCommitsTableName == "tableName-1223")
assert(dynamoDBEndpoint == "endpoint-1224")
assert(backfillBatchSize == 1)
assert(readCapacityUnits == 1226)
assert(writeCapacityUnits == 1227)
assert(skipPathCheck)
- new DynamoDBCommitOwnerClient(
- managedCommitTableName,
+ new DynamoDBCommitCoordinatorClient(
+ coordinatedCommitsTableName,
dynamoDBEndpoint,
ddbClient,
backfillBatchSize,
@@ -275,30 +276,34 @@ abstract class DynamoDBCommitOwnerClientSuite(batchSize: Long)
skipPathCheck)
}
}
- val commitOwnerConf = JsonUtils.toJson(Map(
+ val commitCoordinatorConf = JsonUtils.toJson(Map(
"dynamoDBTableName" -> "tableName-1223",
"dynamoDBEndpoint" -> "endpoint-1224"
))
withSQLConf(
- DeltaConfigs.MANAGED_COMMIT_OWNER_NAME.defaultTablePropertyKey -> "dynamodb-test",
- DeltaConfigs.MANAGED_COMMIT_OWNER_CONF.defaultTablePropertyKey -> commitOwnerConf,
- DeltaSQLConf.MANAGED_COMMIT_DDB_AWS_CREDENTIALS_PROVIDER_NAME.key -> "creds-1225",
- DeltaSQLConf.MANAGED_COMMIT_DDB_SKIP_PATH_CHECK.key -> "true",
- DeltaSQLConf.MANAGED_COMMIT_DDB_READ_CAPACITY_UNITS.key -> "1226",
- DeltaSQLConf.MANAGED_COMMIT_DDB_WRITE_CAPACITY_UNITS.key -> "1227") {
+ DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_NAME.defaultTablePropertyKey ->
+ "dynamodb-test",
+ DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_CONF.defaultTablePropertyKey ->
+ commitCoordinatorConf,
+ DeltaSQLConf.COORDINATED_COMMITS_DDB_AWS_CREDENTIALS_PROVIDER_NAME.key -> "creds-1225",
+ DeltaSQLConf.COORDINATED_COMMITS_DDB_SKIP_PATH_CHECK.key -> "true",
+ DeltaSQLConf.COORDINATED_COMMITS_DDB_READ_CAPACITY_UNITS.key -> "1226",
+ DeltaSQLConf.COORDINATED_COMMITS_DDB_WRITE_CAPACITY_UNITS.key -> "1227") {
// clear default builders
- CommitOwnerProvider.clearNonDefaultBuilders()
- CommitOwnerProvider.registerBuilder(new TestDynamoDBCommitOwnerBuilder())
+ CommitCoordinatorProvider.clearNonDefaultBuilders()
+ CommitCoordinatorProvider.registerBuilder(new TestDynamoDBCommitCoordinatorBuilder())
withTempTableDir { tempDir =>
val tablePath = tempDir.getAbsolutePath
spark.range(1).write.format("delta").mode("overwrite").save(tablePath)
val log = DeltaLog.forTable(spark, tempDir.toString)
- val tableCommitOwnerClient = log.snapshot.tableCommitOwnerClientOpt.get
- assert(tableCommitOwnerClient.commitOwnerClient.isInstanceOf[DynamoDBCommitOwnerClient])
- assert(tableCommitOwnerClient.tableConf.contains("tableId"))
+ val tableCommitCoordinatorClient = log.snapshot.tableCommitCoordinatorClientOpt.get
+ assert(tableCommitCoordinatorClient
+ .commitCoordinatorClient
+ .isInstanceOf[DynamoDBCommitCoordinatorClient])
+ assert(tableCommitCoordinatorClient.tableConf.contains("tableId"))
}
}
}
}
-class DynamoDBCommitOwnerClient5BackfillSuite extends DynamoDBCommitOwnerClientSuite(5)
+class DynamoDBCommitCoordinatorClient5BackfillSuite extends DynamoDBCommitCoordinatorClientSuite(5)
diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/managedcommit/InMemoryCommitOwnerSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/coordinatedcommits/InMemoryCommitCoordinatorSuite.scala
similarity index 60%
rename from spark/src/test/scala/org/apache/spark/sql/delta/managedcommit/InMemoryCommitOwnerSuite.scala
rename to spark/src/test/scala/org/apache/spark/sql/delta/coordinatedcommits/InMemoryCommitCoordinatorSuite.scala
index be1fd909e70..0181e478adc 100644
--- a/spark/src/test/scala/org/apache/spark/sql/delta/managedcommit/InMemoryCommitOwnerSuite.scala
+++ b/spark/src/test/scala/org/apache/spark/sql/delta/coordinatedcommits/InMemoryCommitCoordinatorSuite.scala
@@ -14,40 +14,41 @@
* limitations under the License.
*/
-package org.apache.spark.sql.delta.managedcommit
+package org.apache.spark.sql.delta.coordinatedcommits
import org.apache.spark.sql.delta.DeltaLog
import org.apache.spark.sql.delta.actions.Protocol
import org.apache.hadoop.fs.Path
-abstract class InMemoryCommitOwnerSuite(batchSize: Int) extends CommitOwnerClientImplSuiteBase {
+abstract class InMemoryCommitCoordinatorSuite(batchSize: Int)
+ extends CommitCoordinatorClientImplSuiteBase {
- override protected def createTableCommitOwnerClient(
- deltaLog: DeltaLog): TableCommitOwnerClient = {
- val cs = InMemoryCommitOwnerBuilder(batchSize).build(spark, Map.empty)
+ override protected def createTableCommitCoordinatorClient(
+ deltaLog: DeltaLog): TableCommitCoordinatorClient = {
+ val cs = InMemoryCommitCoordinatorBuilder(batchSize).build(spark, Map.empty)
val conf = cs.registerTable(
deltaLog.logPath,
currentVersion = -1L,
initMetadata,
Protocol(1, 1))
- TableCommitOwnerClient(cs, deltaLog, conf)
+ TableCommitCoordinatorClient(cs, deltaLog, conf)
}
override protected def registerBackfillOp(
- tableCommitOwnerClient: TableCommitOwnerClient,
+ tableCommitCoordinatorClient: TableCommitCoordinatorClient,
deltaLog: DeltaLog,
version: Long): Unit = {
- val commitOwnerClient = tableCommitOwnerClient.commitOwnerClient
- val inMemoryCS = commitOwnerClient.asInstanceOf[InMemoryCommitOwner]
+ val commitCoordinatorClient = tableCommitCoordinatorClient.commitCoordinatorClient
+ val inMemoryCS = commitCoordinatorClient.asInstanceOf[InMemoryCommitCoordinator]
inMemoryCS.registerBackfill(deltaLog.logPath, version)
}
override protected def validateBackfillStrategy(
- tableCommitOwnerClient: TableCommitOwnerClient,
+ tableCommitCoordinatorClient: TableCommitCoordinatorClient,
logPath: Path,
version: Long): Unit = {
val lastExpectedBackfilledVersion = (version - (version % batchSize)).toInt
- val unbackfilledCommitVersionsAll = tableCommitOwnerClient
+ val unbackfilledCommitVersionsAll = tableCommitCoordinatorClient
.getCommits().getCommits.map(_.getVersion)
val expectedVersions = lastExpectedBackfilledVersion + 1 to version.toInt
@@ -69,26 +70,26 @@ abstract class InMemoryCommitOwnerSuite(batchSize: Int) extends CommitOwnerClien
assert(result.getLatestTableVersion == maxVersion)
}
- test("InMemoryCommitOwnerBuilder works as expected") {
- val builder1 = InMemoryCommitOwnerBuilder(5)
+ test("InMemoryCommitCoordinatorBuilder works as expected") {
+ val builder1 = InMemoryCommitCoordinatorBuilder(5)
val cs1 = builder1.build(spark, Map.empty)
- assert(cs1.isInstanceOf[InMemoryCommitOwner])
- assert(cs1.asInstanceOf[InMemoryCommitOwner].batchSize == 5)
+ assert(cs1.isInstanceOf[InMemoryCommitCoordinator])
+ assert(cs1.asInstanceOf[InMemoryCommitCoordinator].batchSize == 5)
val cs1_again = builder1.build(spark, Map.empty)
- assert(cs1_again.isInstanceOf[InMemoryCommitOwner])
+ assert(cs1_again.isInstanceOf[InMemoryCommitCoordinator])
assert(cs1 == cs1_again)
- val builder2 = InMemoryCommitOwnerBuilder(10)
+ val builder2 = InMemoryCommitCoordinatorBuilder(10)
val cs2 = builder2.build(spark, Map.empty)
- assert(cs2.isInstanceOf[InMemoryCommitOwner])
- assert(cs2.asInstanceOf[InMemoryCommitOwner].batchSize == 10)
+ assert(cs2.isInstanceOf[InMemoryCommitCoordinator])
+ assert(cs2.asInstanceOf[InMemoryCommitCoordinator].batchSize == 10)
assert(cs2 ne cs1)
- val builder3 = InMemoryCommitOwnerBuilder(10)
+ val builder3 = InMemoryCommitCoordinatorBuilder(10)
val cs3 = builder3.build(spark, Map.empty)
- assert(cs3.isInstanceOf[InMemoryCommitOwner])
- assert(cs3.asInstanceOf[InMemoryCommitOwner].batchSize == 10)
+ assert(cs3.isInstanceOf[InMemoryCommitCoordinator])
+ assert(cs3.asInstanceOf[InMemoryCommitCoordinator].batchSize == 10)
assert(cs3 ne cs2)
}
@@ -96,15 +97,15 @@ abstract class InMemoryCommitOwnerSuite(batchSize: Int) extends CommitOwnerClien
withTempTableDir { tempDir =>
val log = DeltaLog.forTable(spark, tempDir.toString)
val logPath = log.logPath
- val tcs = createTableCommitOwnerClient(log)
+ val tcs = createTableCommitCoordinatorClient(log)
// Anything other than version-0 or version-1 should be rejected as the first commit
- // version-0 will be directly backfilled and won't be recorded in InMemoryCommitOwner.
- // version-1 is what commit-owner is accepting.
+ // version-0 will be directly backfilled and won't be recorded in InMemoryCommitCoordinator.
+ // version-1 is what commit-coordinator is accepting.
assertCommitFail(2, 1, retryable = false, commit(2, 0, tcs))
}
}
}
-class InMemoryCommitOwner1Suite extends InMemoryCommitOwnerSuite(1)
-class InMemoryCommitOwner5Suite extends InMemoryCommitOwnerSuite(5)
+class InMemoryCommitCoordinator1Suite extends InMemoryCommitCoordinatorSuite(1)
+class InMemoryCommitCoordinator5Suite extends InMemoryCommitCoordinatorSuite(5)
diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/managedcommit/CommitOwnerClientSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/managedcommit/CommitOwnerClientSuite.scala
deleted file mode 100644
index 5faa9b02ecf..00000000000
--- a/spark/src/test/scala/org/apache/spark/sql/delta/managedcommit/CommitOwnerClientSuite.scala
+++ /dev/null
@@ -1,274 +0,0 @@
-/*
- * 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.managedcommit
-
-import scala.reflect.runtime.universe._
-
-import org.apache.spark.sql.delta.{DeltaConfigs, DeltaLog, DeltaOperations, ManagedCommitTableFeature}
-import org.apache.spark.sql.delta.actions._
-import org.apache.spark.sql.delta.storage.LogStore
-import org.apache.spark.sql.delta.test.DeltaSQLCommandTest
-import org.apache.spark.sql.delta.test.DeltaSQLTestUtils
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.Path
-
-import org.apache.spark.sql.{QueryTest, SparkSession}
-import org.apache.spark.sql.test.SharedSparkSession
-
-class CommitOwnerClientSuite extends QueryTest with DeltaSQLTestUtils with SharedSparkSession
- with DeltaSQLCommandTest {
-
- private trait TestCommitOwnerClientBase extends CommitOwnerClient {
- override def commit(
- logStore: LogStore,
- hadoopConf: Configuration,
- logPath: Path,
- managedCommitTableConf: Map[String, String],
- commitVersion: Long,
- actions: Iterator[String],
- updatedActions: UpdatedActions): CommitResponse = {
- throw new UnsupportedOperationException("Not implemented")
- }
-
- override def getCommits(
- logPath: Path,
- managedCommitTableConf: Map[String, String],
- startVersion: Option[Long],
- endVersion: Option[Long] = None): GetCommitsResponse = GetCommitsResponse(Seq.empty, -1)
-
- override def backfillToVersion(
- logStore: LogStore,
- hadoopConf: Configuration,
- logPath: Path,
- managedCommitTableConf: Map[String, String],
- version: Long,
- lastKnownBackfilledVersion: Option[Long]): Unit = {}
-
- override def semanticEquals(other: CommitOwnerClient): Boolean = this == other
- }
-
- private class TestCommitOwnerClient1 extends TestCommitOwnerClientBase
- private class TestCommitOwnerClient2 extends TestCommitOwnerClientBase
-
- override def beforeEach(): Unit = {
- super.beforeEach()
- CommitOwnerProvider.clearNonDefaultBuilders()
- CommitOwnerProvider.registerBuilder(InMemoryCommitOwnerBuilder(batchSize = 1))
- }
-
- test("registering multiple commit-owner builders with same name") {
- object Builder1 extends CommitOwnerBuilder {
- override def build(spark: SparkSession, conf: Map[String, String]): CommitOwnerClient = null
- override def getName: String = "builder-1"
- }
- object BuilderWithSameName extends CommitOwnerBuilder {
- override def build(spark: SparkSession, conf: Map[String, String]): CommitOwnerClient = null
- override def getName: String = "builder-1"
- }
- object Builder3 extends CommitOwnerBuilder {
- override def build(spark: SparkSession, conf: Map[String, String]): CommitOwnerClient = null
- override def getName: String = "builder-3"
- }
- CommitOwnerProvider.registerBuilder(Builder1)
- intercept[Exception] {
- CommitOwnerProvider.registerBuilder(BuilderWithSameName)
- }
- CommitOwnerProvider.registerBuilder(Builder3)
- }
-
- test("getCommitOwner - builder returns same object") {
- object Builder1 extends CommitOwnerBuilder {
- val cs1 = new TestCommitOwnerClient1()
- val cs2 = new TestCommitOwnerClient2()
- override def build(spark: SparkSession, conf: Map[String, String]): CommitOwnerClient = {
- conf.getOrElse("url", "") match {
- case "url1" => cs1
- case "url2" => cs2
- case _ => throw new IllegalArgumentException("Invalid url")
- }
- }
- override def getName: String = "cs-x"
- }
- CommitOwnerProvider.registerBuilder(Builder1)
- val cs1 = CommitOwnerProvider.getCommitOwnerClient("cs-x", Map("url" -> "url1"), spark)
- assert(cs1.isInstanceOf[TestCommitOwnerClient1])
- val cs1_again = CommitOwnerProvider.getCommitOwnerClient("cs-x", Map("url" -> "url1"), spark)
- assert(cs1 eq cs1_again)
- val cs2 =
- CommitOwnerProvider.getCommitOwnerClient("cs-x", Map("url" -> "url2", "a" -> "b"), spark)
- assert(cs2.isInstanceOf[TestCommitOwnerClient2])
- // If builder receives a config which doesn't have expected params, then it can throw exception.
- intercept[IllegalArgumentException] {
- CommitOwnerProvider.getCommitOwnerClient("cs-x", Map("url" -> "url3"), spark)
- }
- }
-
- test("getCommitOwnerClient - builder returns new object each time") {
- object Builder1 extends CommitOwnerBuilder {
- override def build(spark: SparkSession, conf: Map[String, String]): CommitOwnerClient = {
- conf.getOrElse("url", "") match {
- case "url1" => new TestCommitOwnerClient1()
- case _ => throw new IllegalArgumentException("Invalid url")
- }
- }
- override def getName: String = "cs-name"
- }
- CommitOwnerProvider.registerBuilder(Builder1)
- val cs1 = CommitOwnerProvider.getCommitOwnerClient("cs-name", Map("url" -> "url1"), spark)
- assert(cs1.isInstanceOf[TestCommitOwnerClient1])
- val cs1_again = CommitOwnerProvider.getCommitOwnerClient("cs-name", Map("url" -> "url1"), spark)
- assert(cs1 ne cs1_again)
- }
-
- test("MANAGED_COMMIT_PROVIDER_CONF") {
- val m1 = Metadata(
- configuration = Map(
- DeltaConfigs.MANAGED_COMMIT_OWNER_CONF.key ->
- """{"key1": "string_value", "key2Int": 2, "key3ComplexStr": "\"hello\""}""")
- )
- assert(DeltaConfigs.MANAGED_COMMIT_OWNER_CONF.fromMetaData(m1) ===
- Map("key1" -> "string_value", "key2Int" -> "2", "key3ComplexStr" -> "\"hello\""))
-
- val m2 = Metadata(
- configuration = Map(
- DeltaConfigs.MANAGED_COMMIT_OWNER_CONF.key ->
- """{"key1": "string_value", "key2Int": "2""")
- )
- intercept[com.fasterxml.jackson.core.JsonParseException] {
- DeltaConfigs.MANAGED_COMMIT_OWNER_CONF.fromMetaData(m2)
- }
- }
-
- test("Commit fails if we try to put bad value for MANAGED_COMMIT_PROVIDER_CONF") {
- withTempDir { dir =>
- val path = dir.getCanonicalPath
- spark.range(10).write.format("delta").mode("append").save(path)
- val deltaLog = DeltaLog.forTable(spark, path)
-
- val metadataWithCorrectConf = Metadata(
- configuration = Map(
- DeltaConfigs.MANAGED_COMMIT_OWNER_CONF.key ->
- """{"key1": "string_value", "key2Int": 2, "key3ComplexStr": "\"hello\""}""")
- )
- val metadataWithIncorrectConf = Metadata(
- configuration = Map(
- DeltaConfigs.MANAGED_COMMIT_OWNER_CONF.key ->
- """{"key1": "string_value", "key2Int": "2""")
- )
-
- intercept[com.fasterxml.jackson.core.JsonParseException] {
- deltaLog.startTransaction().commit(
- Seq(metadataWithIncorrectConf), DeltaOperations.ManualUpdate)
- }
- DeltaConfigs.MANAGED_COMMIT_OWNER_CONF.fromMetaData(metadataWithCorrectConf)
- }
- }
-
- test("Adding MANAGED_COMMIT_PROVIDER_NAME table property automatically upgrades the Protocol") {
- withTempDir { dir =>
- val path = dir.getCanonicalPath
- spark.range(10).write.format("delta").mode("append").save(path)
- val metadata =
- Metadata(configuration = Map(DeltaConfigs.MANAGED_COMMIT_OWNER_NAME.key -> "in-memory"))
- val deltaLog = DeltaLog.forTable(spark, path)
-
- def getWriterFeatures(log: DeltaLog): Set[String] = {
- log.update().protocol.writerFeatures.getOrElse(Set.empty)
- }
-
- assert(!getWriterFeatures(deltaLog).contains(ManagedCommitTableFeature.name))
- deltaLog.startTransaction().commit(Seq(metadata), DeltaOperations.ManualUpdate)
- assert(getWriterFeatures(deltaLog).contains(ManagedCommitTableFeature.name))
- }
- }
-
- test("Semantic Equality works as expected on CommitOwnerClients") {
- class TestCommitOwnerClient(val key: String) extends TestCommitOwnerClientBase {
- override def semanticEquals(other: CommitOwnerClient): Boolean =
- other.isInstanceOf[TestCommitOwnerClient] &&
- other.asInstanceOf[TestCommitOwnerClient].key == key
- }
- object Builder1 extends CommitOwnerBuilder {
- override def build(spark: SparkSession, conf: Map[String, String]): CommitOwnerClient = {
- new TestCommitOwnerClient(conf("key"))
- }
- override def getName: String = "cs-name"
- }
- CommitOwnerProvider.registerBuilder(Builder1)
-
- // Different CommitOwner with same keys should be semantically equal.
- val obj1 = CommitOwnerProvider.getCommitOwnerClient("cs-name", Map("key" -> "url1"), spark)
- val obj2 = CommitOwnerProvider.getCommitOwnerClient("cs-name", Map("key" -> "url1"), spark)
- assert(obj1 != obj2)
- assert(obj1.semanticEquals(obj2))
-
- // Different CommitOwner with different keys should be semantically unequal.
- val obj3 = CommitOwnerProvider.getCommitOwnerClient("cs-name", Map("key" -> "url2"), spark)
- assert(obj1 != obj3)
- assert(!obj1.semanticEquals(obj3))
- }
-
- private def checkMissing[Interface: TypeTag, Class: TypeTag](): Set[String] = {
- val fields = typeOf[Class].decls.collect {
- case m: MethodSymbol if m.isCaseAccessor => m.name.toString
- }
-
- val getters = typeOf[Interface].decls.collect {
- case m: MethodSymbol if m.isAbstract => m.name.toString
- }.toSet
-
- fields.filterNot { field =>
- getters.contains(s"get${field.capitalize}")
- }.toSet
- }
-
- /**
- * We expect the Protocol action to have the same fields as AbstractProtocol (part of the
- * CommitOwnerClient interface). With this if any change has happened in the Protocol of the
- * table, the same change is propagated to the CommitOwnerClient as AbstractProtocol. The
- * CommitOwnerClient can access the changes using getters and decide to act on the changes
- * based on the spec of the commit owner.
- *
- * This test case ensures that any new field added in the Protocol action is also accessible in
- * the CommitOwnerClient via the getter. If the new field is something which we do not expect to
- * be passed to the CommitOwnerClient, the test needs to be modified accordingly.
- */
- test("AbstractProtocol should have getter methods for all fields in Protocol") {
- val missingFields = checkMissing[AbstractProtocol, Protocol]()
- val expectedMissingFields = Set.empty[String]
- assert(missingFields == expectedMissingFields,
- s"Missing getter methods in AbstractProtocol")
- }
-
- /**
- * We expect the Metadata action to have the same fields as AbstractMetadata (part of the
- * CommitOwnerClient interface). With this if any change has happened in the Metadata of the
- * table, the same change is propagated to the CommitOwnerClient as AbstractMetadata. The
- * CommitOwnerClient can access the changes using getters and decide to act on the changes
- * based on the spec of the commit owner.
- *
- * This test case ensures that any new field added in the Metadata action is also accessible in
- * the CommitOwnerClient via the getter. If the new field is something which we do not expect to
- * be passed to the CommitOwnerClient, the test needs to be modified accordingly.
- */
- test("BaseMetadata should have getter methods for all fields in Metadata") {
- val missingFields = checkMissing[AbstractMetadata, Metadata]()
- val expectedMissingFields = Set("format")
- assert(missingFields == expectedMissingFields,
- s"Missing getter methods in AbstractMetadata")
- }
-}
diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/managedcommit/ManagedCommitTestUtils.scala b/spark/src/test/scala/org/apache/spark/sql/delta/managedcommit/ManagedCommitTestUtils.scala
deleted file mode 100644
index 906e63bea8f..00000000000
--- a/spark/src/test/scala/org/apache/spark/sql/delta/managedcommit/ManagedCommitTestUtils.scala
+++ /dev/null
@@ -1,280 +0,0 @@
-/*
- * 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.managedcommit
-
-import java.util.concurrent.atomic.AtomicInteger
-
-import org.apache.spark.sql.delta.{DeltaConfigs, DeltaLog, DeltaTestUtilsBase}
-import org.apache.spark.sql.delta.DeltaConfigs.MANAGED_COMMIT_OWNER_NAME
-import org.apache.spark.sql.delta.actions.{Action, CommitInfo, Metadata, Protocol}
-import org.apache.spark.sql.delta.storage.LogStore
-import org.apache.spark.sql.delta.util.JsonUtils
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.Path
-
-import org.apache.spark.{SparkConf, SparkFunSuite}
-import org.apache.spark.sql.SparkSession
-import org.apache.spark.sql.test.SharedSparkSession
-
-trait ManagedCommitTestUtils
- extends DeltaTestUtilsBase { self: SparkFunSuite with SharedSparkSession =>
-
- /**
- * Runs a specific test with managed commits default properties unset.
- * Any table created in this test won't have managed commits enabled by default.
- */
- def testWithDefaultCommitOwnerUnset(testName: String)(f: => Unit): Unit = {
- test(testName) {
- withoutManagedCommitDefaultTableProperties {
- f
- }
- }
- }
-
- /**
- * Runs the function `f` with managed commits default properties unset.
- * Any table created in function `f`` won't have managed commits enabled by default.
- */
- def withoutManagedCommitDefaultTableProperties[T](f: => T): T = {
- val commitOwnerKey = MANAGED_COMMIT_OWNER_NAME.defaultTablePropertyKey
- val oldCommitOwnerValue = spark.conf.getOption(commitOwnerKey)
- spark.conf.unset(commitOwnerKey)
- try { f } finally {
- oldCommitOwnerValue.foreach {
- spark.conf.set(commitOwnerKey, _)
- }
- }
- }
-
- /** Run the test with different backfill batch sizes: 1, 2, 10 */
- def testWithDifferentBackfillInterval(testName: String)(f: Int => Unit): Unit = {
- Seq(1, 2, 10).foreach { backfillBatchSize =>
- test(s"$testName [Backfill batch size: $backfillBatchSize]") {
- CommitOwnerProvider.clearNonDefaultBuilders()
- CommitOwnerProvider.registerBuilder(TrackingInMemoryCommitOwnerBuilder(backfillBatchSize))
- CommitOwnerProvider.registerBuilder(InMemoryCommitOwnerBuilder(backfillBatchSize))
- f(backfillBatchSize)
- }
- }
- }
-
- /**
- * Run the test against a [[TrackingCommitOwnerClient]] with backfill batch size =
- * `batchBackfillSize`
- */
- def testWithManagedCommit(backfillBatchSize: Int)(testName: String)(f: => Unit): Unit = {
- test(s"$testName [Backfill batch size: $backfillBatchSize]") {
- CommitOwnerProvider.clearNonDefaultBuilders()
- CommitOwnerProvider.registerBuilder(TrackingInMemoryCommitOwnerBuilder(backfillBatchSize))
- val managedCommitOwnerConf = Map("randomConf" -> "randomConfValue")
- val managedCommitOwnerJson = JsonUtils.toJson(managedCommitOwnerConf)
- withSQLConf(
- DeltaConfigs.MANAGED_COMMIT_OWNER_NAME.defaultTablePropertyKey -> "tracking-in-memory",
- DeltaConfigs.MANAGED_COMMIT_OWNER_CONF.defaultTablePropertyKey ->
- managedCommitOwnerJson) {
- f
- }
- }
- }
-
- /** Run the test with:
- * 1. Without managed-commits
- * 2. With managed-commits with different backfill batch sizes
- */
- def testWithDifferentBackfillIntervalOptional(testName: String)(f: Option[Int] => Unit): Unit = {
- test(s"$testName [Backfill batch size: None]") {
- f(None)
- }
- testWithDifferentBackfillInterval(testName) { backfillBatchSize =>
- val managedCommitOwnerConf = Map("randomConf" -> "randomConfValue")
- val managedCommitOwnerJson = JsonUtils.toJson(managedCommitOwnerConf)
- withSQLConf(
- DeltaConfigs.MANAGED_COMMIT_OWNER_NAME.defaultTablePropertyKey -> "tracking-in-memory",
- DeltaConfigs.MANAGED_COMMIT_OWNER_CONF.defaultTablePropertyKey ->
- managedCommitOwnerJson) {
- f(Some(backfillBatchSize))
- }
- }
- }
-
- def getUpdatedActionsForZerothCommit(
- commitInfo: CommitInfo,
- oldMetadata: Metadata = Metadata()): UpdatedActions = {
- val newMetadataConfiguration =
- oldMetadata.configuration +
- (DeltaConfigs.MANAGED_COMMIT_OWNER_NAME.key -> "tracking-in-memory")
- val newMetadata = oldMetadata.copy(configuration = newMetadataConfiguration)
- UpdatedActions(commitInfo, newMetadata, Protocol(), oldMetadata, Protocol())
- }
-
- def getUpdatedActionsForNonZerothCommit(commitInfo: CommitInfo): UpdatedActions = {
- val updatedActions = getUpdatedActionsForZerothCommit(commitInfo)
- updatedActions.copy(oldMetadata = updatedActions.getNewMetadata)
- }
-}
-
-case class TrackingInMemoryCommitOwnerBuilder(
- batchSize: Long,
- defaultCommitOwnerClientOpt: Option[CommitOwnerClient] = None) extends CommitOwnerBuilder {
- lazy val trackingInMemoryCommitOwnerClient =
- defaultCommitOwnerClientOpt.getOrElse {
- new TrackingCommitOwnerClient(new PredictableUuidInMemoryCommitOwnerClient(batchSize))
- }
-
- override def getName: String = "tracking-in-memory"
- override def build(spark: SparkSession, conf: Map[String, String]): CommitOwnerClient = {
- trackingInMemoryCommitOwnerClient
- }
-}
-
-class PredictableUuidInMemoryCommitOwnerClient(batchSize: Long)
- extends InMemoryCommitOwner(batchSize) {
-
- var nextUuidSuffix = 1L
- override def generateUUID(): String = {
- nextUuidSuffix += 1
- s"uuid-${nextUuidSuffix - 1}"
- }
-}
-
-object TrackingCommitOwnerClient {
- private val insideOperation = new ThreadLocal[Boolean] {
- override def initialValue(): Boolean = false
- }
-}
-
-class TrackingCommitOwnerClient(delegatingCommitOwnerClient: InMemoryCommitOwner)
- extends CommitOwnerClient {
-
- val numCommitsCalled = new AtomicInteger(0)
- val numGetCommitsCalled = new AtomicInteger(0)
- val numBackfillToVersionCalled = new AtomicInteger(0)
- val numRegisterTableCalled = new AtomicInteger(0)
-
- def recordOperation[T](op: String)(f: => T): T = {
- val oldInsideOperation = TrackingCommitOwnerClient.insideOperation.get()
- try {
- if (!TrackingCommitOwnerClient.insideOperation.get()) {
- op match {
- case "commit" => numCommitsCalled.incrementAndGet()
- case "getCommits" => numGetCommitsCalled.incrementAndGet()
- case "backfillToVersion" => numBackfillToVersionCalled.incrementAndGet()
- case "registerTable" => numRegisterTableCalled.incrementAndGet()
- case _ => ()
- }
- }
- TrackingCommitOwnerClient.insideOperation.set(true)
- f
- } finally {
- TrackingCommitOwnerClient.insideOperation.set(oldInsideOperation)
- }
- }
-
- override def commit(
- logStore: LogStore,
- hadoopConf: Configuration,
- logPath: Path,
- managedCommitTableConf: Map[String, String],
- commitVersion: Long,
- actions: Iterator[String],
- updatedActions: UpdatedActions): CommitResponse = recordOperation("commit") {
- delegatingCommitOwnerClient.commit(
- logStore, hadoopConf, logPath, managedCommitTableConf, commitVersion, actions, updatedActions)
- }
-
- override def getCommits(
- logPath: Path,
- managedCommitTableConf: Map[String, String],
- startVersion: Option[Long],
- endVersion: Option[Long] = None): GetCommitsResponse = recordOperation("getCommits") {
- delegatingCommitOwnerClient.getCommits(
- logPath, managedCommitTableConf, startVersion, endVersion)
- }
-
- def removeCommitTestOnly(
- logPath: Path,
- commitVersion: Long
- ): Unit = {
- delegatingCommitOwnerClient.perTableMap.get(logPath).commitsMap.remove(commitVersion)
- }
-
- override def backfillToVersion(
- logStore: LogStore,
- hadoopConf: Configuration,
- logPath: Path,
- managedCommitTableConf: Map[String, String],
- version: Long,
- lastKnownBackfilledVersion: Option[Long]): Unit = recordOperation("backfillToVersion") {
- delegatingCommitOwnerClient.backfillToVersion(
- logStore, hadoopConf, logPath, managedCommitTableConf, version, lastKnownBackfilledVersion)
- }
-
- override def semanticEquals(other: CommitOwnerClient): Boolean = this == other
-
- def reset(): Unit = {
- numCommitsCalled.set(0)
- numGetCommitsCalled.set(0)
- numBackfillToVersionCalled.set(0)
- }
-
- override def registerTable(
- logPath: Path,
- currentVersion: Long,
- currentMetadata: AbstractMetadata,
- currentProtocol: AbstractProtocol): Map[String, String] = recordOperation("registerTable") {
- delegatingCommitOwnerClient.registerTable(
- logPath, currentVersion, currentMetadata, currentProtocol)
- }
-}
-
-/**
- * A helper class which enables managed-commit for the test suite based on the given
- * `managedCommitBackfillBatchSize` conf.
- */
-trait ManagedCommitBaseSuite extends SparkFunSuite with SharedSparkSession {
-
- // If this config is not overridden, managed commits are disabled.
- def managedCommitBackfillBatchSize: Option[Int] = None
-
- final def managedCommitEnabledInTests: Boolean = managedCommitBackfillBatchSize.nonEmpty
-
- override protected def sparkConf: SparkConf = {
- if (managedCommitBackfillBatchSize.nonEmpty) {
- val managedCommitOwnerConf = Map("randomConf" -> "randomConfValue")
- val managedCommitOwnerJson = JsonUtils.toJson(managedCommitOwnerConf)
- super.sparkConf
- .set(DeltaConfigs.MANAGED_COMMIT_OWNER_NAME.defaultTablePropertyKey, "tracking-in-memory")
- .set(DeltaConfigs.MANAGED_COMMIT_OWNER_CONF.defaultTablePropertyKey, managedCommitOwnerJson)
- } else {
- super.sparkConf
- }
- }
-
- override def beforeEach(): Unit = {
- super.beforeEach()
- CommitOwnerProvider.clearNonDefaultBuilders()
- managedCommitBackfillBatchSize.foreach { batchSize =>
- CommitOwnerProvider.registerBuilder(TrackingInMemoryCommitOwnerBuilder(batchSize))
- }
- }
-
- protected def isICTEnabledForNewTables: Boolean = {
- spark.conf.getOption(DeltaConfigs.MANAGED_COMMIT_OWNER_NAME.defaultTablePropertyKey).nonEmpty ||
- spark.conf.getOption(
- DeltaConfigs.IN_COMMIT_TIMESTAMPS_ENABLED.defaultTablePropertyKey).contains("true")
- }
-}