Skip to content

Commit

Permalink
[Delta] Add Test Suites for Coordinated Commits Properties (#3729)
Browse files Browse the repository at this point in the history
<!--
Thanks for sending a pull request!  Here are some tips for you:
1. If this is your first time, please read our contributor guidelines:
https://github.com/delta-io/delta/blob/master/CONTRIBUTING.md
2. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP]
Your PR title ...'.
  3. Be sure to keep the PR description updated to reflect all changes.
  4. Please write your PR title to summarize what this PR proposes.
5. If possible, provide a concise example to reproduce the issue for a
faster review.
6. If applicable, include the corresponding issue number in the PR title
and link it in the body.
-->

#### Which Delta project/connector is this regarding?
<!--
Please add the component selected below to the beginning of the pull
request title
For example: [Spark] Title of my pull request
-->

- [x] Spark
- [ ] Standalone
- [ ] Flink
- [ ] Kernel
- [ ] Other (fill in here)

## Description

<!--
- Describe what this PR changes.
- Describe why we need the change.
 
If this PR resolves an issue be sure to include "Resolves #XXX" to
correctly link and close the issue upon merge.
-->

Added UTs for CC properties' interaction with CREATE/REPLACE/CLONE
against this
[commit](18eb1a6).
All future CC properties-related test cases could go here.

## How was this patch tested?

<!--
If tests were added, say they were added here. Please make sure to test
the changes thoroughly including negative and positive cases if
possible.
If the changes were tested in any way other than unit tests, please
clarify how you tested step by step (ideally copy and paste-able, so
that other reviewers can test and check, and descendants can verify in
the future).
If the changes were not tested, please explain why.
-->

UTs.

## Does this PR introduce _any_ user-facing changes?

<!--
If yes, please clarify the previous behavior and the change this PR
proposes - provide the console output, description and/or an example to
show the behavior difference if possible.
If possible, please also clarify if this is a user-facing change
compared to the released Delta Lake versions or within the unreleased
branches such as master.
If no, write 'No'.
-->

No.
  • Loading branch information
yumingxuanguo-db authored Sep 26, 2024
1 parent 67934f8 commit 0b89ac3
Show file tree
Hide file tree
Showing 2 changed files with 473 additions and 135 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -857,141 +857,6 @@ trait CloneTableSuiteBase extends QueryTest
assert(sourceSnapshot.protocol === targetSnapshot.protocol)
}
}

testAllClones(s"Clone should pick commit coordinator from session settings") {
(source, target, isShallow) =>
val someOtherCommitCoordinatorName = "some-other-commit-coordinator"
case class SomeValidCommitCoordinatorBuilder() extends CommitCoordinatorBuilder {
val commitCoordinator = new InMemoryCommitCoordinator(batchSize = 1000L)
override def getName: String = someOtherCommitCoordinatorName
override def build(spark: SparkSession, conf: Map[String, String]): CommitCoordinatorClient =
commitCoordinator
}
CommitCoordinatorProvider.registerBuilder(SomeValidCommitCoordinatorBuilder())

// Create source table
val sourceLog = DeltaLog.forTable(spark, source)
spark.range(5).write.format("delta").save(source)
spark.range(5, 10).write.format("delta").mode("append").save(source)
spark.range(10, 15).write.format("delta").mode("append").save(source)

// CLONE should pick the commit coordinator from the session settings over the
// commit coordinator from the source table.
withCustomCoordinatedCommitsTableProperties(someOtherCommitCoordinatorName) {
cloneTable(
source,
target,
isReplace = true)
val targetLog = DeltaLog.forTable(spark, target)
val targetCommitCoordinator =
targetLog.update().tableCommitCoordinatorClientOpt.get.commitCoordinatorClient
assert(targetCommitCoordinator.asInstanceOf[InMemoryCommitCoordinator].batchSize == 1000)
}
checkAnswer(
spark.read.format("delta").load(source),
spark.read.format("delta").load(target))
}

testAllClones(s"Clone should ignore commit coordinator if it is not set in session settings") {
(source, target, isShallow) =>
// Create source table
val sourceLog = DeltaLog.forTable(spark, source)
spark.range(5).write.format("delta").save(source)
spark.range(5, 10).write.format("delta").mode("append").save(source)
spark.range(10, 15).write.format("delta").mode("append").save(source)

// Commit-Coordinator for target should not be set because it is unset in session.
withoutCoordinatedCommitsDefaultTableProperties {
cloneTable(
source,
target,
isReplace = true)
val targetLog = DeltaLog.forTable(spark, target)
assert(targetLog.update().tableCommitCoordinatorClientOpt.isEmpty)
}
checkAnswer(
spark.read.format("delta").load(source),
spark.read.format("delta").load(target))
}

testAllClones(s"Clone should give highest priority to commit coordinator specified directly in " +
s"clone command") { (source, target, isShallow) =>
val someOtherCommitCoordinatorName1 = "some-other-commit-coordinator-1"
case class SomeValidCommitCoordinatorBuilder1() extends CommitCoordinatorBuilder {
val commitCoordinator = new InMemoryCommitCoordinator(batchSize = 1000L)
override def getName: String = someOtherCommitCoordinatorName1
override def build(spark: SparkSession, conf: Map[String, String]): CommitCoordinatorClient =
commitCoordinator
}
CommitCoordinatorProvider.registerBuilder(SomeValidCommitCoordinatorBuilder1())
val someOtherCommitCoordinatorName2 = "some-other-commit-coordinator-2"
case class SomeValidCommitCoordinatorBuilder2() extends CommitCoordinatorBuilder {
val commitCoordinator = new InMemoryCommitCoordinator(batchSize = 2000L)
override def getName: String = someOtherCommitCoordinatorName2
override def build(spark: SparkSession, conf: Map[String, String]): CommitCoordinatorClient =
commitCoordinator
}
CommitCoordinatorProvider.registerBuilder(SomeValidCommitCoordinatorBuilder2())

// Create source table
val sourceLog = DeltaLog.forTable(spark, source)
spark.range(5).write.format("delta").save(source)
spark.range(5, 10).write.format("delta").mode("append").save(source)
spark.range(10, 15).write.format("delta").mode("append").save(source)

// When commit-coordinator is specified in both the spark session (with batchSize=1000 here)
// and CLONE command overrides (with batchSize=2000 here, CLONE should give priority to
// properties explicitly overridden with the CLONE command.
withCustomCoordinatedCommitsTableProperties(someOtherCommitCoordinatorName1) {
val e1 = intercept[IllegalArgumentException] {
val properties = Map(
DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_NAME.key ->
someOtherCommitCoordinatorName2)
cloneTable(
source,
target,
isReplace = true,
tableProperties = properties)
}
assert(e1.getMessage.contains("During CREATE with CLONE, either both coordinated commits " +
"configurations (\"delta.coordinatedCommits.commitCoordinator-preview\", \"delta." +
"coordinatedCommits.commitCoordinatorConf-preview\") are set in the command or neither " +
"of them. Missing: \"delta.coordinatedCommits.commitCoordinatorConf-preview\"."))

val e2 = intercept[IllegalArgumentException] {
val properties = Map(
DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_NAME.key ->
someOtherCommitCoordinatorName2,
DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_CONF.key -> "{}",
DeltaConfigs.COORDINATED_COMMITS_TABLE_CONF.key -> "{}")
cloneTable(
source,
target,
isReplace = true,
tableProperties = properties)
}
assert(e2.getMessage.contains("During CREATE with CLONE, configuration \"delta." +
"coordinatedCommits.tableConf-preview\" cannot be set from the command."))

val properties = Map(
DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_NAME.key ->
someOtherCommitCoordinatorName2,
DeltaConfigs.COORDINATED_COMMITS_COORDINATOR_CONF.key -> "{}")
cloneTable(
source,
target,
isReplace = true,
tableProperties = properties)

val targetLog = DeltaLog.forTable(spark, target)
val targetCommitStore =
targetLog.update().tableCommitCoordinatorClientOpt.get.commitCoordinatorClient
assert(targetCommitStore.asInstanceOf[InMemoryCommitCoordinator].batchSize == 2000L)
}
checkAnswer(
spark.read.format("delta").load(source),
spark.read.format("delta").load(target))
}
}


Expand Down
Loading

0 comments on commit 0b89ac3

Please sign in to comment.