-
Notifications
You must be signed in to change notification settings - Fork 28.1k
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
[SPARK-3119] Re-implementation of TorrentBroadcast. #2030
Conversation
This is a re-implementation of TorrentBroadcast, with the following changes: 1. Removes most of the mutable, transient state from TorrentBroadcast (e.g. totalBytes, num of blocks fetched). 2. Removes TorrentInfo and TorrentBlock 3. Replaces the BlockManager.getSingle call in readObject with a getLocal, resuling in one less RPC call to the BlockManagerMasterActor to find the location of the block. 4. Removes the metadata block, resulting in one less block to fetch. 5. Removes an extra memory copy for deserialization (by using Java's SequenceInputStream).
QA tests have started for PR 2030 at commit
|
/** Fetch torrent blocks from the driver and/or other executors. */ | ||
private def readBlocks(): Array[Array[Byte]] = { | ||
// Fetch chunks of data. Note that all these chunks are stored in the BlockManager and reported | ||
// to the driver, so other executors can pull these thunks from this executor as well. |
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.
Typo: thunks -> chunks
Ran into some task failures when testing this commit on EC2 with the SchedulerThroughputTest:
|
arrayOfBlocks(pid) = x.asInstanceOf[TorrentBlock] | ||
hasBlocks += 1 | ||
_value = TorrentBroadcast.unBlockifyObject[T](blocks) | ||
// Store the merged copy in BlockManager so other tasks on this executor doesn't |
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: doesn't -> don't
QA tests have started for PR 2030 at commit
|
@rxin -- Nice work in reducing this to 2 RPCs. The patch looks good in terms of maintaining the same functionality as before. I'll wait for the Snappy fix and for Jenkins and then take another look. |
QA tests have finished for PR 2030 at commit
|
case Some(x) => | ||
blocks(pid) = x.asInstanceOf[Array[Byte]] | ||
numBlocksAvailable += 1 | ||
SparkEnv.get.blockManager.putBytes( |
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.
note to self - this is a bug
Ok I pushed a new version that should've addressed all the comments and fixed the bug. |
QA tests have started for PR 2030 at commit
|
QA tests have finished for PR 2030 at commit
|
QA tests have started for PR 2030 at commit
|
QA tests have finished for PR 2030 at commit
|
QA tests have started for PR 2030 at commit
|
Benchmarked as of 0d8ed5b and the results aren't conclusively faster than Each bar here represents a test where I ran 100 back-to-back jobs, each with 10 tasks, and varied the size of the task’s closure (each bar is the average of 10 runs, ignoring the first run to allow for JIT / warmup). The closure sizes (x-axis) are empty (well, whatever the minimum size was), 1 megabyte, and 10 megabytes; y-axis is time (seconds). This is running on 10 r3.2xlarge nodes in EC2. The test code is based off of my modified version of spark-perf (JoshRosen/spark-perf@0e768b2) Or, in tabular form, the means: and standard deviations: Keep in mind that this is running 100 back-to-back jobs; for example, v1.0.2 averaged 9ms per job for the small jobs. I'll run these benchmarks again tomorrow morning when I'm less tired to make sure I haven't inadvertently misconfigured anything. |
QA tests have finished for PR 2030 at commit
|
QA tests have finished for PR 2030 at commit
|
Jenkins, retest this please. |
Testing again to make sure tests pass two times in a row. |
QA tests have started for PR 2030 at commit
|
@JoshRosen Thanks for testing -- The perf results are a bit surprising (especially that master branch became faster since the earlier one). I also realized we do 3 RPCs, as after fetching the block we report back to the master. We could see if this reporting can be made asynchronous in the future |
I think just this patch vs. before #2028 vs. 1.0.2 should be fine. I just wanted to make sure the performance regression is minimal due to broadcast -- so as long as we are close or better than 1.0.2 numbers it should be good. |
Ok the test failures this time are just because of flaky Python tests.... |
Jenkins, retest this please. |
QA tests have started for PR 2030 at commit
|
* BlockManager, ready for other executors to fetch from. | ||
* | ||
* This prevents the driver from being the bottleneck in sending out multiple copies of the | ||
* broadcast data (one per executor) as done by the [[org.apache.spark.broadcast.HttpBroadcast]]. |
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.
Just curious, have we observed master becoming bottleneck for http broadcast ?
We have run with about 600 executor cores concurrently and have not seen this (the tasks themselves finish in sub seconds) - though it is probably a small enough sample set.
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.
try a larger variable, like a 10MB or 100MB model.
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.
To add, the latency of torrent broadcast is "sort of" twice http (2 requests instead of 1) assuming single piece for both : which is why I am curious about the bottleneck.
Currently we are running our jobs with http explicitly since our tasks are low latency low duration tasks.
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.
Hmm, our broadcast variables are in the 1mb - 3 mb range : so does not qualify I guess ... thx for the comment.
I tested this with a local cluster (which might not be a representative experiment), but this commit makes dummy short tasks finish in about 1/4 of the time. Command:
Master with this change:
Master without this:
1.0.2 (50% slower)
|
@rxin I think we can only benefit from broadcast an rdd when the closure is big enough, such as more than 1M bytes. But his most cases, the closure shoud be less than 10k. How about we turn on broadcasting only when the size of closure is big enough? This may make the code a little bit complicated, but may be make it faster in most cases (small closure). |
@@ -109,99 +155,30 @@ private[spark] class TorrentBroadcast[T: ClassTag]( | |||
private def readObject(in: ObjectInputStream) { | |||
in.defaultReadObject() | |||
TorrentBroadcast.synchronized { |
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.
Cant we not relax the synchronized to just around local block manager interactions ?
Or is there a reason to keep it globally synchronized ?
Multiple cores per container all would block on this.
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 am not sure there is a lot of benefit from having multiple cores execute readObject
. We'll then need synchronization for co-ordinating which piece is being fetched by each thread and cap the number of total outstanding requests etc.
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.
The lock is not on a specific instance - but on global class if I am not wrong.
A single slow readObject will block all other cores waiting for the lock to be released.
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.
The only thing we can improve here is to block only tasks fetching the same block. That'd help actually but requires more complexity.
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.
How 'bad' is it if we allow multiple cores to fetch the same block ? Particularly given that we shuffle the block fetch now (which is really neat change btw !)
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.
Note, I am only referring to case where none of the blocks are availabe locally (first fetch of one or more broadcasts).
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.
We can explore this in a later PR in case it is not possible to do it right now - if this change is targetted for 1.1; since it is obviously much better than what existed before.
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.
yup agree!
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.
Isn't this logic almost exactly what the CacheManager does already, fine grained locking on reconstructing specific blocks?
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.
maybe we can generalize that to provide a lock manager, similar to what dbms would do.
Actually with this change my local cluster testing didn't see much difference between torrent and http for small blocks. |
QA tests have finished for PR 2030 at commit
|
Looks good to me. Re: one of the earlier comments about broadcasting small objects through TorrentBroadcast. A not-so-intrusive way would be to piggyback data on top of the first control message, when data is small. Then small blocks will be like http, but bigger ones will becomes Torrent. |
That's a great idea. @JoshRosen and I were just talking about this before lunch! |
Currently running more jobs from the spark-perf suite against 5bacb9d (the latest commit) as a stress-test to gain more confidence that this is bug-free. |
Thanks Josh -- Perf results look good. LGTM. Can you file JIRAs to track the multi-threading things for 1.2 ? |
SPARK-3115 is an umbrella issue to improve task broadcast latency for small tasks; Reynold has created a bunch of subtasks to track future improvements, including the multi-threading improvements. |
Ok merging in master & branch-1.1. |
This is a re-implementation of TorrentBroadcast, with the following changes: 1. Removes most of the mutable, transient state from TorrentBroadcast (e.g. totalBytes, num of blocks fetched). 2. Removes TorrentInfo and TorrentBlock 3. Replaces the BlockManager.getSingle call in readObject with a getLocal, resuling in one less RPC call to the BlockManagerMasterActor to find the location of the block. 4. Removes the metadata block, resulting in one less block to fetch. 5. Removes an extra memory copy for deserialization (by using Java's SequenceInputStream). Basically for a regular broadcasted object with only one block, the number of RPC calls goes from 5+1 to 2+1). Old TorrentBroadcast for object of a single block: 1 RPC to ask for location of the broadcast variable 1 RPC to ask for location of the metadata block 1 RPC to fetch the metadata block 1 RPC to ask for location of the first data block 1 RPC to fetch the first data block 1 RPC to tell the driver we put the first data block in i.e. 5 + 1 New TorrentBroadcast for object of a single block: 1 RPC to ask for location of the first data block 1 RPC to get the first data block 1 RPC to tell the driver we put the first data block in i.e. 2 + 1 Author: Reynold Xin <rxin@apache.org> Closes #2030 from rxin/torrentBroadcast and squashes the following commits: 5bacb9d [Reynold Xin] Always add the object to driver's block manager. 0d8ed5b [Reynold Xin] Added getBytes to BlockManager and uses that in TorrentBroadcast. 2d6a5fb [Reynold Xin] Use putBytes/getRemoteBytes throughout. 3670f00 [Reynold Xin] Code review feedback. c1185cd [Reynold Xin] [SPARK-3119] Re-implementation of TorrentBroadcast. (cherry picked from commit 8adfbc2) Signed-off-by: Reynold Xin <rxin@apache.org>
@JoshRosen Did you ever see a significant difference between pre-PR and post-PR numbers? 1.0.2 was always much worse in all your graphs, but was this PR significant for larger blocks? By the way, 24 hour turnarounds on critical PRs that are going in at the end of a release cycle seem like a really bad way to preserve stability. |
FYI we actually reran all the mllib and scheduler throughput tests for this PR to verify. |
@aarondav The table and graph in #2030 (comment) compares pre-PR to post-PR. Actually it breaks it down into three runs: pre-PR, after 1st PR (#2028) and after 2nd PR (#2030). For large closures this PR had little to no effect. For smaller closures we see an improvement from 19.7ms before PR to 17.9 after 1st PR to 13.6 after the 2nd PR. |
This is a re-implementation of TorrentBroadcast, with the following changes: 1. Removes most of the mutable, transient state from TorrentBroadcast (e.g. totalBytes, num of blocks fetched). 2. Removes TorrentInfo and TorrentBlock 3. Replaces the BlockManager.getSingle call in readObject with a getLocal, resuling in one less RPC call to the BlockManagerMasterActor to find the location of the block. 4. Removes the metadata block, resulting in one less block to fetch. 5. Removes an extra memory copy for deserialization (by using Java's SequenceInputStream). Basically for a regular broadcasted object with only one block, the number of RPC calls goes from 5+1 to 2+1). Old TorrentBroadcast for object of a single block: 1 RPC to ask for location of the broadcast variable 1 RPC to ask for location of the metadata block 1 RPC to fetch the metadata block 1 RPC to ask for location of the first data block 1 RPC to fetch the first data block 1 RPC to tell the driver we put the first data block in i.e. 5 + 1 New TorrentBroadcast for object of a single block: 1 RPC to ask for location of the first data block 1 RPC to get the first data block 1 RPC to tell the driver we put the first data block in i.e. 2 + 1 Author: Reynold Xin <rxin@apache.org> Closes apache#2030 from rxin/torrentBroadcast and squashes the following commits: 5bacb9d [Reynold Xin] Always add the object to driver's block manager. 0d8ed5b [Reynold Xin] Added getBytes to BlockManager and uses that in TorrentBroadcast. 2d6a5fb [Reynold Xin] Use putBytes/getRemoteBytes throughout. 3670f00 [Reynold Xin] Code review feedback. c1185cd [Reynold Xin] [SPARK-3119] Re-implementation of TorrentBroadcast.
It could be fixed by #2624 It's strange that I can not see this comment on PR #2030. On Tue, Oct 7, 2014 at 6:28 AM, DB Tsai notifications@github.com wrote:
|
I thought it was a close issue, so I moved my comment to JIRA. I ran into Sent from my Google Nexus 5
|
This is a re-implementation of TorrentBroadcast, with the following changes:
Basically for a regular broadcasted object with only one block, the number of RPC calls goes from 5+1 to 2+1).
Old TorrentBroadcast for object of a single block:
1 RPC to ask for location of the broadcast variable
1 RPC to ask for location of the metadata block
1 RPC to fetch the metadata block
1 RPC to ask for location of the first data block
1 RPC to fetch the first data block
1 RPC to tell the driver we put the first data block in
i.e. 5 + 1
New TorrentBroadcast for object of a single block:
1 RPC to ask for location of the first data block
1 RPC to get the first data block
1 RPC to tell the driver we put the first data block in
i.e. 2 + 1