Skip to content

Commit

Permalink
Fixed bug in get peers and unit tests to test get-peers and replicati…
Browse files Browse the repository at this point in the history
…on under executor churn.
  • Loading branch information
tdas committed Sep 11, 2014
1 parent 9f0ac9f commit d081bf6
Show file tree
Hide file tree
Showing 5 changed files with 155 additions and 41 deletions.
37 changes: 31 additions & 6 deletions core/src/main/scala/org/apache/spark/storage/BlockManager.scala
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ import org.apache.spark.network._
import org.apache.spark.serializer.Serializer
import org.apache.spark.shuffle.ShuffleManager
import org.apache.spark.util._
import scala.collection.mutable


private[spark] sealed trait BlockValues
Expand Down Expand Up @@ -111,7 +112,8 @@ private[spark] class BlockManager(
MetadataCleanerType.BLOCK_MANAGER, this.dropOldNonBroadcastBlocks, conf)
private val broadcastCleaner = new MetadataCleaner(
MetadataCleanerType.BROADCAST_VARS, this.dropOldBroadcastBlocks, conf)

private val cachedPeers = new ArrayBuffer[BlockManagerId]
private var lastPeerFetchTime = 0L
initialize()

/* The compression codec to use. Note that the "lazy" val is necessary because we want to delay
Expand Down Expand Up @@ -786,20 +788,42 @@ private[spark] class BlockManager(
updatedBlocks
}

/**
* Get peer block managers in the system.
*/
private def getPeers(numPeers: Int): Seq[BlockManagerId] = cachedPeers.synchronized {
val currentTime = System.currentTimeMillis
// If cache is empty or has insufficient number of peers, fetch from master
if (cachedPeers.isEmpty || numPeers > cachedPeers.size) {
cachedPeers.clear()
cachedPeers ++= master.getPeers(blockManagerId)
logDebug("Fetched peers from master: " + cachedPeers.mkString("[", ",", "]"))
lastPeerFetchTime = System.currentTimeMillis
}
if (numPeers > cachedPeers.size) {
// if enough peers cannot be provided, return all of them
logDebug(s"Not enough peers - cached peers = ${cachedPeers.size}, required peers = $numPeers")
cachedPeers
} else {
cachedPeers.take(numPeers)
}
}

/**
* Replicate block to another node.
*/
@volatile var cachedPeers: Seq[BlockManagerId] = null
private def replicate(blockId: BlockId, data: ByteBuffer, level: StorageLevel): Unit = {
val tLevel = StorageLevel(
level.useDisk, level.useMemory, level.useOffHeap, level.deserialized, 1)
if (cachedPeers == null) {
cachedPeers = master.getPeers(blockManagerId, level.replication - 1)
val selectedPeers = getPeers(level.replication - 1)
if (selectedPeers.size < level.replication - 1) {
logWarning(s"Failed to replicate block to ${level.replication - 1} peer(s) " +
s"as only ${selectedPeers.size} peer(s) were found")
}
for (peer: BlockManagerId <- cachedPeers) {
selectedPeers.foreach { peer =>
val start = System.nanoTime
data.rewind()
logDebug(s"Try to replicate $blockId once; The size of the data is ${data.limit()} Bytes. " +
logDebug(s"Try to replicate $blockId once; The size of the data is ${data.limit()} bytes. " +
s"To node: $peer")

try {
Expand All @@ -808,6 +832,7 @@ private[spark] class BlockManager(
} catch {
case e: Exception =>
logError(s"Failed to replicate block to $peer", e)
cachedPeers.synchronized { cachedPeers.clear() }
}

logDebug("Replicating BlockId %s once used %fs; The size of the data is %d bytes."
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -84,13 +84,8 @@ class BlockManagerMaster(
}

/** Get ids of other nodes in the cluster from the driver */
def getPeers(blockManagerId: BlockManagerId, numPeers: Int): Seq[BlockManagerId] = {
val result = askDriverWithReply[Seq[BlockManagerId]](GetPeers(blockManagerId, numPeers))
if (result.length != numPeers) {
throw new SparkException(
"Error getting peers, only got " + result.size + " instead of " + numPeers)
}
result
def getPeers(blockManagerId: BlockManagerId): Seq[BlockManagerId] = {
askDriverWithReply[Seq[BlockManagerId]](GetPeers(blockManagerId))
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -83,8 +83,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus
case GetLocationsMultipleBlockIds(blockIds) =>
sender ! getLocationsMultipleBlockIds(blockIds)

case GetPeers(blockManagerId, size) =>
sender ! getPeers(blockManagerId, size)
case GetPeers(blockManagerId) =>
sender ! getPeers(blockManagerId)

case GetMemoryStatus =>
sender ! memoryStatus
Expand Down Expand Up @@ -403,16 +403,20 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus
blockIds.map(blockId => getLocations(blockId))
}

private def getPeers(blockManagerId: BlockManagerId, size: Int): Seq[BlockManagerId] = {
val peers: Array[BlockManagerId] = blockManagerInfo.keySet.toArray

val selfIndex = peers.indexOf(blockManagerId)
/** Get the list of the peers of the given block manager */
private def getPeers(blockManagerId: BlockManagerId): Seq[BlockManagerId] = {
val blockManagerIds = blockManagerInfo.keySet.toArray
val selfIndex = blockManagerIds.indexOf(blockManagerId)
if (selfIndex == -1) {
throw new SparkException("Self index for " + blockManagerId + " not found")
logError("Self index for " + blockManagerId + " not found")
Seq.empty
} else {
// If the blockManagerIds is [ id1 id2 id3 id4 id5 ] and the blockManagerId is id2
// Then this code will return the list [ id3 id4 id5 id1 ]
Array.tabulate[BlockManagerId](blockManagerIds.size - 1) { i =>
blockManagerIds((selfIndex + i + 1) % blockManagerIds.size)
}
}

// 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 @@ -88,7 +88,7 @@ private[spark] object BlockManagerMessages {

case class GetLocationsMultipleBlockIds(blockIds: Array[BlockId]) extends ToBlockManagerMaster

case class GetPeers(blockManagerId: BlockManagerId, size: Int) extends ToBlockManagerMaster
case class GetPeers(blockManagerId: BlockManagerId) extends ToBlockManagerMaster

case class RemoveExecutor(execId: String) extends ToBlockManagerMaster

Expand Down
124 changes: 107 additions & 17 deletions core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -49,10 +49,6 @@ import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat
import org.apache.spark.util.{AkkaUtils, ByteBufferInputStream, SizeEstimator, Utils}
import org.apache.spark.storage.StorageLevel._
import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat
import org.apache.spark.storage.BroadcastBlockId
import org.apache.spark.storage.RDDBlockId
import org.apache.spark.storage.ShuffleBlockId
import org.apache.spark.storage.TestBlockId


class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter
Expand Down Expand Up @@ -126,7 +122,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter

System.clearProperty("spark.test.useCompressedOops")
}
/*

test("StorageLevel object caching") {
val level1 = StorageLevel(false, false, false, false, 3)
val level2 = StorageLevel(false, false, false, false, 3) // this should return the same object as level1
Expand Down Expand Up @@ -195,7 +191,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter
store = makeBlockManager(2000, "exec1")
store2 = makeBlockManager(2000, "exec2")

val peers = master.getPeers(store.blockManagerId, 1)
val peers = master.getPeers(store.blockManagerId)
assert(peers.size === 1, "master did not return the other manager as a peer")
assert(peers.head === store2.blockManagerId, "peer returned by master is not the other manager")

Expand Down Expand Up @@ -454,7 +450,6 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter
val list2DiskGet = store.get("list2disk")
assert(list2DiskGet.isDefined, "list2memory expected to be in store")
assert(list2DiskGet.get.data.size === 3)
System.out.println(list2DiskGet)
// We don't know the exact size of the data on disk, but it should certainly be > 0.
assert(list2DiskGet.get.inputMetrics.bytesRead > 0)
assert(list2DiskGet.get.inputMetrics.readMethod === DataReadMethod.Disk)
Expand Down Expand Up @@ -1234,14 +1229,52 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter
assert(unrollMemoryAfterB6 === unrollMemoryAfterB4)
assert(unrollMemoryAfterB7 === unrollMemoryAfterB4)
}
*/

test("get peers with store addition and removal") {
val numStores = 4
val stores = (1 to numStores - 1).map { i => makeBlockManager(1000, s"store$i") }
try {
val storeIds = stores.map { _.blockManagerId }.toSet
assert(master.getPeers(stores(0).blockManagerId).toSet ===
storeIds.filterNot { _ == stores(0).blockManagerId })
assert(master.getPeers(stores(1).blockManagerId).toSet ===
storeIds.filterNot { _ == stores(1).blockManagerId })
assert(master.getPeers(stores(2).blockManagerId).toSet ===
storeIds.filterNot { _ == stores(2).blockManagerId })

// Add a new store and test whether get peers returns it
val newStore = makeBlockManager(1000, s"store$numStores")
assert(master.getPeers(stores(0).blockManagerId).toSet ===
storeIds.filterNot { _ == stores(0).blockManagerId } + newStore.blockManagerId)
assert(master.getPeers(stores(1).blockManagerId).toSet ===
storeIds.filterNot { _ == stores(1).blockManagerId } + newStore.blockManagerId)
assert(master.getPeers(stores(2).blockManagerId).toSet ===
storeIds.filterNot { _ == stores(2).blockManagerId } + newStore.blockManagerId)
assert(master.getPeers(newStore.blockManagerId).toSet === storeIds)

// Remove a store and test whether get peers returns it
val storeIdToRemove = stores(0).blockManagerId
master.removeExecutor(storeIdToRemove.executorId)
assert(!master.getPeers(stores(1).blockManagerId).contains(storeIdToRemove))
assert(!master.getPeers(stores(2).blockManagerId).contains(storeIdToRemove))
assert(!master.getPeers(newStore.blockManagerId).contains(storeIdToRemove))

// Test whether asking for peers of a unregistered block manager id returns empty list
assert(master.getPeers(stores(0).blockManagerId).isEmpty)
assert(master.getPeers(BlockManagerId("", "", 1)).isEmpty)
} finally {
stores.foreach { _.stop() }
}
}

test("block replication - 2x") {
testReplication(2,
Seq(MEMORY_ONLY, MEMORY_ONLY_SER, DISK_ONLY, MEMORY_AND_DISK_2, MEMORY_AND_DISK_SER_2)
)
}

test("block replication - 3x") {
// Generate storage levels with 3x replication
val storageLevels = {
Seq(MEMORY_ONLY, MEMORY_ONLY_SER, DISK_ONLY, MEMORY_AND_DISK, MEMORY_AND_DISK_SER).map {
level => StorageLevel(
Expand All @@ -1252,18 +1285,77 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter
}

test("block replication - mixed between 1x to 5x") {
// Generate storage levels with varying replication
val storageLevels = Seq(
MEMORY_ONLY,
MEMORY_ONLY_SER_2,
StorageLevel(true, true, false, true, 3),
StorageLevel(true, false, false, false, 4),
StorageLevel(false, false, false, false, 5),
StorageLevel(true, false, false, false, 4),
StorageLevel(true, true, false, true, 3),
StorageLevel(true, false, false, false, 3),
StorageLevel(true, true, false, true, 4),
StorageLevel(true, true, false, false, 5),
StorageLevel(true, true, false, true, 4),
StorageLevel(true, false, false, false, 3),
MEMORY_ONLY_SER_2,
MEMORY_ONLY
)
testReplication(3, storageLevels)
testReplication(5, storageLevels)
}

test("block replication with addition and removal of executors") {
val blockSize = 1000
val storeSize = 10000
val allStores = new ArrayBuffer[BlockManager]()

try {
val initialStores = (1 to 2).map { i => makeBlockManager(storeSize, s"store$i") }
allStores ++= initialStores

// 2x replication works
initialStores(0).putSingle("a1", new Array[Byte](blockSize), StorageLevel.MEMORY_AND_DISK_2)
assert(master.getLocations("a1").size === 2)

// 3x replication should only replicate 2x
initialStores(0).putSingle(
"a2", new Array[Byte](blockSize), StorageLevel(true, true, false, true, 3))
assert(master.getLocations("a2").size === 2)

val newStore1 = makeBlockManager(storeSize, s"newstore1")
allStores += newStore1

// 3x replication should work now
initialStores(0).putSingle(
"a3", new Array[Byte](blockSize), StorageLevel(true, true, false, true, 3))
assert(master.getLocations("a3").size === 3)

// 4x replication should only replicate 3x
initialStores(0).putSingle(
"a4", new Array[Byte](blockSize), StorageLevel(true, true, false, true, 4))
assert(master.getLocations("a4").size === 3)

val newStore2 = makeBlockManager(storeSize, s"newstore2")
allStores += newStore2

// 4x replication should work now
initialStores(0).putSingle(
"a5", new Array[Byte](blockSize), StorageLevel(true, true, false, true, 4))
assert(master.getLocations("a5").size === 4)

// Remove all the stores and add new stores
(initialStores ++ Seq(newStore1, newStore2)).map { store =>
store.blockManagerId.executorId
}.foreach { execId =>
master.removeExecutor(execId)
}

// Add new stores and test if replication works
val newStores = (3 to 5).map { i => makeBlockManager(storeSize, s"newstore$i") }
allStores ++= newStores

newStores(0).putSingle(
"a6", new Array[Byte](blockSize), StorageLevel(true, true, false, true, 3))
assert(master.getLocations("a6").size === 3)
} finally {
allStores.foreach { _.stop() }
}
}

/**
Expand All @@ -1273,7 +1365,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter
* is correct. Then it also drops the block from memory of each store (using LRU) and
* again checks whether the master's knowledge gets updated.
*/
def testReplication(maxReplication: Int, storageLevels: Seq[StorageLevel]) {
private def testReplication(maxReplication: Int, storageLevels: Seq[StorageLevel]) {
import org.apache.spark.storage.StorageLevel._

assert(maxReplication > 1,
Expand All @@ -1291,7 +1383,6 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter

try {
storageLevels.foreach {storageLevel =>

// Put the block into one of the stores
val blockId = new TestBlockId(
"block-with-" + storageLevel.description.replace(" ", "-").toLowerCase)
Expand Down Expand Up @@ -1364,4 +1455,3 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter
}
}
}

0 comments on commit d081bf6

Please sign in to comment.