-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-24297][CORE] Fetch-to-disk by default for > 2gb #21474
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
Fetch-to-mem is guaranteed to fail if the message is bigger than 2 GB, so we might as well use fetch-to-disk in that case. The message includes some metadata in addition to the block data itself (in particular UploadBlock has a lot of metadata), so we leave a little room.
|
Test build #91365 has finished for PR 21474 at commit
|
| // as well use fetch-to-disk in that case. The message includes some metadata in addition | ||
| // to the block data itself (in particular UploadBlock has a lot of metadata), so we leave | ||
| // extra room. | ||
| .createWithDefault(Int.MaxValue - 500) |
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.
is the fixed buffer of 500 bytes enough for metadata?
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.
no guarantee its big enough. Seemed OK in the test I tried. But UploadBlock has some variable length strings so can't say for sure.
I'm fine making this much bigger, eg. 1 MB -- you'd only be bigger than that with a pathological case. then there would be some cases where we'd be taking an old message which was fine with fetch-to-mem and we'd switch to fetch-to-disk. But such a tiny case, and not an unreasonable change even for that ... so should be OK.
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.
right, it's the default and the user can change the value to leave more margin.
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.
Actually I prefer 512 to 500 :)
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.
sounds good, updated
| // as well use fetch-to-disk in that case. The message includes some metadata in addition | ||
| // to the block data itself (in particular UploadBlock has a lot of metadata), so we leave | ||
| // extra room. | ||
| .createWithDefault(Int.MaxValue - 500) |
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.
right, it's the default and the user can change the value to leave more margin.
|
Test build #92428 has finished for PR 21474 at commit
|
|
retest this please |
HyukjinKwon
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
|
Test build #93066 has finished for PR 21474 at commit
|
|
retest this please |
|
LGTM |
|
Test build #93093 has finished for PR 21474 at commit
|
|
@jerryshao, busy? I think you better take a look before merging this in if you are available. |
|
I will take a look at this sometime day, but don't block on me if it is urgent. |
| "external shuffle service, this feature can only be worked when external shuffle" + | ||
| "service is newer than Spark 2.2.") | ||
| .bytesConf(ByteUnit.BYTE) | ||
| .createWithDefault(Long.MaxValue) |
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.
I think the original purpose to set to Long.MaxValue is to avoid using this configuration by default, user should set to a proper size to enable this feature. But anyway I think the current change is also fine.
|
Still LGTM but let me leave it to you @squito and @jerryshao . |
|
retest this please |
|
Test build #93423 has finished for PR 21474 at commit
|
|
retest this please |
|
Hi @squito , would you please also update the changes in the doc, thanks! |
|
good point @jerryshao , I've updated the docs now, please take a look, thanks |
|
Test build #93501 has finished for PR 21474 at commit
|
jerryshao
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, merging to master.
|
just to clarify here, so the failure case if you are using a shuffle server < 2.2, does it throw an obvious error to the user? |
|
oh no, good point Tom. it'll fail with lemme open a jira to improve this. |
Fetch-to-mem is guaranteed to fail if the message is bigger than 2 GB, so we might as well use fetch-to-disk in that case. The message includes some metadata in addition to the block data itself (in particular UploadBlock has a lot of metadata), so we leave a little room. Author: Imran Rashid <[email protected]> Closes apache#21474 from squito/SPARK-24297.
Fetch-to-mem is guaranteed to fail if the message is bigger than 2 GB,
so we might as well use fetch-to-disk in that case. The message includes
some metadata in addition to the block data itself (in particular
UploadBlock has a lot of metadata), so we leave a little room.