Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
23 changes: 22 additions & 1 deletion core/src/main/scala/org/apache/spark/SparkContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, Job => NewHad
import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat}

import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.broadcast.{Broadcast, BroadcastMode}
import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil}
import org.apache.spark.input.{FixedLengthBinaryInputFormat, PortableDataStream, StreamInputFormat, WholeTextFileInputFormat}
import org.apache.spark.internal.Logging
Expand Down Expand Up @@ -1487,6 +1487,27 @@ class SparkContext(config: SparkConf) extends Logging {
bc
}

/**
* :: DeveloperApi ::
* Broadcast a read-only variable to the cluster, returning a
* [[org.apache.spark.broadcast.Broadcast]] object for reading it in distributed functions.
* The variable will be sent to each cluster only once.
*
* Notice that the RDD to be broadcasted should be cached and materilized first so we can
* access its data on the executors.
*/
@DeveloperApi
def broadcastRDDOnExecutor[T: ClassTag, U: ClassTag](
rdd: RDD[T], mode: BroadcastMode[T]): Broadcast[U] = {
assertNotStopped()
val bc = env.broadcastManager.newBroadcastOnExecutor[T, U](rdd, mode, isLocal)
rdd.broadcast(bc)
val callSite = getCallSite
logInfo("Created broadcast " + bc.id + " from " + callSite.shortForm)
cleaner.foreach(_.registerBroadcastForCleanup(bc))
bc
}

/**
* Add a file to be downloaded with this Spark job on every node.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,8 @@ package org.apache.spark.broadcast

import scala.reflect.ClassTag

import org.apache.spark.SecurityManager
import org.apache.spark.SparkConf
import org.apache.spark.{SecurityManager, SparkConf}
import org.apache.spark.rdd.RDD

/**
* An interface for all the broadcast implementations in Spark (to allow
Expand All @@ -40,6 +40,21 @@ private[spark] trait BroadcastFactory {
*/
def newBroadcast[T: ClassTag](value: T, isLocal: Boolean, id: Long): Broadcast[T]

/**
* Creates a new broadcast variable which is broadcasted on executors without collecting first
* to the driver.
*
* @param rdd the RDD to be broadcasted among executors
* @param mode the broadcast mode used to transform the result of RDD to broadcasted object
* @param isLocal whether we are in local mode (single JVM process)
* @param id unique id representing this broadcast variable
*/
def newBroadcastOnExecutor[T: ClassTag, U: ClassTag](
rdd: RDD[T],
mode: BroadcastMode[T],
isLocal: Boolean,
id: Long): Broadcast[U]

def unbroadcast(id: Long, removeFromDriver: Boolean, blocking: Boolean): Unit

def stop(): Unit
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import scala.reflect.ClassTag

import org.apache.spark.{SecurityManager, SparkConf}
import org.apache.spark.internal.Logging
import org.apache.spark.rdd.RDD

private[spark] class BroadcastManager(
val isDriver: Boolean,
Expand Down Expand Up @@ -56,6 +57,14 @@ private[spark] class BroadcastManager(
broadcastFactory.newBroadcast[T](value_, isLocal, nextBroadcastId.getAndIncrement())
}

def newBroadcastOnExecutor[T: ClassTag, U: ClassTag](
rdd_ : RDD[T],
mode: BroadcastMode[T],
isLocal: Boolean): Broadcast[U] = {
broadcastFactory.newBroadcastOnExecutor[T, U](rdd_, mode, isLocal,
nextBroadcastId.getAndIncrement())
}

def unbroadcast(id: Long, removeFromDriver: Boolean, blocking: Boolean) {
broadcastFactory.unbroadcast(id, removeFromDriver, blocking)
}
Expand Down
30 changes: 30 additions & 0 deletions core/src/main/scala/org/apache/spark/broadcast/BroadcastMode.scala
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
}
Original file line number Diff line number Diff line change
Expand Up @@ -209,6 +209,7 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long)
val blockManager = SparkEnv.get.blockManager
blockManager.getLocalValues(broadcastId) match {
case Some(blockResult) =>
// Found broadcasted value in local [[BlockManager]]. Use it directly.
if (blockResult.data.hasNext) {
val x = blockResult.data.next().asInstanceOf[T]
releaseLock(broadcastId)
Expand All @@ -217,6 +218,7 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long)
throw new SparkException(s"Failed to get locally stored broadcast data: $broadcastId")
}
case None =>
// Not found. Going to fetch the chunks of the broadcasted value from driver/executors.
logInfo("Started reading broadcast variable " + id)
val startTimeMs = System.currentTimeMillis()
val blocks = readBlocks()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,9 +20,10 @@ package org.apache.spark.broadcast
import scala.reflect.ClassTag

import org.apache.spark.{SecurityManager, SparkConf}
import org.apache.spark.rdd.RDD

/**
* A [[org.apache.spark.broadcast.Broadcast]] implementation that uses a BitTorrent-like
* A [[org.apache.spark.broadcast.BroadcastFactory]] implementation that uses a BitTorrent-like
* protocol to do a distributed transfer of the broadcasted data to the executors. Refer to
* [[org.apache.spark.broadcast.TorrentBroadcast]] for more details.
*/
Expand All @@ -34,6 +35,13 @@ private[spark] class TorrentBroadcastFactory extends BroadcastFactory {
new TorrentBroadcast[T](value_, id)
}

override def newBroadcastOnExecutor[T: ClassTag, U: ClassTag](
rdd: RDD[T],
mode: BroadcastMode[T],
isLocal: Boolean, id: Long): Broadcast[U] = {
new TorrentExecutorBroadcast[T, U](rdd, mode, id)
}

override def stop() { }

/**
Expand Down
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,
Copy link
Contributor

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 BroadcastBlockId and the correct ID will be removed. Maybe it would be good to add a test around unpersistance to verify its behaving as expected?

Copy link
Member Author

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.

Copy link
Member Author

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.

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

@holdenk holdenk Sep 28, 2016

Choose a reason for hiding this comment

The 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)
}
}
}
19 changes: 19 additions & 0 deletions core/src/main/scala/org/apache/spark/rdd/RDD.scala
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ import org.apache.spark._
import org.apache.spark.Partitioner._
import org.apache.spark.annotation.{DeveloperApi, Since}
import org.apache.spark.api.java.JavaRDD
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.internal.Logging
import org.apache.spark.partial.BoundedDouble
import org.apache.spark.partial.CountEvaluator
Expand Down Expand Up @@ -206,6 +207,24 @@ abstract class RDD[T: ClassTag](
*/
def cache(): this.type = persist()

/**
* Broadcast this RDD on executors. The executor side broadcast variable is created by
* [[SparkContext]]. This RDD should be cached and materialized first before calling on
* this method.
*/
private[spark] def broadcast[U: ClassTag](broadcasted: Broadcast[U]): Unit = {
// The RDD should be cached and materialized before it can be executor side broadcasted.
// We do the checking here.
if (storageLevel == StorageLevel.NONE) {
throw new SparkException("To broadcast this RDD on executors, it should be cached first.")
}
// Create the executor side broadcast object on executors.
mapPartitionsInternal { iter: Iterator[T] =>
broadcasted.value
Iterator.empty.asInstanceOf[Iterator[T]]
}.count
}

/**
* Mark the RDD as non-persistent, and remove all blocks for it from memory and disk.
*
Expand Down
Loading