Skip to content

Conversation

@squito
Copy link
Contributor

@squito squito commented May 31, 2018

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.

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.
@SparkQA
Copy link

SparkQA commented Jun 1, 2018

Test build #91365 has finished for PR 21474 at commit 6b7c4f7.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

// 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)
Copy link
Member

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?

Copy link
Contributor Author

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.

Copy link
Member

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.

Copy link
Contributor

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 :)

Copy link
Contributor Author

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)
Copy link
Member

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.

@SparkQA
Copy link

SparkQA commented Jun 28, 2018

Test build #92428 has finished for PR 21474 at commit 61e1991.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@HyukjinKwon
Copy link
Member

retest this please

Copy link
Member

@HyukjinKwon HyukjinKwon left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

@SparkQA
Copy link

SparkQA commented Jul 16, 2018

Test build #93066 has finished for PR 21474 at commit 61e1991.

  • This patch fails due to an unknown error code, -9.
  • This patch merges cleanly.
  • This patch adds no public classes.

@HyukjinKwon
Copy link
Member

retest this please

@jiangxb1987
Copy link
Contributor

LGTM

@SparkQA
Copy link

SparkQA commented Jul 16, 2018

Test build #93093 has finished for PR 21474 at commit 61e1991.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@HyukjinKwon
Copy link
Member

@jerryshao, busy? I think you better take a look before merging this in if you are available.

@jerryshao
Copy link
Contributor

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)
Copy link
Contributor

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.

@HyukjinKwon
Copy link
Member

Still LGTM but let me leave it to you @squito and @jerryshao .

@HyukjinKwon
Copy link
Member

retest this please

@SparkQA
Copy link

SparkQA commented Jul 23, 2018

Test build #93423 has finished for PR 21474 at commit 61e1991.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@HyukjinKwon
Copy link
Member

retest this please

@jerryshao
Copy link
Contributor

Hi @squito , would you please also update the changes in the doc, thanks!

@squito
Copy link
Contributor Author

squito commented Jul 24, 2018

good point @jerryshao , I've updated the docs now, please take a look, thanks

@SparkQA
Copy link

SparkQA commented Jul 24, 2018

Test build #93501 has finished for PR 21474 at commit 0ead6a1.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

Copy link
Contributor

@jerryshao jerryshao left a 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.

@asfgit asfgit closed this in 15fff79 Jul 25, 2018
@tgravescs
Copy link
Contributor

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?

@squito
Copy link
Contributor Author

squito commented Jul 26, 2018

oh no, good point Tom. it'll fail with

18/07/26 07:15:02 WARN scheduler.TaskSetManager: Lost task 1.0 in stage 1.3 (TID 15, irashid-large-2.gce.cloudera.com, executor 2): FetchFailed(BlockManagerId(1
, irashid-large-3.gce.cloudera.com, 7337, None), shuffleId=0, mapId=1, reduceId=1, message=
org.apache.spark.shuffle.FetchFailedException: java.lang.UnsupportedOperationException
        at org.apache.spark.network.server.StreamManager.openStream(StreamManager.java:60)
        at org.apache.spark.network.server.TransportRequestHandler.processStreamRequest(TransportRequestHandler.java:136)
...

lemme open a jira to improve this.

@squito
Copy link
Contributor Author

squito commented Jul 26, 2018

otterc pushed a commit to linkedin/spark that referenced this pull request Mar 22, 2023
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.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

7 participants