-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-17556][SQL] Executor side broadcast for broadcast joins #15178
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
Closed
Closed
Changes from all commits
Commits
Show all changes
12 commits
Select commit
Hold shift + click to select a range
57987d1
Broadcast on executors.
viirya f50cf31
Add developer api and a config to enable executor-side broadcast. Ref…
viirya 3494728
Add test cases. Refactoring.
viirya 8744fe9
Make executor side broadast more reliable against RDD getting unpersi…
viirya 17b4470
Refactor BroadcastMode.
viirya 0440cc7
Fix test.
viirya 1b499d1
Merge remote-tracking branch 'upstream/master' into broadcast-on-exec…
viirya 34a49d5
Merge remote-tracking branch 'upstream/master' into broadcast-on-exec…
viirya ecebb3f
Fix typo.
viirya ef987ae
Fix test.
viirya 93ecabb
Merge remote-tracking branch 'upstream/master' into broadcast-on-exec…
viirya a1f2faa
Merge remote-tracking branch 'upstream/master' into broadcast-on-exec…
viirya File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
30 changes: 30 additions & 0 deletions
30
core/src/main/scala/org/apache/spark/broadcast/BroadcastMode.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,30 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.spark.broadcast | ||
|
|
||
| /** | ||
| * The trait used in executor side broadcast. The implementation of `transform` identify the shape | ||
| * in which the results of a RDD are broadcasted. | ||
| * | ||
| * @tparam T The type of RDD elements. | ||
| */ | ||
| trait BroadcastMode[T] extends Serializable { | ||
| def transform(rows: Array[T]): Any | ||
| def transform(rows: Iterator[T], sizeHint: Option[Long]): Any | ||
| def canonicalized: BroadcastMode[T] = this | ||
| } |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
178 changes: 178 additions & 0 deletions
178
core/src/main/scala/org/apache/spark/broadcast/TorrentExecutorBroadcast.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,178 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.spark.broadcast | ||
|
|
||
| import java.io.ObjectOutputStream | ||
|
|
||
| import scala.collection.mutable.ArrayBuffer | ||
| import scala.reflect.ClassTag | ||
| import scala.util.Random | ||
|
|
||
| import org.apache.spark._ | ||
| import org.apache.spark.internal.Logging | ||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.storage.{BlockId, BlockResult, BroadcastBlockId, RDDBlockId, StorageLevel} | ||
| import org.apache.spark.util.Utils | ||
|
|
||
| /** | ||
| * A BitTorrent-like implementation of [[org.apache.spark.broadcast.Broadcast]]. | ||
| * | ||
| * Different to [[TorrentBroadcast]], this implementation doesn't divide the object to broadcast. | ||
| * In contrast, this implementation performs broadcast on executor side for a RDD. So the results | ||
| * of the RDD does not need to collect first back to the driver before broadcasting. | ||
| * | ||
| * The mechanism is as follows: | ||
| * | ||
| * On each executor, the executor first attempts to fetch the object from its BlockManager. If | ||
| * it doesn not exist, it then uses remote fetches to fetch the blocks of the RDD from other | ||
| * executors if available. Once it gets the blocks, it puts the blocks in its own BlockManager, | ||
| * ready for other executors to fetch from. | ||
| * | ||
| * @tparam T The type of the element of RDD to be broadcasted. | ||
| * @tparam U The type of object transformed from the collection of elements of the RDD. | ||
| * | ||
| * @param rdd The RDD to be broadcasted on executors. | ||
| * @param mode The [[org.apache.spark.broadcast.BroadcastMode]] object used to transform the result | ||
| * of RDD to the object which will be stored in block manager. | ||
| * @param id A unique identifier for the broadcast variable. | ||
| */ | ||
| private[spark] class TorrentExecutorBroadcast[T: ClassTag, U: ClassTag]( | ||
| @transient private val rdd: RDD[T], | ||
| mode: BroadcastMode[T], | ||
| id: Long) extends Broadcast[U](id) with Logging with Serializable { | ||
|
|
||
| // Total number of blocks this broadcast variable contains. | ||
| private val numBlocks: Int = rdd.getNumPartitions | ||
| // The id of the RDD to be broadcasted on executors. | ||
| private val rddId: Int = rdd.id | ||
|
|
||
| /** | ||
| * Value of the broadcast object on executors. This is reconstructed by [[readBroadcastBlock]], | ||
| * which builds this value by reading blocks from other executors. | ||
| */ | ||
| @transient private lazy val _value: U = readBroadcastBlock() | ||
|
|
||
| private val broadcastId = BroadcastBlockId(id) | ||
|
|
||
| override protected def getValue() = { | ||
| _value | ||
| } | ||
|
|
||
| /** Fetch torrent blocks from other executors. */ | ||
| private def readBlocks(): Array[T] = { | ||
| // 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 chunks from this executor as well. | ||
| val blocks = new Array[Array[T]](numBlocks) | ||
| val bm = SparkEnv.get.blockManager | ||
|
|
||
| for (pid <- Random.shuffle(Seq.range(0, numBlocks))) { | ||
| val pieceId = RDDBlockId(rddId, pid) | ||
| // First try getLocalValues because there is a chance that previous attempts to fetch the | ||
| // broadcast blocks have already fetched some of the blocks. In that case, some blocks | ||
| // would be available locally (on this executor). | ||
| bm.getLocalValues(pieceId) match { | ||
| case Some(block: BlockResult) => | ||
| blocks(pid) = block.data.asInstanceOf[Iterator[T]].toArray | ||
| case None => | ||
| bm.get[T](pieceId) match { | ||
| case Some(b) => | ||
| val data = b.data.asInstanceOf[Iterator[T]].toArray | ||
| // We found the block from remote executors' BlockManager, so put the block | ||
| // in this executor's BlockManager. | ||
| if (!bm.putIterator(pieceId, data.toIterator, | ||
| StorageLevel.MEMORY_AND_DISK_SER, tellMaster = true)) { | ||
| throw new SparkException( | ||
| s"Failed to store $pieceId of $broadcastId in local BlockManager") | ||
| } | ||
| blocks(pid) = data | ||
| case None => | ||
| throw new SparkException(s"Failed to get $pieceId of $broadcastId") | ||
| } | ||
| } | ||
| } | ||
| blocks.flatMap(x => x) | ||
| } | ||
|
|
||
| /** | ||
| * Remove all persisted state associated with this Torrent broadcast on the executors. | ||
| */ | ||
| override protected def doUnpersist(blocking: Boolean) { | ||
| TorrentBroadcast.unpersist(id, removeFromDriver = false, blocking) | ||
|
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. Similar comment to as above - does this do what we want? |
||
| } | ||
|
|
||
| /** | ||
| * Remove all persisted state associated with this Torrent broadcast on the executors | ||
| * and driver. | ||
| */ | ||
| override protected def doDestroy(blocking: Boolean) { | ||
| TorrentBroadcast.unpersist(id, removeFromDriver = true, blocking) | ||
| } | ||
|
|
||
| /** Used by the JVM when serializing this object. */ | ||
| private def writeObject(out: ObjectOutputStream): Unit = Utils.tryOrIOException { | ||
| assertValid() | ||
| out.defaultWriteObject() | ||
| } | ||
|
|
||
| private def readBroadcastBlock(): U = Utils.tryOrIOException { | ||
| TorrentBroadcast.synchronized { | ||
| val blockManager = SparkEnv.get.blockManager | ||
| blockManager.getLocalValues(broadcastId).map(_.data.next()) match { | ||
| case Some(x) => | ||
| // Found broadcasted value in local [[BlockManager]]. Use it directly. | ||
| releaseLock(broadcastId) | ||
| x.asInstanceOf[U] | ||
|
|
||
| case None => | ||
| // Not found. Going to fetch the chunks of the broadcasted value from executors. | ||
| logInfo("Started reading broadcast variable " + id) | ||
| val startTimeMs = System.currentTimeMillis() | ||
| val rawInput = readBlocks() | ||
| logInfo("Reading broadcast variable " + id + " took" + Utils.getUsedTimeMs(startTimeMs)) | ||
|
|
||
| val obj = mode.transform(rawInput.toArray).asInstanceOf[U] | ||
| // 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") | ||
| } | ||
| obj | ||
| } | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * If running in a task, register the given block's locks for release upon task completion. | ||
| * Otherwise, if not running in a task then immediately release the lock. | ||
| */ | ||
| private def releaseLock(blockId: BlockId): Unit = { | ||
| val blockManager = SparkEnv.get.blockManager | ||
| Option(TaskContext.get()) match { | ||
| case Some(taskContext) => | ||
| taskContext.addTaskCompletionListener(_ => blockManager.releaseLock(blockId)) | ||
| case None => | ||
| // This should only happen on the driver, where broadcast variables may be accessed | ||
| // outside of running tasks (e.g. when computing rdd.partitions()). In order to allow | ||
| // broadcast variables to be garbage collected we need to free the reference here | ||
| // which is slightly unsafe but is technically okay because broadcast variables aren't | ||
| // stored off-heap. | ||
| blockManager.releaseLock(blockId) | ||
| } | ||
| } | ||
| } | ||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
So were storing an RDD pieceId here, but I think in unpersist only things with
BroadcastBlockIdand the correct ID will be removed. Maybe it would be good to add a test around unpersistance to verify its behaving as expected?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.
For RDD, there is a cleaning mechanism that the persisted pieces will be removed once the RDD is not referred. Because we fetch and use RDD pieces here instead of broadcast pieces in driver side broadcast, I think it should be fine to deliver the cleaning to current mechanism.
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.
One solution might be to store the fetched RDD pieces with broadcast piece ID, so in unpersist we can remove all the fetched pieces. However, then we must consider fetch both RDD piece IDs broadcast IDs from other executors under the BitTorrent-like approach. Thus I would prefer the above way and let current cleaning mechanism do its work.