-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-22986][Core] Use a cache to avoid instantiating multiple instances of broadcast variable values #20183
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
…multiple times on executors when memory is constrained.
jiangxb1987
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.
This should be useful, cc @cloud-fan to also take a look.
|
|
||
| test("One broadcast value instance per executor") { | ||
| val conf = new SparkConf() | ||
| .setMaster("local[10]") |
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: normally local[4] is sufficient.
|
Please also update the PR title and description. Thanks! |
|
Updated the above, thanks! |
| private val nextBroadcastId = new AtomicLong(0) | ||
|
|
||
| private[broadcast] val cachedValues = { | ||
| new ReferenceMap(AbstractReferenceMap.HARD, AbstractReferenceMap.WEAK) |
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.
why is the key not a weak reference?
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.
Suppose the first thread to request the broadcast variable's value destroyed it's instance of the broadcast variable (which, I believe, is what will happen when that thread finishes processing it's partition) - if the key were a weak reference in the above cache it would become eligible for GC at that point. I'm reasonably certain at that point the associated key/value pair would be removed from the cache; in other words, if the key were a weak reference the key/value pair would be removed as soon as the key or value was garbage collected.
Note that I haven't used ReferenceMap extensively, so I could be wrong about the above - feel free to correct me if that's the case.
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.
what's the difference between this and "weak key, hard value"?
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.
This is the state of an executor at some point in time:
Cache: IdInstance1 => ValueInstance1
Thread1: TorrentBroadcastInstance1(broadcastId = IdInstance1, value = ValueInstance1)
Thread2: TorrentBroadcastInstance2(broadcastId = IdInstance2, value = ValueInstance1)
Thread3: TorrentBroadcastInstance3(broadcastId = IdInstance3, value = ValueInstance1)
Thread4: TorrentBroadcastInstance4(broadcastId = IdInstance4, value = ValueInstance1)
After some time Thread1 finishes process the partition it's working on and starts on the next - the state becomes:
Cache: IdInstance1 => ValueInstance1
Thread1: TorrentBroadcastInstance5(broadcastId = IdInstance5, value = ValueInstance1)
Thread2: TorrentBroadcastInstance2(broadcastId = IdInstance2, value = ValueInstance1)
Thread3: TorrentBroadcastInstance3(broadcastId = IdInstance3, value = ValueInstance1)
Thread4: TorrentBroadcastInstance4(broadcastId = IdInstance4, value = ValueInstance1)
At some point the GC destroys TorrentBroadcastInstance1. Now, if the key is a weak reference, the state becomes:
Cache: Empty
Thread1: TorrentBroadcastInstance5(broadcastId = IdInstance5, value = ValueInstance1)
Thread2: TorrentBroadcastInstance2(broadcastId = IdInstance2, value = ValueInstance1)
Thread3: TorrentBroadcastInstance3(broadcastId = IdInstance3, value = ValueInstance1)
Thread4: TorrentBroadcastInstance4(broadcastId = IdInstance4, value = ValueInstance1)
The next thread to finish processing a partition then creates a new instance of the broadcast value:
Cache: IdInstance6 => ValueInstance2
Thread1: TorrentBroadcastInstance5(broadcastId = IdInstance5, value = ValueInstance1)
Thread2: TorrentBroadcastInstance2(broadcastId = IdInstance2, value = ValueInstance1)
Thread3: TorrentBroadcastInstance6(broadcastId = IdInstance6, value = ValueInstance2)
Thread4: TorrentBroadcastInstance4(broadcastId = IdInstance4, value = ValueInstance1)
On the other hand, if the key is a strong reference the the value is weak, the cached value isn't eligible for GC above. As such, when Thread3 finishes processing it's partition and starts the next, the state becomes:
Cache: IdInstance1 => ValueInstance1
Thread1: TorrentBroadcastInstance5(broadcastId = IdInstance5, value = ValueInstance1)
Thread2: TorrentBroadcastInstance2(broadcastId = IdInstance2, value = ValueInstance1)
Thread3: TorrentBroadcastInstance6(broadcastId = IdInstance6, value = ValueInstance1)
Thread4: TorrentBroadcastInstance4(broadcastId = IdInstance4, value = ValueInstance1)
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.
ah i see, because we never get the key reference outside of the map, makes sense.
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.
This is what I found from the doc:
Hash-based Map implementation that allows mappings to be removed by the garbage collector.
When you construct a ReferenceMap, you can specify what kind of references are used to store the map's keys and values. If non-hard references are used, then the garbage collector can remove mappings if a key or value becomes unreachable, or if the JVM's memory is running low. For information on how the different reference types behave, see Reference.
It only mentions that non-hard references can be removed by GC, please correct me if I'm wrong.
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.
That should be fine even if that hard references not removed, since the memory consumption should be quite minor.
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.
BTW - welcome back @jerryshao ! long time no see!
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.
can remove mappings... That's why I say entries can be removed automatically.
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.
ohh, I see. Thanks!
| throw new SparkException(s"Failed to store $broadcastId in BlockManager") | ||
| val broadcastCache = SparkEnv.get.broadcastManager.cachedValues | ||
|
|
||
| Option(broadcastCache.get(broadcastId)).map(_.asInstanceOf[T]).getOrElse({ |
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: code style
...getOrElse {
xxx
}
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.
Fixed, thanks!
| val broadcastCache = SparkEnv.get.broadcastManager.cachedValues | ||
|
|
||
| Option(broadcastCache.get(broadcastId)).map(_.asInstanceOf[T]).getOrElse({ | ||
| setConf(SparkEnv.get.conf) |
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 to confirm, the following code is exactly same as before, right?
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.
Yes - everything within the getOrElse block is unchanged.
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, sorry - the cache update takes place within that block. With the exception of those blocks (lines 220-222 and lines 244-246), yes.
|
ok to test |
| throw new SparkException(s"Failed to store $broadcastId in BlockManager") | ||
| val broadcastCache = SparkEnv.get.broadcastManager.cachedValues | ||
|
|
||
| Option(broadcastCache.get(broadcastId)).map(_.asInstanceOf[T]).getOrElse { |
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 this ReferenceMap thread safe?
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.
ReferenceMap is not thread safe, no - however, all operations on broadcastCache occur within the context of a synchronized block; TorrentBroadcast.scala lines 208-254.
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 see, thanks for pointing out.
|
Test build #86010 has finished for PR 20183 at commit
|
|
Test build #86016 has finished for PR 20183 at commit
|
|
thanks, merging to master/2.3! |
…nces of broadcast variable values When resources happen to be constrained on an executor the first time a broadcast variable is instantiated it is persisted to disk by the BlockManager. Consequently, every subsequent call to TorrentBroadcast::readBroadcastBlock from other instances of that broadcast variable spawns another instance of the underlying value. That is, broadcast variables are spawned once per executor **unless** memory is constrained, in which case every instance of a broadcast variable is provided with a unique copy of the underlying value. This patch fixes the above by explicitly caching the underlying values using weak references in a ReferenceMap. Author: ho3rexqj <[email protected]> Closes #20183 from ho3rexqj/fix/cache-broadcast-values. (cherry picked from commit cbe7c6f) Signed-off-by: Wenchen Fan <[email protected]>
When resources happen to be constrained on an executor the first time a broadcast variable is instantiated it is persisted to disk by the BlockManager. Consequently, every subsequent call to TorrentBroadcast::readBroadcastBlock from other instances of that broadcast variable spawns another instance of the underlying value. That is, broadcast variables are spawned once per executor unless memory is constrained, in which case every instance of a broadcast variable is provided with a unique copy of the underlying value.
This patch fixes the above by explicitly caching the underlying values using weak references in a ReferenceMap.