-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-33545][CORE] Support Fallback Storage during Worker decommission #30492
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Conversation
core/src/main/scala/org/apache/spark/storage/FallbackStorage.scala
Outdated
Show resolved
Hide resolved
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
|
Test build #131768 has finished for PR 30492 at commit
|
|
@dongjoon-hyun is this only for shuffled data? I was wondering if it would also be possible to cover |
|
Thank you for review, @Tagar . Yes, they are separate options:
This PR is still under review. If this is accepted, I believe we can move on to that. |
|
Hi @dongjoon-hyun, I'm taking this week away from open source to take my puppy to go see snow for the first time. I'll do a review on Monday. Thanks for understanding :) |
|
Thank you, @holdenk . Sorry for pinging you on the holiday season. |
|
BTW, cc @dbtsai , too. |
core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala
Show resolved
Hide resolved
core/src/main/scala/org/apache/spark/storage/FallbackStorage.scala
Outdated
Show resolved
Hide resolved
|
Thank you so much for your review, @viirya ! |
core/src/main/scala/org/apache/spark/storage/FallbackStorage.scala
Outdated
Show resolved
Hide resolved
core/src/main/scala/org/apache/spark/internal/config/package.scala
Outdated
Show resolved
Hide resolved
|
Thanks for adding the application id. I don't have time to look at details recently. Will defer that to @viirya |
|
Test build #131939 has finished for PR 30492 at commit
|
|
Got it, @zsxwing . |
holdenk
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for waiting on the review. I’ve got a few minor nits/ questions but overall this looks good to me. I’m excited for us to support this as part of dynamic scale down in Spark 3.1 :)
core/src/main/scala/org/apache/spark/internal/config/package.scala
Outdated
Show resolved
Hide resolved
| try { | ||
| shuffleManager.shuffleBlockResolver.getBlockData(blockId) | ||
| } catch { | ||
| case e: IOException => |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nit/ question: Could we move the if up as part of the case and avoid the need for explicit rethrow?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It's because we should access the normal access path by default. We are able to use Fallback path only if the normal access path fail because conf.get(config.STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH).isDefined doesn't mean the fallback storage has the data.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Please let me know if I misunderstand your comment.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah so I mean we still have a try/catch just the case statement has the if as part of it (eg “ Using if expressions in case statements”)
core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
Show resolved
Hide resolved
holdenk
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM
viirya
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Looks okay to me.
What changes were proposed in this pull request?
This PR aims to support storage migration to the fallback storage like cloud storage (
S3) during worker decommission for the corner cases where the exceptions occur or there is no live peer left.Although this PR focuses on cloud storage like
S3which has a TTL feature in order to simplify Spark's logic, we can use alternative fallback storages like HDFS/NFS(EFS) if the user provides a clean-up mechanism.Why are the changes needed?
Currently, storage migration is not possible when there is no available executor. For example, when there is one executor, the executor cannot perform storage migration because it has no peer.
Does this PR introduce any user-facing change?
Yes. This is a new feature.
How was this patch tested?
Pass the CIs with newly added test cases.