Skip to content

Commit 930ff26

Browse files
authored
Fix Redirect so that redirect-rw take precedence over redirect-w (delta-io#4352)
<!-- 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. --> When both redirect-rw and redirect-w configuration exist, we want redirect-rw take precedence over redirect-w. ## How was this patch tested? `TableRedirectSuite.scala` <!-- 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. --> ## Does this PR introduce _any_ user-facing changes? No <!-- 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'. -->
1 parent 23a7f90 commit 930ff26

File tree

2 files changed

+30
-6
lines changed

2 files changed

+30
-6
lines changed

spark/src/main/scala/org/apache/spark/sql/delta/redirect/TableRedirect.scala

Lines changed: 9 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -490,6 +490,10 @@ object RedirectFeature {
490490
mapper.readValue(configString, classOf[TableRedirectConfiguration])
491491
}
492492

493+
/**
494+
* Get the current `TableRedirectConfiguration` object from the table properties.
495+
* Note that the redirect-reader-writer takes precedence over redirect-writer-only.
496+
*/
493497
def getRedirectConfiguration(
494498
properties: Map[String, String]): Option[TableRedirectConfiguration] = {
495499
properties.get(DeltaConfigs.REDIRECT_READER_WRITER.key)
@@ -514,13 +518,12 @@ object RedirectFeature {
514518
}
515519
}
516520

517-
/** Get the current `TableRedirectConfiguration` object from the snapshot. */
521+
/**
522+
* Get the current `TableRedirectConfiguration` object from the snapshot.
523+
* Note that the redirect-reader-writer takes precedence over redirect-writer-only.
524+
*/
518525
def getRedirectConfiguration(snapshot: Snapshot): Option[TableRedirectConfiguration] = {
519-
if (RedirectWriterOnly.isFeatureSupported(snapshot)) {
520-
RedirectWriterOnly.getRedirectConfiguration(snapshot.metadata)
521-
} else {
522-
RedirectReaderWriter.getRedirectConfiguration(snapshot.metadata)
523-
}
526+
getRedirectConfiguration(snapshot.metadata.configuration)
524527
}
525528

526529
/** Determines whether `configs` contains redirect configuration. */

spark/src/test/scala/org/apache/spark/sql/delta/TableRedirectSuite.scala

Lines changed: 21 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@ import org.apache.spark.sql.delta.redirect.{
2525
EnableRedirectInProgress,
2626
NoRedirectRule,
2727
PathBasedRedirectSpec,
28+
RedirectFeature,
2829
RedirectReaderWriter,
2930
RedirectReady,
3031
RedirectSpec,
@@ -396,4 +397,24 @@ class TableRedirectSuite extends QueryTest
396397
assert(result.length == 4)
397398
}
398399
}
400+
401+
test("test getRedirectConfiguration") {
402+
val redirectSpec = new PathBasedRedirectSpec("sourcePath", "targetPath")
403+
val properties1 = RedirectReaderWriter.generateRedirectMetadata(
404+
PathBasedRedirectSpec.REDIRECT_TYPE,
405+
EnableRedirectInProgress,
406+
redirectSpec,
407+
noRedirectRules = Set.empty)
408+
val properties2 = RedirectWriterOnly.generateRedirectMetadata(
409+
PathBasedRedirectSpec.REDIRECT_TYPE,
410+
RedirectReady,
411+
redirectSpec,
412+
noRedirectRules = Set.empty)
413+
414+
val configuration = RedirectFeature.getRedirectConfiguration(properties1 ++ properties2)
415+
assert(configuration.isDefined)
416+
// redirect-reader-writer should be preferred over redirect-writer-only.
417+
assert(JsonUtils.toJson(configuration.get) ==
418+
properties1(DeltaConfigs.REDIRECT_READER_WRITER.key))
419+
}
399420
}

0 commit comments

Comments
 (0)