@@ -49,10 +49,6 @@ import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat
4949import org .apache .spark .util .{AkkaUtils , ByteBufferInputStream , SizeEstimator , Utils }
5050import org .apache .spark .storage .StorageLevel ._
5151import org .apache .spark .storage .BlockManagerMessages .BlockManagerHeartbeat
52- import org .apache .spark .storage .BroadcastBlockId
53- import org .apache .spark .storage .RDDBlockId
54- import org .apache .spark .storage .ShuffleBlockId
55- import org .apache .spark .storage .TestBlockId
5652
5753
5854class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter
@@ -126,7 +122,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter
126122
127123 System .clearProperty(" spark.test.useCompressedOops" )
128124 }
129- /*
125+
130126 test(" StorageLevel object caching" ) {
131127 val level1 = StorageLevel (false , false , false , false , 3 )
132128 val level2 = StorageLevel (false , false , false , false , 3 ) // this should return the same object as level1
@@ -195,7 +191,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter
195191 store = makeBlockManager(2000 , " exec1" )
196192 store2 = makeBlockManager(2000 , " exec2" )
197193
198- val peers = master.getPeers(store.blockManagerId, 1 )
194+ val peers = master.getPeers(store.blockManagerId)
199195 assert(peers.size === 1 , " master did not return the other manager as a peer" )
200196 assert(peers.head === store2.blockManagerId, " peer returned by master is not the other manager" )
201197
@@ -454,7 +450,6 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter
454450 val list2DiskGet = store.get(" list2disk" )
455451 assert(list2DiskGet.isDefined, " list2memory expected to be in store" )
456452 assert(list2DiskGet.get.data.size === 3 )
457- System.out.println(list2DiskGet)
458453 // We don't know the exact size of the data on disk, but it should certainly be > 0.
459454 assert(list2DiskGet.get.inputMetrics.bytesRead > 0 )
460455 assert(list2DiskGet.get.inputMetrics.readMethod === DataReadMethod .Disk )
@@ -1234,14 +1229,52 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter
12341229 assert(unrollMemoryAfterB6 === unrollMemoryAfterB4)
12351230 assert(unrollMemoryAfterB7 === unrollMemoryAfterB4)
12361231 }
1237- */
1232+
1233+ test(" get peers with store addition and removal" ) {
1234+ val numStores = 4
1235+ val stores = (1 to numStores - 1 ).map { i => makeBlockManager(1000 , s " store $i" ) }
1236+ try {
1237+ val storeIds = stores.map { _.blockManagerId }.toSet
1238+ assert(master.getPeers(stores(0 ).blockManagerId).toSet ===
1239+ storeIds.filterNot { _ == stores(0 ).blockManagerId })
1240+ assert(master.getPeers(stores(1 ).blockManagerId).toSet ===
1241+ storeIds.filterNot { _ == stores(1 ).blockManagerId })
1242+ assert(master.getPeers(stores(2 ).blockManagerId).toSet ===
1243+ storeIds.filterNot { _ == stores(2 ).blockManagerId })
1244+
1245+ // Add a new store and test whether get peers returns it
1246+ val newStore = makeBlockManager(1000 , s " store $numStores" )
1247+ assert(master.getPeers(stores(0 ).blockManagerId).toSet ===
1248+ storeIds.filterNot { _ == stores(0 ).blockManagerId } + newStore.blockManagerId)
1249+ assert(master.getPeers(stores(1 ).blockManagerId).toSet ===
1250+ storeIds.filterNot { _ == stores(1 ).blockManagerId } + newStore.blockManagerId)
1251+ assert(master.getPeers(stores(2 ).blockManagerId).toSet ===
1252+ storeIds.filterNot { _ == stores(2 ).blockManagerId } + newStore.blockManagerId)
1253+ assert(master.getPeers(newStore.blockManagerId).toSet === storeIds)
1254+
1255+ // Remove a store and test whether get peers returns it
1256+ val storeIdToRemove = stores(0 ).blockManagerId
1257+ master.removeExecutor(storeIdToRemove.executorId)
1258+ assert(! master.getPeers(stores(1 ).blockManagerId).contains(storeIdToRemove))
1259+ assert(! master.getPeers(stores(2 ).blockManagerId).contains(storeIdToRemove))
1260+ assert(! master.getPeers(newStore.blockManagerId).contains(storeIdToRemove))
1261+
1262+ // Test whether asking for peers of a unregistered block manager id returns empty list
1263+ assert(master.getPeers(stores(0 ).blockManagerId).isEmpty)
1264+ assert(master.getPeers(BlockManagerId (" " , " " , 1 )).isEmpty)
1265+ } finally {
1266+ stores.foreach { _.stop() }
1267+ }
1268+ }
1269+
12381270 test(" block replication - 2x" ) {
12391271 testReplication(2 ,
12401272 Seq (MEMORY_ONLY , MEMORY_ONLY_SER , DISK_ONLY , MEMORY_AND_DISK_2 , MEMORY_AND_DISK_SER_2 )
12411273 )
12421274 }
12431275
12441276 test(" block replication - 3x" ) {
1277+ // Generate storage levels with 3x replication
12451278 val storageLevels = {
12461279 Seq (MEMORY_ONLY , MEMORY_ONLY_SER , DISK_ONLY , MEMORY_AND_DISK , MEMORY_AND_DISK_SER ).map {
12471280 level => StorageLevel (
@@ -1252,18 +1285,77 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter
12521285 }
12531286
12541287 test(" block replication - mixed between 1x to 5x" ) {
1288+ // Generate storage levels with varying replication
12551289 val storageLevels = Seq (
12561290 MEMORY_ONLY ,
12571291 MEMORY_ONLY_SER_2 ,
1258- StorageLevel (true , true , false , true , 3 ),
1259- StorageLevel (true , false , false , false , 4 ),
1260- StorageLevel (false , false , false , false , 5 ),
1261- StorageLevel (true , false , false , false , 4 ),
1262- StorageLevel (true , true , false , true , 3 ),
1292+ StorageLevel (true , false , false , false , 3 ),
1293+ StorageLevel (true , true , false , true , 4 ),
1294+ StorageLevel (true , true , false , false , 5 ),
1295+ StorageLevel (true , true , false , true , 4 ),
1296+ StorageLevel (true , false , false , false , 3 ),
12631297 MEMORY_ONLY_SER_2 ,
12641298 MEMORY_ONLY
12651299 )
1266- testReplication(3 , storageLevels)
1300+ testReplication(5 , storageLevels)
1301+ }
1302+
1303+ test(" block replication with addition and removal of executors" ) {
1304+ val blockSize = 1000
1305+ val storeSize = 10000
1306+ val allStores = new ArrayBuffer [BlockManager ]()
1307+
1308+ try {
1309+ val initialStores = (1 to 2 ).map { i => makeBlockManager(storeSize, s " store $i" ) }
1310+ allStores ++= initialStores
1311+
1312+ // 2x replication works
1313+ initialStores(0 ).putSingle(" a1" , new Array [Byte ](blockSize), StorageLevel .MEMORY_AND_DISK_2 )
1314+ assert(master.getLocations(" a1" ).size === 2 )
1315+
1316+ // 3x replication should only replicate 2x
1317+ initialStores(0 ).putSingle(
1318+ " a2" , new Array [Byte ](blockSize), StorageLevel (true , true , false , true , 3 ))
1319+ assert(master.getLocations(" a2" ).size === 2 )
1320+
1321+ val newStore1 = makeBlockManager(storeSize, s " newstore1 " )
1322+ allStores += newStore1
1323+
1324+ // 3x replication should work now
1325+ initialStores(0 ).putSingle(
1326+ " a3" , new Array [Byte ](blockSize), StorageLevel (true , true , false , true , 3 ))
1327+ assert(master.getLocations(" a3" ).size === 3 )
1328+
1329+ // 4x replication should only replicate 3x
1330+ initialStores(0 ).putSingle(
1331+ " a4" , new Array [Byte ](blockSize), StorageLevel (true , true , false , true , 4 ))
1332+ assert(master.getLocations(" a4" ).size === 3 )
1333+
1334+ val newStore2 = makeBlockManager(storeSize, s " newstore2 " )
1335+ allStores += newStore2
1336+
1337+ // 4x replication should work now
1338+ initialStores(0 ).putSingle(
1339+ " a5" , new Array [Byte ](blockSize), StorageLevel (true , true , false , true , 4 ))
1340+ assert(master.getLocations(" a5" ).size === 4 )
1341+
1342+ // Remove all the stores and add new stores
1343+ (initialStores ++ Seq (newStore1, newStore2)).map { store =>
1344+ store.blockManagerId.executorId
1345+ }.foreach { execId =>
1346+ master.removeExecutor(execId)
1347+ }
1348+
1349+ // Add new stores and test if replication works
1350+ val newStores = (3 to 5 ).map { i => makeBlockManager(storeSize, s " newstore $i" ) }
1351+ allStores ++= newStores
1352+
1353+ newStores(0 ).putSingle(
1354+ " a6" , new Array [Byte ](blockSize), StorageLevel (true , true , false , true , 3 ))
1355+ assert(master.getLocations(" a6" ).size === 3 )
1356+ } finally {
1357+ allStores.foreach { _.stop() }
1358+ }
12671359 }
12681360
12691361 /**
@@ -1273,7 +1365,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter
12731365 * is correct. Then it also drops the block from memory of each store (using LRU) and
12741366 * again checks whether the master's knowledge gets updated.
12751367 */
1276- def testReplication (maxReplication : Int , storageLevels : Seq [StorageLevel ]) {
1368+ private def testReplication (maxReplication : Int , storageLevels : Seq [StorageLevel ]) {
12771369 import org .apache .spark .storage .StorageLevel ._
12781370
12791371 assert(maxReplication > 1 ,
@@ -1291,7 +1383,6 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter
12911383
12921384 try {
12931385 storageLevels.foreach {storageLevel =>
1294-
12951386 // Put the block into one of the stores
12961387 val blockId = new TestBlockId (
12971388 " block-with-" + storageLevel.description.replace(" " , " -" ).toLowerCase)
@@ -1364,4 +1455,3 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter
13641455 }
13651456 }
13661457}
1367-
0 commit comments