Skip to content

Commit

Permalink
Add tests for unpersisting broadcast
Browse files Browse the repository at this point in the history
There is not currently a way to query the blocks on the executors,
an operation that is deceptively simple to accomplish. This commit
adds this mechanism in order to verify that blocks are in fact
persisted/unpersisted on the executors in the tests.
  • Loading branch information
andrewor14 committed Mar 27, 2014
1 parent 544ac86 commit e95479c
Show file tree
Hide file tree
Showing 9 changed files with 309 additions and 63 deletions.
16 changes: 13 additions & 3 deletions core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala
Original file line number Diff line number Diff line change
Expand Up @@ -48,16 +48,26 @@ import java.io.Serializable
* @tparam T Type of the data contained in the broadcast variable.
*/
abstract class Broadcast[T](val id: Long) extends Serializable {

/**
* Whether this Broadcast is actually usable. This should be false once persisted state is
* removed from the driver.
*/
protected var isValid: Boolean = true

def value: T

/**
* Remove all persisted state associated with this broadcast.
* Remove all persisted state associated with this broadcast. Overriding implementations
* should set isValid to false if persisted state is also removed from the driver.
*
* @param removeFromDriver Whether to remove state from the driver.
* If true, the resulting broadcast should no longer be valid.
*/
def unpersist(removeFromDriver: Boolean)

// We cannot have an abstract readObject here due to some weird issues with
// readObject having to be 'private' in sub-classes.
// We cannot define abstract readObject and writeObject here due to some weird issues
// with these methods having to be 'private' in sub-classes.

override def toString = "Broadcast(" + id + ")"
}
13 changes: 10 additions & 3 deletions core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,8 @@

package org.apache.spark.broadcast

import java.io.{File, FileOutputStream, ObjectInputStream, OutputStream}
import java.net.{URL, URLConnection, URI}
import java.io.{File, FileOutputStream, ObjectInputStream, ObjectOutputStream, OutputStream}
import java.net.{URI, URL, URLConnection}
import java.util.concurrent.TimeUnit

import it.unimi.dsi.fastutil.io.{FastBufferedInputStream, FastBufferedOutputStream}
Expand Down Expand Up @@ -49,10 +49,17 @@ private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolea
* @param removeFromDriver Whether to remove state from the driver.
*/
override def unpersist(removeFromDriver: Boolean) {
isValid = !removeFromDriver
HttpBroadcast.unpersist(id, removeFromDriver)
}

// Called by JVM when deserializing an object
// Used by the JVM when serializing this object
private def writeObject(out: ObjectOutputStream) {
assert(isValid, "Attempted to serialize a broadcast variable that has been destroyed!")
out.defaultWriteObject()
}

// Used by the JVM when deserializing this object
private def readObject(in: ObjectInputStream) {
in.defaultReadObject()
HttpBroadcast.synchronized {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,12 +17,12 @@

package org.apache.spark.broadcast

import java.io._
import java.io.{ByteArrayInputStream, ObjectInputStream, ObjectOutputStream}

import scala.math
import scala.util.Random

import org.apache.spark._
import org.apache.spark.{Logging, SparkConf, SparkEnv, SparkException}
import org.apache.spark.storage.{BroadcastBlockId, BroadcastHelperBlockId, StorageLevel}
import org.apache.spark.util.Utils

Expand Down Expand Up @@ -76,10 +76,17 @@ private[spark] class TorrentBroadcast[T](@transient var value_ : T, isLocal: Boo
* @param removeFromDriver Whether to remove state from the driver.
*/
override def unpersist(removeFromDriver: Boolean) {
isValid = !removeFromDriver
TorrentBroadcast.unpersist(id, removeFromDriver)
}

// Called by JVM when deserializing an object
// Used by the JVM when serializing this object
private def writeObject(out: ObjectOutputStream) {
assert(isValid, "Attempted to serialize a broadcast variable that has been destroyed!")
out.defaultWriteObject()
}

// Used by the JVM when deserializing this object
private def readObject(in: ObjectInputStream) {
in.defaultReadObject()
TorrentBroadcast.synchronized {
Expand Down
20 changes: 9 additions & 11 deletions core/src/main/scala/org/apache/spark/storage/BlockManager.scala
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@ import akka.actor.{ActorSystem, Cancellable, Props}
import it.unimi.dsi.fastutil.io.{FastBufferedOutputStream, FastByteArrayOutputStream}
import sun.nio.ch.DirectBuffer

import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv, SparkException, MapOutputTracker}
import org.apache.spark._
import org.apache.spark.io.CompressionCodec
import org.apache.spark.network._
import org.apache.spark.serializer.Serializer
Expand Down Expand Up @@ -58,7 +58,7 @@ private[spark] class BlockManager(

private val blockInfo = new TimeStampedHashMap[BlockId, BlockInfo]

private[storage] val memoryStore: BlockStore = new MemoryStore(this, maxMemory)
private[storage] val memoryStore = new MemoryStore(this, maxMemory)
private[storage] val diskStore = new DiskStore(this, diskBlockManager)

// If we use Netty for shuffle, start a new Netty-based shuffle sender service.
Expand Down Expand Up @@ -210,9 +210,9 @@ private[spark] class BlockManager(
}

/**
* Get storage level of local block. If no info exists for the block, then returns null.
* Get storage level of local block. If no info exists for the block, return None.
*/
def getLevel(blockId: BlockId): StorageLevel = blockInfo.get(blockId).map(_.level).orNull
def getLevel(blockId: BlockId): Option[StorageLevel] = blockInfo.get(blockId).map(_.level)

/**
* Tell the master about the current storage status of a block. This will send a block update
Expand Down Expand Up @@ -496,9 +496,8 @@ private[spark] class BlockManager(

/**
* A short circuited method to get a block writer that can write data directly to disk.
* The Block will be appended to the File specified by filename.
* This is currently used for writing shuffle files out. Callers should handle error
* cases.
* The Block will be appended to the File specified by filename. This is currently used for
* writing shuffle files out. Callers should handle error cases.
*/
def getDiskWriter(
blockId: BlockId,
Expand Down Expand Up @@ -816,8 +815,7 @@ private[spark] class BlockManager(
* @return The number of blocks removed.
*/
def removeRdd(rddId: Int): Int = {
// TODO: Instead of doing a linear scan on the blockInfo map, create another map that maps
// from RDD.id to blocks.
// TODO: Avoid a linear scan by creating another mapping of RDD.id to blocks.
logInfo("Removing RDD " + rddId)
val blocksToRemove = blockInfo.keys.flatMap(_.asRDDId).filter(_.rddId == rddId)
blocksToRemove.foreach { blockId => removeBlock(blockId, tellMaster = false) }
Expand All @@ -827,13 +825,13 @@ private[spark] class BlockManager(
/**
* Remove all blocks belonging to the given broadcast.
*/
def removeBroadcast(broadcastId: Long) {
def removeBroadcast(broadcastId: Long, removeFromDriver: Boolean) {
logInfo("Removing broadcast " + broadcastId)
val blocksToRemove = blockInfo.keys.filter(_.isBroadcast).collect {
case bid: BroadcastBlockId if bid.broadcastId == broadcastId => bid
case bid: BroadcastHelperBlockId if bid.broadcastId.broadcastId == broadcastId => bid
}
blocksToRemove.foreach { blockId => removeBlock(blockId) }
blocksToRemove.foreach { blockId => removeBlock(blockId, removeFromDriver) }
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -147,6 +147,24 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log
askDriverWithReply[Array[StorageStatus]](GetStorageStatus)
}

/**
* Mainly for testing. Ask the driver to query all executors for their storage levels
* regarding this block. This provides an avenue for the driver to learn the storage
* levels of blocks it has not been informed of.
*
* WARNING: This could lead to deadlocks if there are any outstanding messages the
* executors are already expecting from the driver. In this case, while the driver is
* waiting for the executors to respond to its GetStorageLevel query, the executors
* are also waiting for a response from the driver to a prior message.
*
* The interim solution is to wait for a brief window of time to pass before asking.
* This should suffice, since this mechanism is largely introduced for testing only.
*/
def askForStorageLevels(blockId: BlockId, waitTimeMs: Long = 1000) = {
Thread.sleep(waitTimeMs)
askDriverWithReply[Map[BlockManagerId, StorageLevel]](AskForStorageLevels(blockId))
}

/** Stop the driver actor, called only on the Spark driver node */
def stop() {
if (driverActor != null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import java.util.{HashMap => JHashMap}

import scala.collection.mutable
import scala.collection.JavaConversions._
import scala.concurrent.Future
import scala.concurrent.{Await, Future}
import scala.concurrent.duration._

import akka.actor.{Actor, ActorRef, Cancellable}
Expand Down Expand Up @@ -126,6 +126,9 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus
case HeartBeat(blockManagerId) =>
sender ! heartBeat(blockManagerId)

case AskForStorageLevels(blockId) =>
sender ! askForStorageLevels(blockId)

case other =>
logWarning("Got unknown message: " + other)
}
Expand Down Expand Up @@ -158,6 +161,11 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus
blockManagerInfo.values.foreach { bm => bm.slaveActor ! removeMsg }
}

/**
* Delegate RemoveBroadcast messages to each BlockManager because the master may not notified
* of all broadcast blocks. If removeFromDriver is false, broadcast blocks are only removed
* from the executors, but not from the driver.
*/
private def removeBroadcast(broadcastId: Long, removeFromDriver: Boolean) {
// TODO(aor): Consolidate usages of <driver>
val removeMsg = RemoveBroadcast(broadcastId)
Expand Down Expand Up @@ -246,6 +254,19 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus
}.toArray
}

// For testing. Ask all block managers for the given block's local storage level, if any.
private def askForStorageLevels(blockId: BlockId): Map[BlockManagerId, StorageLevel] = {
val getStorageLevel = GetStorageLevel(blockId)
blockManagerInfo.values.flatMap { info =>
val future = info.slaveActor.ask(getStorageLevel)(akkaTimeout)
val result = Await.result(future, akkaTimeout)
if (result != null) {
// If the block does not exist on the slave, the slave replies None
result.asInstanceOf[Option[StorageLevel]].map { reply => (info.blockManagerId, reply) }
} else None
}.toMap
}

private def register(id: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) {
if (!blockManagerInfo.contains(id)) {
blockManagerIdByExecutor.get(id.executorId) match {
Expand Down Expand Up @@ -329,6 +350,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus
// Note that this logic will select the same node multiple times if there aren't enough peers
Array.tabulate[BlockManagerId](size) { i => peers((selfIndex + i + 1) % peers.length) }.toSeq
}

}


Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,9 @@ private[storage] object BlockManagerMessages {
case class RemoveBroadcast(broadcastId: Long, removeFromDriver: Boolean = true)
extends ToBlockManagerSlave

// For testing. Ask the slave for the block's storage level.
case class GetStorageLevel(blockId: BlockId) extends ToBlockManagerSlave


//////////////////////////////////////////////////////////////////////////////////
// Messages from slaves to the master.
Expand Down Expand Up @@ -116,4 +119,8 @@ private[storage] object BlockManagerMessages {
case object ExpireDeadHosts extends ToBlockManagerMaster

case object GetStorageStatus extends ToBlockManagerMaster

// For testing. Have the master ask all slaves for the given block's storage level.
case class AskForStorageLevels(blockId: BlockId) extends ToBlockManagerMaster

}
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,10 @@ class BlockManagerSlaveActor(
mapOutputTracker.unregisterShuffle(shuffleId)
}

case RemoveBroadcast(broadcastId, _) =>
blockManager.removeBroadcast(broadcastId)
case RemoveBroadcast(broadcastId, removeFromDriver) =>
blockManager.removeBroadcast(broadcastId, removeFromDriver)

case GetStorageLevel(blockId) =>
sender ! blockManager.getLevel(blockId)
}
}
Loading

0 comments on commit e95479c

Please sign in to comment.