-
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
Changes from all commits
45d5024
c3e2f42
8e13585
a6e5e81
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -206,36 +206,50 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) | |
|
|
||
| private def readBroadcastBlock(): T = Utils.tryOrIOException { | ||
| TorrentBroadcast.synchronized { | ||
| setConf(SparkEnv.get.conf) | ||
| val blockManager = SparkEnv.get.blockManager | ||
| blockManager.getLocalValues(broadcastId) match { | ||
| case Some(blockResult) => | ||
| if (blockResult.data.hasNext) { | ||
| val x = blockResult.data.next().asInstanceOf[T] | ||
| releaseLock(broadcastId) | ||
| x | ||
| } else { | ||
| throw new SparkException(s"Failed to get locally stored broadcast data: $broadcastId") | ||
| } | ||
| case None => | ||
| logInfo("Started reading broadcast variable " + id) | ||
| val startTimeMs = System.currentTimeMillis() | ||
| val blocks = readBlocks() | ||
| logInfo("Reading broadcast variable " + id + " took" + Utils.getUsedTimeMs(startTimeMs)) | ||
|
|
||
| try { | ||
| val obj = TorrentBroadcast.unBlockifyObject[T]( | ||
| blocks.map(_.toInputStream()), SparkEnv.get.serializer, compressionCodec) | ||
| // Store the merged copy in BlockManager so other tasks on this executor don't | ||
| // need to re-fetch it. | ||
| val storageLevel = StorageLevel.MEMORY_AND_DISK | ||
| if (!blockManager.putSingle(broadcastId, obj, storageLevel, tellMaster = false)) { | ||
| throw new SparkException(s"Failed to store $broadcastId in BlockManager") | ||
| val broadcastCache = SparkEnv.get.broadcastManager.cachedValues | ||
|
|
||
| Option(broadcastCache.get(broadcastId)).map(_.asInstanceOf[T]).getOrElse { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is this
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I see, thanks for pointing out. |
||
| setConf(SparkEnv.get.conf) | ||
| val blockManager = SparkEnv.get.blockManager | ||
| blockManager.getLocalValues(broadcastId) match { | ||
| case Some(blockResult) => | ||
| if (blockResult.data.hasNext) { | ||
| val x = blockResult.data.next().asInstanceOf[T] | ||
| releaseLock(broadcastId) | ||
|
|
||
| if (x != null) { | ||
| broadcastCache.put(broadcastId, x) | ||
| } | ||
|
|
||
| x | ||
| } else { | ||
| throw new SparkException(s"Failed to get locally stored broadcast data: $broadcastId") | ||
| } | ||
| obj | ||
| } finally { | ||
| blocks.foreach(_.dispose()) | ||
| } | ||
| case None => | ||
| logInfo("Started reading broadcast variable " + id) | ||
| val startTimeMs = System.currentTimeMillis() | ||
| val blocks = readBlocks() | ||
| logInfo("Reading broadcast variable " + id + " took" + Utils.getUsedTimeMs(startTimeMs)) | ||
|
|
||
| try { | ||
| val obj = TorrentBroadcast.unBlockifyObject[T]( | ||
| blocks.map(_.toInputStream()), SparkEnv.get.serializer, compressionCodec) | ||
| // Store the merged copy in BlockManager so other tasks on this executor don't | ||
| // need to re-fetch it. | ||
| val storageLevel = StorageLevel.MEMORY_AND_DISK | ||
| if (!blockManager.putSingle(broadcastId, obj, storageLevel, tellMaster = false)) { | ||
| throw new SparkException(s"Failed to store $broadcastId in BlockManager") | ||
| } | ||
|
|
||
| if (obj != null) { | ||
| broadcastCache.put(broadcastId, obj) | ||
| } | ||
|
|
||
| obj | ||
| } finally { | ||
| blocks.foreach(_.dispose()) | ||
| } | ||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
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:
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!
Uh oh!
There was an error while loading. Please reload this page.
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!