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
2 changes: 1 addition & 1 deletion core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ private[spark]
class BlockRDD[T: ClassTag](sc: SparkContext, @transient val blockIds: Array[BlockId])
extends RDD[T](sc, Nil) {

@transient lazy val _locations = BlockManager.blockIdsToHosts(blockIds, SparkEnv.get)
@transient lazy val _locations = BlockManager.blockIdsToLocations(blockIds, SparkEnv.get)
@volatile private var _isValid = true

override def getPartitions: Array[Partition] = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@ import org.apache.spark.network.netty.SparkTransportConf
import org.apache.spark.network.shuffle.{ExternalShuffleClient, TempFileManager}
import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo
import org.apache.spark.rpc.RpcEnv
import org.apache.spark.scheduler.ExecutorCacheTaskLocation
import org.apache.spark.serializer.{SerializerInstance, SerializerManager}
import org.apache.spark.shuffle.ShuffleManager
import org.apache.spark.storage.memory._
Expand Down Expand Up @@ -1554,7 +1555,7 @@ private[spark] class BlockManager(
private[spark] object BlockManager {
private val ID_GENERATOR = new IdGenerator

def blockIdsToHosts(
def blockIdsToLocations(
blockIds: Array[BlockId],
env: SparkEnv,
blockManagerMaster: BlockManagerMaster = null): Map[BlockId, Seq[String]] = {
Expand All @@ -1569,7 +1570,9 @@ private[spark] object BlockManager {

val blockManagers = new HashMap[BlockId, Seq[String]]
for (i <- 0 until blockIds.length) {
blockManagers(blockIds(i)) = blockLocations(i).map(_.host)
blockManagers(blockIds(i)) = blockLocations(i).map { loc =>
ExecutorCacheTaskLocation(loc.host, loc.executorId).toString
}
}
blockManagers.toMap
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1422,6 +1422,19 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
assert(mockBlockTransferService.tempFileManager === store.remoteBlockTempFileManager)
}

test("query locations of blockIds") {
val mockBlockManagerMaster = mock(classOf[BlockManagerMaster])
val blockLocations = Seq(BlockManagerId("1", "host1", 100), BlockManagerId("2", "host2", 200))
when(mockBlockManagerMaster.getLocations(mc.any[Array[BlockId]]))
.thenReturn(Array(blockLocations))
val env = mock(classOf[SparkEnv])

val blockIds: Array[BlockId] = Array(StreamBlockId(1, 2))
val locs = BlockManager.blockIdsToLocations(blockIds, env, mockBlockManagerMaster)
val expectedLocs = Seq("executor_host1_1", "executor_host2_2")
assert(locs(blockIds(0)) == expectedLocs)
}

class MockBlockTransferService(val maxFailures: Int) extends BlockTransferService {
var numCalls = 0
var tempFileManager: TempFileManager = null
Expand Down