ShuffleReader
1.概述
本次分析基于spark版本2.11进行;
在【spark源码-shuffle原理分析-1-ShuffleWriter】中分析了shuffle过程中,stage结果数据落地到磁盘的逻辑;针对stage结果数据是从哪里来的,并没有进行分析;接下来就分析下shuffle过程中,shuffle的数据是从哪里来的;
2.RDD数据缓存级别
默认缓存级别:MEMORY_ONLY
Storage Level | Remark |
---|---|
None | 不缓存 |
MEMORY_ONLY | 将 RDD 作为反序列化的 Java 对象存储在 JVM 中。如果 RDD 不适合内存,则某些分区将不会被缓存,并且会在每次需要时重新计算。这是默认级别。 |
MEMORY_ONLY_SER | 将RDD 存储为序列化的Java 对象(每个分区一个字节数组)。这通常比反序列化对象更节省空间,尤其是在使用 快速序列化程序时,但读取时更占用 CPU。 |
MYMORY_AND_DISK | 将 RDD 作为反序列化的 Java 对象存储在 JVM 中。如果 RDD 不适合内存,请将不适合的分区存储在磁盘上,并在需要时从那里读取它们。 |
MEMORY_AND_DISK_SER | 类似于 MEMORY_ONLY_SER,但将不适合内存的分区溢出到磁盘,而不是在每次需要时动态地重新计算它们。 |
DISK_ONLY | 仅将 RDD 分区存储在磁盘上。 |
MEMORY_ONLY_2 MEMORY_ONLY_SER_2 MEMORY_AND_DISK_2 MEMORY_AND_DISK_SER_2 DISK_ONLY_2 | 与上述级别相同,但在两个集群节点上复制每个分区。 |
OFF_HEAP(实验性) | 类似于 MEMORY_ONLY_SER,但将数据存储在 堆外内存中。这需要启用堆外内存。 |
3.获取RDD的分区数据迭代器
说明:
- 在spark中,每个task处理rdd一个分区的数据;
- 在执行shuffle任务时,也只需要从RDD中取出任务对应分区数据;
private[spark] class ShuffleMapTask(
stageId: Int,
stageAttemptId: Int,
taskBinary: Broadcast[Array[Byte]],
partition: Partition,
@transient private var locs: Seq[TaskLocation],
localProperties: Properties,
serializedTaskMetrics: Array[Byte],
jobId: Option[Int] = None,
appId: Option[String] = None,
appAttemptId: Option[String] = None,
isBarrier: Boolean = false)
extends Task[MapStatus](stageId, stageAttemptId, partition.index, localProperties,
serializedTaskMetrics, jobId, appId, appAttemptId, isBarrier)
with Logging {
override def runTask(context: TaskContext): MapStatus = {
//---------其他代码---------
val ser = SparkEnv.get.closureSerializer.newInstance()
val (rdd, dep) = ser.deserialize[(RDD[_], ShuffleDependency[_, _, _])](
ByteBuffer.wrap(taskBinary.value), Thread.currentThread.getContextClassLoader)
//---------其他代码---------
var writer: ShuffleWriter[Any, Any] = null
try {
val manager = SparkEnv.get.shuffleManager
writer = manager.getWriter[Any, Any](dep.shuffleHandle, partitionId, context)
//获取RDD分区的数据迭代器,通过写入器将数据落地磁盘
writer.write(rdd.iterator(partition, context).asInstanceOf[Iterator[_ <: Product2[Any, Any]]])
writer.stop(success = true).get
} catch {
//---------其他代码---------
}
}
}
3.1.RDD#iterator
说明:
- 存储级别设置有缓存,从缓存中获取数据;
- 存储级别设置不缓存,计算或读取Checkpoint获取数据;
abstract class RDD[T: ClassTag](
@transient private var _sc: SparkContext,
@transient private var deps: Seq[Dependency[_]]
) extends Serializable with Logging {
final def iterator(split: Partition, context: TaskContext): Iterator[T] = {
//匹配缓存级别
if (storageLevel != StorageLevel.NONE) {//缓存
//从缓存中获取数据;
getOrCompute(split, context)
} else {//不缓存
//计算或读取Checkpoint获取数据
computeOrReadCheckpoint(split, context)
}
}
}
3.1.1.RDD#getOrCompute-从缓存中获取数据迭代器
说明:
- 定义通过计算或读取Checkpoint获取数据迭代器的函数;
- 根据blockId按照存储级别读取数据;
- 缓存中有数据,读取缓存数据;
- 先读内存中的数据;
- 内存中没有,再读磁盘中的数据;
- 缓存中无数据,计算或读取Checkpoint获取数据,然后将数据进行缓存,最后再从缓存中读取数据;
- 缓存中有数据,读取缓存数据;
- 将读取的数据封装到InterruptibleIterator迭代器中;
abstract class RDD[T: ClassTag](
@transient private var _sc: SparkContext,
@transient private var deps: Seq[Dependency[_]]
) extends Serializable with Logging {
private[spark] def getOrCompute(partition: Partition, context: TaskContext): Iterator[T] = {
//组织blockId
val blockId = RDDBlockId(id, partition.index)
var readCachedBlock = true
//根据blockId按照存储级别读取数据
SparkEnv.get.blockManager.getOrElseUpdate(
blockId,
storageLevel,
elementClassTag,
//定义通过计算或读取Checkpoint获取数据迭代器的函数
() => {
readCachedBlock = false
//过计算或读取Checkpoint获取数据迭代器
computeOrReadCheckpoint(partition, context)
}
)
//判断数据获取方式,将数据封装到InterruptibleIterator迭代器中;
match {
//从缓存中获取的数据
case Left(blockResult) =>
if (readCachedBlock) {
val existingMetrics = context.taskMetrics().inputMetrics
existingMetrics.incBytesRead(blockResult.bytes)
new InterruptibleIterator[T](context, blockResult.data.asInstanceOf[Iterator[T]]) {
override def next(): T = {
existingMetrics.incRecordsRead(1)
delegate.next()
}
}
} else {
new InterruptibleIterator(context, blockResult.data.asInstanceOf[Iterator[T]])
}
//通过计算或读取Checkpoint获取的数据
case Right(iter) =>
new InterruptibleIterator(context, iter.asInstanceOf[Iterator[T]])
}
}
}
3.1.1.1.BlockManager#getOrElseUpdate-获取数据
功能:
从缓存中获取数据,缓存中获取数据失败,调用makeIterator函数获取数据,然后将数据进行缓存,然后再从缓存中获取数据;
说明:
- 从缓存中获取的数据,封装在Left中;
- 通过计算或读取Checkpoint获取的数据,封装在Right中;
流程:
- 从缓存中获取数据,获取成功,将获取的数据封装到Left返回;
- 缓存数据获取失败,调用BlockManager#doPutIterator获取数据;
- 调用makeIterator计算数据;
- 将计算的数据进行缓存,缓存失败则返回失败的数据集合,缓存成功则返回None;
- 计算数据缓存成功,从本地缓存中获取数据,将数据封装到Left返回;
- 计算数据缓存失败,将计算获取的数据封装到Right中返回;
private[spark] class BlockManager(
executorId: String,
rpcEnv: RpcEnv,
val master: BlockManagerMaster,
val serializerManager: SerializerManager,
val conf: SparkConf,
memoryManager: MemoryManager,
mapOutputTracker: MapOutputTracker,
shuffleManager: ShuffleManager,
val blockTransferService: BlockTransferService,
securityManager: SecurityManager,
numUsableCores: Int)
extends BlockDataManager with BlockEvictionHandler with Logging {
def getOrElseUpdate[T](
blockId: BlockId,
level: StorageLevel,
classTag: ClassTag[T],
makeIterator: () => Iterator[T]): Either[BlockResult, Iterator[T]] = {
// 根据blockId从缓存中获取数据
get[T](blockId)(classTag) match {
case Some(block) =>
//数据获取成功,返回block
return Left(block)
case _ =>
// Need to compute the block.
}
//通过makeIterator函数计算数据,将计算的数据缓存到内存/磁盘;
//返回缓存失败的数据:成功返回None,失败返回失败的数据;
doPutIterator(blockId, makeIterator, level, classTag, keepReadLock = true) match {
case None =>
// 数据缓存成功,从缓存中获取数据
val blockResult = getLocalValues(blockId).getOrElse {
//从缓存中获取数据失败,释放block锁
releaseLock(blockId)
throw new SparkException(s"get() failed for block $blockId even though we held a lock")
}
// //从缓存中获取数据成功,释放block锁
releaseLock(blockId)
//将从缓存中获取的数据封装到Left中
Left(blockResult)
case Some(iter) =>
// 数据缓存失败,将通过makeIterator函数计算的数据封装到Right中
Right(iter)
}
}
}
3.1.1.1.1.BlockManager#get-获取缓存数据
数据获取流程:
- 先获取本地缓存数据,有缓存,返回获取的缓存数据;
- 没有本地缓存,从远端获取数据,获取成功即返回;
- 本地和远端都没有获取到数据,返回None;
private[spark] class BlockManager(
executorId: String,
rpcEnv: RpcEnv,
val master: BlockManagerMaster,
val serializerManager: SerializerManager,
val conf: SparkConf,
memoryManager: MemoryManager,
mapOutputTracker: MapOutputTracker,
shuffleManager: ShuffleManager,
val blockTransferService: BlockTransferService,
securityManager: SecurityManager,
numUsableCores: Int)
extends BlockDataManager with BlockEvictionHandler with Logging {
def get[T: ClassTag](blockId: BlockId): Option[BlockResult] = {
//获取本地缓存
val local = getLocalValues(blockId)
if (local.isDefined) {
logInfo(s"Found block $blockId locally")
return local
}
//获取远端缓存
val remote = getRemoteValues[T](blockId)
if (remote.isDefined) {
logInfo(s"Found block $blockId remotely")
return remote
}
None
}
}
3.1.1.1.1.1.BlockManager#getLocalValues-获取本地缓存数据
本地数据获取流程:
- 通过BlockInfoManager锁定数据block;
- 如果内存中缓存由该block,从内存中获取block数据迭代器;
- 数据反序列化存储,从反序列化内存块中获取数据迭代器;
- 从反序列化内存块中获取数据数组;
- 将数据数据组转为数据迭代器,然后然后返回数据迭代器;
- 数据序列化存储,从序列化内存块中获取数据迭代器;
- 从序列化内存块中获取数据buffer并返回;
- 获取数据buffer的输入流;
- 反序列化数据buffer输入流;
- 获取反序列化后的数据迭代器并返回;
- 释放block锁;
- BlockResult封装数据获取结果;
- 数据反序列化存储,从反序列化内存块中获取数据迭代器;
- 如果内存中没有、磁盘中有缓存该block,从磁盘中获取block数据迭代器;
- 从磁盘中获取数据;
- 构造数据迭代器;
- 根据数据存储是否序列化/反序列化,对数据进行加工;
- 尝试将数据缓存到内存;
- 释放block锁;
- BlockResult封装数据获取结果;
private[spark] class BlockManager(
executorId: String,
rpcEnv: RpcEnv,
val master: BlockManagerMaster,
val serializerManager: SerializerManager,
val conf: SparkConf,
memoryManager: MemoryManager,
mapOutputTracker: MapOutputTracker,
shuffleManager: ShuffleManager,
val blockTransferService: BlockTransferService,
securityManager: SecurityManager,
numUsableCores: Int)
extends BlockDataManager with BlockEvictionHandler with Logging {
def getLocalValues(blockId: BlockId): Option[BlockResult] = {
logDebug(s"Getting local block $blockId")
//锁定block,判断是否存在
blockInfoManager.lockForReading(blockId) match {
case None =>
logDebug(s"Block $blockId was not found")
None
case Some(info) =>
val level = info.level
logDebug(s"Level for block $blockId is $level")
val taskAttemptId = Option(TaskContext.get()).map(_.taskAttemptId())
//内存中有缓存该block
if (level.useMemory && memoryStore.contains(blockId)) {
//判断是否反序列化,返回数据迭代器
val iter: Iterator[Any] = if (level.deserialized) {//有反序列化
//从内存中获取数据迭代器:只能取反序列化内存块的数据
memoryStore.getValues(blockId).get
} else {//没有反序列化
//从内存中获取数据迭代器:能耐取序列化内存块的数据
//先获取数据buffer、接着反序列化、最后获取迭代器
serializerManager.dataDeserializeStream(
blockId, memoryStore.getBytes(blockId).get.toInputStream())(info.classTag)
}
//释放block锁
val ci = CompletionIterator[Any, Iterator[Any]](iter, {
releaseLock(blockId, taskAttemptId)
})
//BlockResult封装数据获取结果
Some(new BlockResult(ci, DataReadMethod.Memory, info.size))
}
//磁盘中有缓存该block
else if (level.useDisk && diskStore.contains(blockId)) {
//从磁盘中获取数据
val diskData = diskStore.getBytes(blockId)
//构造数据迭代器
val iterToReturn: Iterator[Any] = {
if (level.deserialized) {//反序列化存储
val diskValues = serializerManager.dataDeserializeStream(
blockId,
diskData.toInputStream())(info.classTag)
//尝试将数据缓存到内存
maybeCacheDiskValuesInMemory(info, blockId, level, diskValues)
} else {//序列化存储
//尝试将数据缓存到内存
val stream = maybeCacheDiskBytesInMemory(info, blockId, level, diskData)
.map { _.toInputStream(dispose = false) }
.getOrElse { diskData.toInputStream() }
serializerManager.dataDeserializeStream(blockId, stream)(info.classTag)
}
}
//释放block锁
val ci = CompletionIterator[Any, Iterator[Any]](iterToReturn, {
releaseLockAndDispose(blockId, diskData, taskAttemptId)
})
//BlockResult封装数据获取结果
Some(new BlockResult(ci, DataReadMethod.Disk, info.size))
} else {
handleLocalReadFailure(blockId)
}
}
}
}
private[spark] class MemoryStore(
conf: SparkConf,
blockInfoManager: BlockInfoManager,
serializerManager: SerializerManager,
memoryManager: MemoryManager,
blockEvictionHandler: BlockEvictionHandler)
extends Logging {
//获取数据buffer
def getBytes(blockId: BlockId): Option[ChunkedByteBuffer] = {
//获取block内存块
val entry = entries.synchronized { entries.get(blockId) }
entry match {
case null => None
case e: DeserializedMemoryEntry[_] =>
throw new IllegalArgumentException("should only call getBytes on serialized blocks")
//只能获取序列化内存块存储的数据buffer
case SerializedMemoryEntry(bytes, _, _) => Some(bytes)
}
}
//获取数据迭代器
def getValues(blockId: BlockId): Option[Iterator[_]] = {
//获取block内存块
val entry = entries.synchronized { entries.get(blockId) }
entry match {
case null => None
case e: SerializedMemoryEntry[_] =>
throw new IllegalArgumentException("should only call getValues on deserialized blocks")
//只能获取反序列化内存块的数据数组
case DeserializedMemoryEntry(values, _, _) =>
val x = Some(values)
//返回数据迭代器
x.map(_.iterator)
}
}
}
3.1.1.1.1.2.BlockManager#getRemoteValues-获取远端缓存数据
流程:
- 从远端获取ChunkedByteBuffer形式的数据;
- 从BlockManagerMaster获取持有block的所有远端BlockManager位置;
- 遍历远端BlockManager位置,抓取数据;
- 失败次数超过5次,刷新位置信息,防止过期;
- 当所有位置都抓取失败后,认为抓取失败,染回None;
- 数据封装为ChunkedByteBuffer形式返回;
- 将数据封装到BlockResult中返回;
private[spark] class BlockManager(
executorId: String,
rpcEnv: RpcEnv,
val master: BlockManagerMaster,
val serializerManager: SerializerManager,
val conf: SparkConf,
memoryManager: MemoryManager,
mapOutputTracker: MapOutputTracker,
shuffleManager: ShuffleManager,
val blockTransferService: BlockTransferService,
securityManager: SecurityManager,
numUsableCores: Int)
extends BlockDataManager with BlockEvictionHandler with Logging {
private def getRemoteValues[T: ClassTag](blockId: BlockId): Option[BlockResult] = {
val ct = implicitly[ClassTag[T]]
//获取远端buffer数据,将数据封装到BlockResult中
getRemoteBytes(blockId).map { data =>
val values =
serializerManager.dataDeserializeStream(blockId, data.toInputStream(dispose = true))(ct)
new BlockResult(values, DataReadMethod.Network, data.size)
}
}
def getRemoteBytes(blockId: BlockId): Option[ChunkedByteBuffer] = {
logDebug(s"Getting remote block $blockId")
require(blockId != null, "BlockId is null")
var runningFailureCount = 0
var totalFailureCount = 0
// 获取所有持有该block信息的远端BlockManager位置
val locationsAndStatus = master.getLocationsAndStatus(blockId)
val blockSize = locationsAndStatus.map { b =>
b.status.diskSize.max(b.status.memSize)
}.getOrElse(0L)
val blockLocations = locationsAndStatus.map(_.locations).getOrElse(Seq.empty)
// 如果块大小高于阈值,则应该将FileManger传递给BlockTransferService,它将利用它溢出区块;如果没有,则传入null值表示块将持久化在内存中
val tempFileManager = if (blockSize > maxRemoteBlockToMem) {
remoteBlockTempFileManager
} else {
null
}
//根据BlockManagerId中的拓扑信息对BlockManager的位置进行排序
val locations = sortLocations(blockLocations)
//抓取最大失败次数
val maxFetchFailures = locations.size
var locationIterator = locations.iterator
//对于每个远端BlockManager,同步地获取块数据,并以ChunkedByteBuffer形式返回
while (locationIterator.hasNext) {
val loc = locationIterator.next()
logDebug(s"Getting remote block $blockId from $loc")
val data = try {
//获取当前位置的数据
blockTransferService.fetchBlockSync(
loc.host, loc.port, loc.executorId, blockId.toString, tempFileManager)
} catch {
case NonFatal(e) =>
runningFailureCount += 1
totalFailureCount += 1
//当所有位置都抓取失败后,认为抓取失败,染回None
if (totalFailureCount >= maxFetchFailures) {
logWarning(s"Failed to fetch block after $totalFailureCount fetch failures. " +
s"Most recent failure cause:", e)
return None
}
logWarning(s"Failed to fetch remote block $blockId " +
s"from $loc (failed attempt $runningFailureCount)", e)
// 失败次数超过5次,刷新位置信息,防止过期
if (runningFailureCount >= maxFailuresBeforeLocationRefresh) {
locationIterator = sortLocations(master.getLocations(blockId)).iterator
logDebug(s"Refreshed locations from the driver " +
s"after ${runningFailureCount} fetch failures.")
runningFailureCount = 0
}
// 这个位置失败了,所以我们通过返回null从另一个位置重试获取
null
}
if (data != null) {
// 以ChunkedByteBuffer的形式返回数据
if (remoteReadNioBufferConversion) {
return Some(new ChunkedByteBuffer(data.nioByteBuffer()))
} else {
return Some(ChunkedByteBuffer.fromManagedBuffer(data))
}
}
logDebug(s"The value of block $blockId is null")
}
logDebug(s"Block $blockId not found")
None
}
}
3.1.1.1.2.BlockManager#doPutIterator-计算、缓存数据
功能:
定义一个获取数据迭代器,将数据缓存到内存/磁盘的函数,并调用BlockManager.doPut()
函数执行定义的函数,完成数据获取、数据缓存的工作;
说明:
- 定义一个函数,用于获取数据迭代器、并将数据写入内存/磁盘中进行缓存;
- 支持缓存到内存;
- 执行函数(获取数据迭代器的函数)获取数据迭代器;
- 判断,如果需要反序列化存储,以反序列化的方式将数据缓存到内存;
- 判断,如果需要序列化存储,以序列化的方式将数据缓存到内存;
- 匹配数据缓存到内存的结果;
- 缓存成功,拿到缓存数据的数据量;
- 缓存失败,如果支持将数据缓存到磁盘,就将数据缓存到磁盘,并拿到缓存数据的数据量;
- 缓存失败,如果不支持间数据缓存到磁盘,将数据添加到缓存失败数据集中;
- 不支持缓存到内存,支持缓存到磁盘;
- 定义数据缓存到磁盘的函数
- 获取缓存块block的管道输出流;
- 执行函数(获取数据迭代器的函数)获取数据迭代器;
- 将获取的数据通过输出流缓存到磁盘
- 拿到缓存数据的数据量
- 定义数据缓存到磁盘的函数
- 数据缓存成功;
- 将缓存数据量绑定到blockInfo对象中;
- 向BlockManagerMaster汇报数据缓存情况;
- 将该Block放入updatedBlocks;
- 在副本数要求 > 1的情况下,将block复制到远端节点,作为副本备份;
- 确定数据类型,判断是否支持使用Kryo进行序列化;
- 将block复制到远端节点,作为副本备份;
- 返回缓存失败的数据:成功返回None,失败返回失败的数据;
- 支持缓存到内存;
- 通过执行
BlockManager.doPut()
函数,完成数据缓存工作; - 返回缓存失败的数据:成功返回None,失败返回失败的数据;
private[spark] class BlockManager(
executorId: String,
rpcEnv: RpcEnv,
val master: BlockManagerMaster,
val serializerManager: SerializerManager,
val conf: SparkConf,
memoryManager: MemoryManager,
mapOutputTracker: MapOutputTracker,
shuffleManager: ShuffleManager,
val blockTransferService: BlockTransferService,
securityManager: SecurityManager,
numUsableCores: Int)
extends BlockDataManager with BlockEvictionHandler with Logging {
private def doPutIterator[T](
blockId: BlockId,
iterator: () => Iterator[T],
level: StorageLevel,
classTag: ClassTag[T],
tellMaster: Boolean = true,
keepReadLock: Boolean = false): Option[PartiallyUnrolledIterator[T]] = {
doPut(blockId, level, classTag, tellMaster = tellMaster, keepReadLock = keepReadLock) {
//定义的一个将数据写入缓存的函数
info =>
val startTimeMs = System.currentTimeMillis
//内存缓存数据失败的数据迭代器统计
var iteratorFromFailedMemoryStorePut: Option[PartiallyUnrolledIterator[T]] = None
// 统计缓存block的大小
var size = 0L
//缓存到内存
if (level.useMemory) {
if (level.deserialized) {// 反序列化存储
//以反序列化的方式将数据存储到内存
memoryStore.putIteratorAsValues(
blockId,
//调用获取数迭代器的函数
iterator(),
classTag
) match {
case Right(s) =>
size = s
case Left(iter) =>
// 内存空间不足,尝试将数据落地磁盘
if (level.useDisk) {//支持落地磁盘
logWarning(s"Persisting block $blockId to disk instead.")
//数据落地磁盘
diskStore.put(blockId) { channel =>
val out = Channels.newOutputStream(channel)
serializerManager.dataSerializeStream(blockId, out, iter)(classTag)
}
size = diskStore.getSize(blockId)
} else {//不支持落地磁盘
//统计落地内存失败且不支持落地磁盘的数据
iteratorFromFailedMemoryStorePut = Some(iter)
}
}
} else { // 序列化存储
//以序列化的方式将数据存储到内存
memoryStore.putIteratorAsBytes(
blockId,
//调用获取数迭代器的函数
iterator(),
classTag,
level.memoryMode
) match {
case Right(s) =>
size = s
case Left(partiallySerializedValues) =>
// 内存空间不足,尝试将数据落地磁盘
if (level.useDisk) {//支持落地磁盘
logWarning(s"Persisting block $blockId to disk instead.")
//数据落地磁盘
diskStore.put(blockId) { channel =>
val out = Channels.newOutputStream(channel)
partiallySerializedValues.finishWritingToStream(out)
}
size = diskStore.getSize(blockId)
} else {//不支持落地磁盘
//统计落地内存失败且不支持落地磁盘的数据
iteratorFromFailedMemoryStorePut = Some(partiallySerializedValues.valuesIterator)
}
}
}
}
//缓存到磁盘
else if (level.useDisk) {
//通过定义的函数将数据缓存的磁盘
diskStore.put(blockId) {
//定义数据缓存到磁盘的函数
channel =>
//获取缓存块block的管道输出流
val out = Channels.newOutputStream(channel)
//将获取的数据通过输出流缓存到磁盘
serializerManager.dataSerializeStream(
blockId,
out,
//调用获取数迭代器的函数
iterator()
)(classTag)
}
size = diskStore.getSize(blockId)
}
val putBlockStatus = getCurrentBlockStatus(blockId, info)
val blockWasSuccessfullyStored = putBlockStatus.storageLevel.isValid
//block缓存成功
if (blockWasSuccessfullyStored) {
info.size = size
// 向BlockManagerMaster汇报
if (tellMaster && info.tellMaster) {
reportBlockStatus(blockId, putBlockStatus)
}
//将该Block放入updatedBlocks
addUpdatedBlockStatusToTaskMetrics(blockId, putBlockStatus)
logDebug("Put block %s locally took %s".format(blockId, Utils.getUsedTimeMs(startTimeMs)))
//要求副本数大于1
if (level.replication > 1) {
val remoteStartTime = System.currentTimeMillis
val bytesToReplicate = doGetLocalBytes(blockId, info)
//确定数据类型,判断是否支持使用Kryo进行序列化
val remoteClassTag = if (!serializerManager.canUseKryo(classTag)) {
scala.reflect.classTag[Any]
} else {
classTag
}
try {
//数据复制到远程节点,作为副本备份
replicate(blockId, bytesToReplicate, level, remoteClassTag)
} finally {
bytesToReplicate.dispose()
}
logDebug("Put block %s remotely took %s"
.format(blockId, Utils.getUsedTimeMs(remoteStartTime)))
}
}
assert(blockWasSuccessfullyStored == iteratorFromFailedMemoryStorePut.isEmpty)
//返回缓存失败的数据
iteratorFromFailedMemoryStorePut
}
}
}
3.1.1.1.2.1.BlockManager#doPut-执行数据缓存
功能:
执行函数,将数据写入缓存(内存 / 磁盘)中;
说明:
- 构建指定存储级别的blockInfo对象;
- 将构建的blockInfo对象注册到BlockInfoManager中;
- 如果BlockInfoManager已经存在blockId对应的blockInfo对象,不需要写入数据,返回None;
- 否则,将构建的blockInfo对象与blockId映射关系绑定到BlockInfoManager中;
- 调用将数据写入缓存中的函数,完成数据缓存工作;获取函数调用结果;
- 数据缓存成功;
- 当需要继续持有block锁时,将独占写锁降级为共享读锁;默认继续持有,方便后面读取缓存数据;
- 否则,释放block锁;
- 数据缓存失败,从内存或磁盘中移除block;
- 数据缓存成功;
- 返回数据缓存工作的结果;
private[spark] class BlockManager(
executorId: String,
rpcEnv: RpcEnv,
val master: BlockManagerMaster,
val serializerManager: SerializerManager,
val conf: SparkConf,
memoryManager: MemoryManager,
mapOutputTracker: MapOutputTracker,
shuffleManager: ShuffleManager,
val blockTransferService: BlockTransferService,
securityManager: SecurityManager,
numUsableCores: Int)
extends BlockDataManager with BlockEvictionHandler with Logging {
private[storage] val blockInfoManager = new BlockInfoManager
private def doPut[T](
blockId: BlockId,
level: StorageLevel,
classTag: ClassTag[_],
tellMaster: Boolean,
keepReadLock: Boolean)(putBody: BlockInfo => Option[T]): Option[T] = {
require(blockId != null, "BlockId is null")
require(level != null && level.isValid, "StorageLevel is null or invalid")
val putBlockInfo = {
//获取新的blockInfo对象
val newInfo = new BlockInfo(level, classTag, tellMaster)
//将新blockInfo对象注册到blockInfoManager中,从持有blockInfo对象的锁
if (blockInfoManager.lockNewBlockForWriting(blockId, newInfo)) {
newInfo
} else {//block已存在,不需要写入数据
logWarning(s"Block $blockId already exists on this machine; not re-adding it")
if (!keepReadLock) {
// 释放block锁
releaseLock(blockId)
}
return None
}
}
val startTimeMs = System.currentTimeMillis
//初始化定义抛异常
var exceptionWasThrown: Boolean = true
val result: Option[T] = try {
//执行函数,将数据进行缓存
val res = putBody(putBlockInfo)
//不抛异常
exceptionWasThrown = false
if (res.isEmpty) {//数据缓存成功
if (keepReadLock) {//继续持有block锁
//将独占写锁降级为共享读锁
blockInfoManager.downgradeLock(blockId)
} else {//不在持有block锁
//释放block锁
blockInfoManager.unlock(blockId)
}
} else {//数据缓存失败
//从内存或磁盘中移除block
removeBlockInternal(blockId, tellMaster = false)
logWarning(s"Putting block $blockId failed")
}
res
} catch {
case NonFatal(e) =>
logWarning(s"Putting block $blockId failed due to exception $e.")
throw e
} finally {
// 抛异常
if (exceptionWasThrown) {
removeBlockInternal(blockId, tellMaster = tellMaster)
addUpdatedBlockStatusToTaskMetrics(blockId, BlockStatus.empty)
}
}
if (level.replication > 1) {
logDebug("Putting block %s with replication took %s"
.format(blockId, Utils.getUsedTimeMs(startTimeMs)))
} else {
logDebug("Putting block %s without replication took %s"
.format(blockId, Utils.getUsedTimeMs(startTimeMs)))
}
//返回缓存失败的数据:成功返回None,失败返回失败的数据;
result
}
}
3.1.2.RDD#computeOrReadCheckpoint-通过计算获取数据迭代器
- 如果RDD存在检查点或持久化数据,调用父RDD的iterator读取数据;
- 否则,调用RDD的compute函数进行计算;
abstract class RDD[T: ClassTag](
@transient private var _sc: SparkContext,
@transient private var deps: Seq[Dependency[_]]
) extends Serializable with Logging {
private[spark] def computeOrReadCheckpoint(split: Partition, context: TaskContext): Iterator[T] =
{
if (isCheckpointedAndMaterialized) {//存在检查点或持久化数据
//调用父RDD的iterator读取数据
firstParent[T].iterator(split, context)
} else {//不存在存在检查点和持久化数据
//调用RDD的compute函数进行计算
compute(split, context)
}
}
}
3.2.ShuffledRDD#compute
流程:
- 获取RDD依赖;
- 根据依赖和分区去,构建Reader;
- 调用reader的read函数读取数据;
说明:
- shuffleMapTask的第一个RDD,都是ShuffledRDD;
RDD.computeOrReadCheckpoint
函数中,针对shuffleMapTask的第一个RDD,不会存在检查点或持久化数据,需要从上一个stage写出的数据文件中读取数据;- 因此,调用
ShuffledRDD.compute()
函数,从上一个stage写出的数据文件中读取数据; - 如果是第一个stage的第一个RDD,调用
ShuffledRDD.compute()
函数,从外部文件中读取数据;
- 因此,调用
class ShuffledRDD[K: ClassTag, V: ClassTag, C: ClassTag](
@transient var prev: RDD[_ <: Product2[K, V]],
part: Partitioner)
extends RDD[(K, C)](prev.context, Nil) {
override def compute(split: Partition, context: TaskContext): Iterator[(K, C)] = {
//获取RDD的依赖
val dep = dependencies.head.asInstanceOf[ShuffleDependency[K, V, C]]
//根据依赖和分区去,构建Reader
SparkEnv.get.shuffleManager.getReader(dep.shuffleHandle, split.index, split.index + 1, context)
//调用reader的read函数
.read()
.asInstanceOf[Iterator[(K, C)]]
}
}
3.3.RDD#iterator总结
说明:
- 通过RDD的iterator函数作为入口,根据RDD血缘向上追溯,从血缘链第一个RDD(ShuffleRdd)读取数据,后续RDD依次处理数据(根据算子对数据进行计算或转换),直到最后一个RDD将结果数据进行返回;
数据获取流程:
-
调用
RDD.iterator(split: Partition, context: TaskContext)
函数获取RDD对应分区的数据;-
判断数据缓存级别,如果缓存级别为None(不缓存),调用
RDD.computeOrReadCheckpoint()
从检查点读取数据获取直接直接计算数据;- 如果RDD存在检查点或持久化数据,调用父RDD的iterator读取数据;
- 否则,调用RDD的compute函数进行计算;
-
如果缓存级别不为None,调用
RDD.getOrCompute()
通过BlockManager获取缓存数据;- 首先从缓存中读取数据,将读取的数据通过Left进行封装并返回;
- 先从本地读取缓存数据,将成功读取的数据返回;
- 如果从本地读取数据失败,尝试从远程节点读取缓存,将成功读取的数据返回;
- 如果都读取失败,返回None;
- 如果数据读取失败(读取后返回的是None),调用
RDD.computeOrReadCheckpoint()
通过计算获取数据,将获取的数据根据存缓存级别进行缓存,再从缓存中读取数据;- 计算数据缓存成功,将从缓存中读取的数据封装到Left中并返回;
- 计算数据缓存失败,将计算的数据封装到Right中并返回;
- 首先从缓存中读取数据,将读取的数据通过Left进行封装并返回;
-
-
在这个流程中,存在多个循环,实现RDD沿着血缘向上追溯计算的功能;
- 调用
RDD.iterator(split: Partition, context: TaskContext)
函数获取数据时,如果不支持缓存或者支持缓存但是缓存没有数据,调用RDD.computeOrReadCheckpoint()
计算数据;- 此时如果RDD的存在检查点或缓存,调用父RDD的
iterator(split: Partition, context: TaskContext)
函数获取数据;这是一个循环; - 如果RDD的不存在检查点和缓存,执行
compute()
函数,在这个函数中,如果不是ShuffleRDD,都会先调用父RDD的iterator()
函数从父RDD那里获取数据,然后在当前RDD进行处理后返回数据;
- 此时如果RDD的存在检查点或缓存,调用父RDD的
- 上面描述存在一个递归循环,RDD调用iterator函数获取数据,在iterator函数中,都会调用父RDD的iterator函数获取父RDD的数据;这个递归循环将会沿着血缘链追溯到task的第一个RDD的compute函数中;
- task是shufflMapTask,它的第一个RDD肯定是ShuffleRDD,这个RDD的compute函数负责将上一个stage的task写出到磁盘的数据读取过来;这个读取步骤通过
ShuffleReader.read()
函数实现;
- 调用
4.ShuffleReader
4.1.SortShuffleManager#getReader-获取reader
说明:
- 通过startPartition、endPartition指定分区;
- 根据ShuffleHandle、分区构建一个BlockStoreShuffleReader对象;
private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager with Logging {
override def getReader[K, C](
handle: ShuffleHandle,
startPartition: Int,
endPartition: Int,
context: TaskContext): ShuffleReader[K, C] = {
//指定ShuffleHandle、分区
//实例化BlockStoreShuffleReader对象
new BlockStoreShuffleReader(
handle.asInstanceOf[BaseShuffleHandle[K, _, C]], startPartition, endPartition, context)
}
}
private[spark] class BlockStoreShuffleReader[K, C](
handle: BaseShuffleHandle[K, _, C],
startPartition: Int,
endPartition: Int,
context: TaskContext,
serializerManager: SerializerManager = SparkEnv.get.serializerManager,
blockManager: BlockManager = SparkEnv.get.blockManager,
mapOutputTracker: MapOutputTracker = SparkEnv.get.mapOutputTracker)
extends ShuffleReader[K, C] with Logging {
//从handle中获取依赖信息
private val dep = handle.dependency
}
4.2.ShuffleReader#read-读取数据
功能:
- 实现了Shuffle过程数据读取的功能;
说明:
- 构建一个拉取数据的迭代器;
- 从MapOutputTracker中根据shuffleId和分区获取存在数据库的每个节点,对应的数据块列表;
- 本地数据通过BlockManager获取;‘
- 远端数据,通过NettyBlockTransferService进行数据拉取;
- 拉取的数据由FetchResult进行封装,由LinkedBlockingQueue阻塞队列缓存;
- 将迭代器中数据反序列化;
- 数据聚合;
- 已经map端聚合的,使用
Aggregator.combineCombinersByKey()
函数进行数据聚合; - 未map端聚合的,使用
Aggregator.combineValuesByKey()
函数进行数据聚合;
- 已经map端聚合的,使用
- 数据排序;
- 使用ExternalSorter进行数据排序;
- 结果数据封装为InterruptibleIterator迭代器;
private[spark] class BlockStoreShuffleReader[K, C](
handle: BaseShuffleHandle[K, _, C],
startPartition: Int,
endPartition: Int,
context: TaskContext,
serializerManager: SerializerManager = SparkEnv.get.serializerManager,
blockManager: BlockManager = SparkEnv.get.blockManager,
mapOutputTracker: MapOutputTracker = SparkEnv.get.mapOutputTracker)
extends ShuffleReader[K, C] with Logging {
override def read(): Iterator[Product2[K, C]] = {
//创建一个拉取数据块的迭代器
val wrappedStreams = new ShuffleBlockFetcherIterator(
context,
blockManager.shuffleClient,
blockManager,
//根据执行器Id获取块大小
mapOutputTracker.getMapSizesByExecutorId(handle.shuffleId, startPartition, endPartition),
serializerManager.wrapStream,
// Note: we use getSizeAsMb when no suffix is provided for backwards compatibility
SparkEnv.get.conf.getSizeAsMb("spark.reducer.maxSizeInFlight", "48m") * 1024 * 1024,
SparkEnv.get.conf.getInt("spark.reducer.maxReqsInFlight", Int.MaxValue),
SparkEnv.get.conf.get(config.REDUCER_MAX_BLOCKS_IN_FLIGHT_PER_ADDRESS),
SparkEnv.get.conf.get(config.MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM),
SparkEnv.get.conf.getBoolean("spark.shuffle.detectCorrupt", true))
val serializerInstance = dep.serializer.newInstance()
//构建一个数据块的k-v迭代器
val recordIter = wrappedStreams.flatMap { case (blockId, wrappedStream) =>
// 将每个block的数据进行反序列化
serializerInstance.deserializeStream(wrappedStream).asKeyValueIterator
}
// Update the context task metrics for each record read.
val readMetrics = context.taskMetrics.createTempShuffleReadMetrics()
val metricIter = CompletionIterator[(Any, Any), Iterator[(Any, Any)]](
recordIter.map { record =>
readMetrics.incRecordsRead(1)
record
},
context.taskMetrics().mergeShuffleReadMetrics())
// An interruptible iterator must be used here in order to support task cancellation
val interruptibleIter = new InterruptibleIterator[(Any, Any)](context, metricIter)
//数据聚合
val aggregatedIter: Iterator[Product2[K, C]] = if (dep.aggregator.isDefined) {
if (dep.mapSideCombine) {
//已经进行过map端聚合
val combinedKeyValuesIterator = interruptibleIter.asInstanceOf[Iterator[(K, C)]]
dep.aggregator.get.combineCombinersByKey(combinedKeyValuesIterator, context)
} else {
//没有map端聚合
val keyValuesIterator = interruptibleIter.asInstanceOf[Iterator[(K, Nothing)]]
dep.aggregator.get.combineValuesByKey(keyValuesIterator, context)
}
} else {
interruptibleIter.asInstanceOf[Iterator[Product2[K, C]]]
}
// 数据排序
val resultIter = dep.keyOrdering match {
case Some(keyOrd: Ordering[K]) =>
//使用ExternalSorter进行数据排序
val sorter =
new ExternalSorter[K, C, C](context, ordering = Some(keyOrd), serializer = dep.serializer)
sorter.insertAll(aggregatedIter)
context.taskMetrics().incMemoryBytesSpilled(sorter.memoryBytesSpilled)
context.taskMetrics().incDiskBytesSpilled(sorter.diskBytesSpilled)
context.taskMetrics().incPeakExecutionMemory(sorter.peakMemoryUsedBytes)
// Use completion callback to stop sorter if task was finished/cancelled.
context.addTaskCompletionListener[Unit](_ => {
sorter.stop()
})
CompletionIterator[Product2[K, C], Iterator[Product2[K, C]]](sorter.iterator, sorter.stop())
case None =>
aggregatedIter
}
//结果数据封装
resultIter match {
case _: InterruptibleIterator[Product2[K, C]] => resultIter
case _ =>
// Use another interruptible iterator here to support task cancellation as aggregator
// or(and) sorter may have consumed previous interruptible iterator.
new InterruptibleIterator[Product2[K, C]](context, resultIter)
}
}
}
4.2.1.创建一个拉取数据块的迭代器
说明:
- 定义LinkedBlockingQueue阻塞队列作为类属性,缓存结果数据;
- 结果数据由FetchResult进行封装;
在实例化ShuffleBlockFetcherIterator对象时,调用initialize函数进行数据拉取工作,将拉取的结果封装在FetchResult中,缓存在LinkedBlockingQueue结果阻塞队列中;
- 数据块拉取分流
- 将本地数据库缓存在链表结构的localBlocks属性中;
- 返回远程数据块拉取请求数组
- 针对远程数据块,向远端节点发送请求,直到最大远程请求数
- 针对本地数据库,直接本地获取;
final class ShuffleBlockFetcherIterator(
context: TaskContext, //用于指标更新
shuffleClient: ShuffleClient, //用于获取远程块
blockManager: BlockManager, //用于读取本地块的块管理器
blocksByAddress: Iterator[(BlockManagerId, Seq[(BlockId, Long)])], //按BlockManagerId分组的块列表
streamWrapper: (BlockId, InputStream) => InputStream, //用于包装返回的输入流的函数
maxBytesInFlight: Long, //每个远程节点同一时间能够处理的数据量的最大值;
maxReqsInFlight: Int, //每个远程节点同一时间能够处理的请求数量的最大值;
maxBlocksInFlightPerAddress: Int, //从远程节点上获取的数据块数量的最大值;
maxReqSizeShuffleToMem: Long, //一个可以被shuffle到内存的请求的最大大小(以字节为单位)
detectCorrupt: Boolean) //是否检测取到的块中的任何损坏
extends Iterator[(BlockId, InputStream)] with DownloadFileManager with Logging {
//要获取的块的总数。
//这应该等于blocksByAddress中的总块数,因为我们已经在blocksByAddress中过滤掉了零大小的块;
//这应该等于localBlocks + remoteBlocks.size大小
private[this] var numBlocksToFetch = 0
//调用者处理的块的数量
//当numBlocksProcessed == numBlocksToFetch时,迭代器耗尽
private[this] var numBlocksProcessed = 0
private[this] val startTime = System.currentTimeMillis
//要获取的本地块,不包括零大小的块
private[this] val localBlocks = scala.collection.mutable.LinkedHashSet[BlockId]()
//要获取的远程块,不包括零大小的块
private[this] val remoteBlocks = new HashSet[BlockId]()
//保存结果的队列
//这将把org.apache.spark.network.BlockTransferService提供的异步模型转换为同步模型(迭代器)
private[this] val results = new LinkedBlockingQueue[FetchResult]
//当前正在处理的ShuffleBlockFetcherIterator.FetchResult。
//我们跟踪它,以便在处理当前缓冲区时发生运行时异常时释放当前缓冲区。
@volatile private[this] var currentResult: SuccessFetchResult = null
//要发出的获取请求的队列
//我们将逐渐将请求从这里拉出,以确保飞行中的字节数限制为maxBytesInFlight
private[this] val fetchRequests = new Queue[FetchRequest]
//缓存每个节点需要延迟处理的请求队列
private[this] val deferredFetchRequests = new HashMap[BlockManagerId, Queue[FetchRequest]]()
//当前正在处理的数据量
private[this] var bytesInFlight = 0L
//当前正在处理的请求数
private[this] var reqsInFlight = 0
//当前在每个主机:端口中拉取的数据块数据量
private[this] val numBlocksInFlightPerAddress = new HashMap[BlockManagerId, Int]()
//对于不能成功解压的块,它用于保证我们对那些损坏的块最多重试一次
private[this] val corruptedBlocks = mutable.HashSet[BlockId]()
private[this] val shuffleMetrics = context.taskMetrics().createTempShuffleReadMetrics()
//迭代器是否仍然活动
//如果isZombie为真,回调接口将不再将获取的块放入结果中
private[this] var isZombie = false
//存储用于移动远程巨大块的文件的集合
//清理时将删除此集中的文件。这是针对磁盘文件泄漏的防御层
private[this] val shuffleFilesSet = mutable.HashSet[DownloadFile]()
//调用初始化函数
initialize()
private[this] def initialize(): Unit = {
//为清理添加一个任务完成回调(在成功和失败的情况下都调用)
context.addTaskCompletionListener[Unit](_ => cleanup())
//数据块拉取分流,获取远程数据块拉取请求数组
val remoteRequests = splitLocalRemoteBlocks()
//以随机顺序将远程请求添加到数据拉取请求队列中
fetchRequests ++= Utils.randomize(remoteRequests)
assert ((0 == reqsInFlight) == (0 == bytesInFlight),
"expected reqsInFlight = 0 but found reqsInFlight = " + reqsInFlight +
", expected bytesInFlight = 0 but found bytesInFlight = " + bytesInFlight)
//发送请求,直到最大远程请求数
fetchUpToMaxBytes()
val numFetches = remoteRequests.size - fetchRequests.size
logInfo("Started " + numFetches + " remote fetches in" + Utils.getUsedTimeMs(startTime))
// 获取本地block
fetchLocalBlocks()
logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime))
}
}
4.2.1.1.splitLocalRemoteBlocks-数据块拉取分流
功能:
- 统计本地数据块;
- 构建拉取远程数据块的请求并返回;
说明:
- 定义每个请求的数据量阈值;
- 远程请求的数据量大小不超过maxBytesInFlight / 5;
- 使它们小于maxBytesInFlight的原因是允许从最多5个节点进行多次并行读取,而不是阻塞从一个节点读取输出;
- 定义缓存拉取远程数据块的请求的数组;
- 遍历所有的数据库节点,统计本地数据库,构建拉取远程数据库的请求;
- 过滤大小不大于0的数据库
- 针对本地节点;
- 将blockId添加到本地数据块集合中
- 将数据库大小累加到总的需要拉取的数据量中;
- 针对非本地节点;
- 遍历节点的数据块,统计节点数据块的id,数量,大小;
- 当数据量或数据块数量达到阈值时,构建一个数据块拉取请求,并添加到请求数组中;
- 为剩余的数据块构建一个数据块拉取请求,添加到请求数组中;
- 返回拉取远程数据库的请求的数组;
private[spark]
final class ShuffleBlockFetcherIterator(
context: TaskContext,
shuffleClient: ShuffleClient,
blockManager: BlockManager,
blocksByAddress: Iterator[(BlockManagerId, Seq[(BlockId, Long)])],
streamWrapper: (BlockId, InputStream) => InputStream,
maxBytesInFlight: Long,
maxReqsInFlight: Int,
maxBlocksInFlightPerAddress: Int,
maxReqSizeShuffleToMem: Long,
detectCorrupt: Boolean)
extends Iterator[(BlockId, InputStream)] with DownloadFileManager with Logging {
//要获取的块的总数
private[this] var numBlocksToFetch = 0
//要获取的本地块集合,不包括零大小的块
private[this] val localBlocks = scala.collection.mutable.LinkedHashSet[BlockId]()
//要获取的远端数据块集合
private[this] val remoteBlocks = new HashSet[BlockId]()
private[this] def splitLocalRemoteBlocks(): ArrayBuffer[FetchRequest] = {
// 远程请求的长度不超过maxBytesInFlight / 5
//使它们小于maxBytesInFlight的原因是允许从最多5个节点进行多次并行读取,而不是阻塞从一个节点读取输出
val targetRequestSize = math.max(maxBytesInFlight / 5, 1L)
logDebug("maxBytesInFlight: " + maxBytesInFlight + ", targetRequestSize: " + targetRequestSize
+ ", maxBlocksInFlightPerAddress: " + maxBlocksInFlightPerAddress)
//拉取远程块的请求数组
//为了限制传输中的数据量,远程块被进一步分割为大小不超过maxBytesInFlight的拉取请求
val remoteRequests = new ArrayBuffer[FetchRequest]
//遍历按BlockManagerId分组的块列表
for ((address, blockInfos) <- blocksByAddress) {
if (address.executorId == blockManager.blockManagerId.executorId) {//本地数据块
//block大小不大于0的,抛异常
blockInfos.find(_._2 <= 0) match {
case Some((blockId, size)) if size < 0 =>
throw new BlockException(blockId, "Negative block size " + size)
case Some((blockId, size)) if size == 0 =>
throw new BlockException(blockId, "Zero-sized blocks should be excluded.")
case None => // do nothing.
}
//将blockId添加到本地数据块集合中
localBlocks ++= blockInfos.map(_._1)
//需要获取的数据块数量
numBlocksToFetch += localBlocks.size
} else {//非本地数据库
val iterator = blockInfos.iterator
//向当前节点发送的请求数量
var curRequestSize = 0L
//向当前节点请求的block列表
var curBlocks = new ArrayBuffer[(BlockId, Long)]
//遍历每个节点的block数据块
while (iterator.hasNext) {
val (blockId, size) = iterator.next()
//block大小不大于0的,抛异常
if (size < 0) {
throw new BlockException(blockId, "Negative block size " + size)
} else if (size == 0) {
throw new BlockException(blockId, "Zero-sized blocks should be excluded.")
} else {
curBlocks += ((blockId, size))
//将blockId添加到远端数据块集合中
remoteBlocks += blockId
//累加需要获取的数据库数量
numBlocksToFetch += 1
//向当前节点请求的数据大小
curRequestSize += size
}
//目前请求的数据块大小的总和大于目标值:targetRequestSize(实际上是:maxBytesInFlight / 5)
//或者目前的数据块数:curBlocks大于对应远端节点获取shuffle数据块的最大数量
if (curRequestSize >= targetRequestSize ||
curBlocks.size >= maxBlocksInFlightPerAddress) {
//构建拉取数据的请求,将请求添加到请求数组中
remoteRequests += new FetchRequest(address, curBlocks)
logDebug(s"Creating fetch request of $curRequestSize at $address "
+ s"with ${curBlocks.size} blocks")
//初始化curBlocks、curRequestSize
curBlocks = new ArrayBuffer[(BlockId, Long)]
curRequestSize = 0
}
}
// 构建剩余数据块的拉取请求,并添加到请求数组中
if (curBlocks.nonEmpty) {
remoteRequests += new FetchRequest(address, curBlocks)
}
}
}
logInfo(s"Getting $numBlocksToFetch non-empty blocks including ${localBlocks.size}" +
s" local blocks and ${remoteBlocks.size} remote blocks")
//返回请求数组
remoteRequests
}
}
4.2.1.2.fetchUpToMaxBytes-发送远程数据拉取请求
功能:
- 发送拉取数据块的请求;
说明:
- 处理每个节点延迟发送的请求;
- 遍历延迟处理的请求队列的每个节点;
- 判断节点处理能力,发送请求直到节点处理上限;
- 从节点的延迟处理请求队列中取出请求并发送;
- 节点请求全部发送出去后,将节点从延迟处理队列中移除;
- 处理数据拉取请求队列中的请求;
- 判断请求队列余量和阶段处理能力,发送请求到请求处理完毕或者达到节点处理上限;
- 取出请求;
- 确认新增请求是否会超过节点处理能力;
- 超过处理能力,将请求添加到该节点的延迟处理请求队列;
- 为超过处理能力,发送请求;
- 判断请求队列余量和阶段处理能力,发送请求到请求处理完毕或者达到节点处理上限;
- 请求队列余量和节点处理能力的判断;
- 请求队列不为空
- 请求的当前字节数为0 或者 (请求数不超过节点最大请求数 且 数据量不超过节点最大数据量)
- 以上2个条件要求同时满足;
private[spark]
final class ShuffleBlockFetcherIterator(
context: TaskContext,
shuffleClient: ShuffleClient,
blockManager: BlockManager,
blocksByAddress: Iterator[(BlockManagerId, Seq[(BlockId, Long)])],
streamWrapper: (BlockId, InputStream) => InputStream,
maxBytesInFlight: Long,
maxReqsInFlight: Int,
maxBlocksInFlightPerAddress: Int,
maxReqSizeShuffleToMem: Long,
detectCorrupt: Boolean)
extends Iterator[(BlockId, InputStream)] with DownloadFileManager with Logging {
//缓存每个节点需要延迟处理的请求队列
private[this] val deferredFetchRequests = new HashMap[BlockManagerId, Queue[FetchRequest]]()
private def fetchUpToMaxBytes(): Unit = {
//处理每个节点延迟处理的请求
if (deferredFetchRequests.nonEmpty) {
//遍历延迟处理的请求队列的每个节点
for ((remoteAddress, defReqQueue) <- deferredFetchRequests) {
//判断节点处理能力,发送请求直到节点处理上限
while (isRemoteBlockFetchable(defReqQueue) &&
!isRemoteAddressMaxedOut(remoteAddress, defReqQueue.front)) {
//取出请求
val request = defReqQueue.dequeue()
logDebug(s"Processing deferred fetch request for $remoteAddress with "
+ s"${request.blocks.length} blocks")
send(remoteAddress, request)
//节点请求全部发送出去后,将节点从延迟处理队列中移除
if (defReqQueue.isEmpty) {
deferredFetchRequests -= remoteAddress
}
}
}
}
// 判断请求队列余量和节点处理能力,发送请求到请求处理完毕或者达到节点处理上限
while (isRemoteBlockFetchable(fetchRequests)) {
//取出请求
val request = fetchRequests.dequeue()
val remoteAddress = request.address
//确认新增请求是否会超过节点处理能力
if (isRemoteAddressMaxedOut(remoteAddress, request)) {//超过处理能力
logDebug(s"Deferring fetch request for $remoteAddress with ${request.blocks.size} blocks")
//将远端请求添加到延迟处理请求队列
val defReqQueue = deferredFetchRequests.getOrElse(remoteAddress, new Queue[FetchRequest]())
defReqQueue.enqueue(request)
deferredFetchRequests(remoteAddress) = defReqQueue
} else {
//发送数据拉取请求
send(remoteAddress, request)
}
}
def send(remoteAddress: BlockManagerId, request: FetchRequest): Unit = {
sendRequest(request)
numBlocksInFlightPerAddress(remoteAddress) =
numBlocksInFlightPerAddress.getOrElse(remoteAddress, 0) + request.blocks.size
}
//判断是否可以执行数据块拉取请求
def isRemoteBlockFetchable(fetchReqQueue: Queue[FetchRequest]): Boolean = {
fetchReqQueue.nonEmpty && //请求队列不为空
(bytesInFlight == 0 || //请求的当前字节数为0
(reqsInFlight + 1 <= maxReqsInFlight && //请求数不超过节点最大请求数
bytesInFlight + fetchReqQueue.front.size <= maxBytesInFlight)) //数据量不超过节点最大数据量
}
def isRemoteAddressMaxedOut(remoteAddress: BlockManagerId, request: FetchRequest): Boolean = {
//节点处理数据量超过节点最大可处理数据量
numBlocksInFlightPerAddress.getOrElse(remoteAddress, 0) + request.blocks.size >
maxBlocksInFlightPerAddress
}
}
}
4.2.1.2.1.ShuffleBlockFetcherIterator#send
说明:
- 更新当前正在处理的数据量、请求数;
- 定义数据拉取成功的监听器:数据拉取成功后的处理流程;
- 数据抓取成功,将数据buffer封装到SuccessFetchResult只不过,添加到结果队列;
- 数据抓取失败,将失败的blcokId封装到FailureFetchResult,添加到结果队列;
- 通过NettyBlockTransferService#fetchBlocks进行数据拉去工作,执行数据拉取监听器;
- 当请求的数据量超过maxReqSizeShuffleToMem限制时,将数据直接落地到磁盘文件,不缓存在内存;
- 更新节点拉取数据量;
private[spark]
final class ShuffleBlockFetcherIterator(
context: TaskContext,
shuffleClient: ShuffleClient,
blockManager: BlockManager,
blocksByAddress: Iterator[(BlockManagerId, Seq[(BlockId, Long)])],
streamWrapper: (BlockId, InputStream) => InputStream,
maxBytesInFlight: Long,
maxReqsInFlight: Int,
maxBlocksInFlightPerAddress: Int,
maxReqSizeShuffleToMem: Long,
detectCorrupt: Boolean)
extends Iterator[(BlockId, InputStream)] with DownloadFileManager with Logging {
def send(remoteAddress: BlockManagerId, request: FetchRequest): Unit = {
//调用sendRequest实现请求发送
sendRequest(request)
//更新节点拉取数据量
numBlocksInFlightPerAddress(remoteAddress) =
numBlocksInFlightPerAddress.getOrElse(remoteAddress, 0) + request.blocks.size
}
private[this] def sendRequest(req: FetchRequest) {
logDebug("Sending request for %d blocks (%s) from %s".format(
req.blocks.size, Utils.bytesToString(req.size), req.address.hostPort))
//更新当前正在处理的数据量、请求数
bytesInFlight += req.size
reqsInFlight += 1
// so we can look up the size of each blockID
val sizeMap = req.blocks.map { case (blockId, size) => (blockId.toString, size) }.toMap
//正在处理的blockId集合
val remainingBlocks = new HashSet[String]() ++= sizeMap.keys
val blockIds = req.blocks.map(_._1.toString)
//节点地址
val address = req.address
//定义数据拉取成功的监听器:数据拉取成功后的处理流程
val blockFetchingListener = new BlockFetchingListener {
override def onBlockFetchSuccess(blockId: String, buf: ManagedBuffer): Unit = {
// Only add the buffer to results queue if the iterator is not zombie,
// i.e. cleanup() has not been called yet.
ShuffleBlockFetcherIterator.this.synchronized {
if (!isZombie) {
//将buf的引用次数 + 1
buf.retain()
//从正在处理的blockId集合中移除数据拉取成功的blockId
remainingBlocks -= blockId
//将数据buffer封装到SuccessFetchResult只不过,添加到结果队列
results.put(new SuccessFetchResult(BlockId(blockId), address, sizeMap(blockId), buf,
remainingBlocks.isEmpty))
logDebug("remainingBlocks: " + remainingBlocks)
}
}
logTrace("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime))
}
//定义数据拉取失败的监听器:数据拉取失败后的处理流程
override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = {
logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e)
//将失败的blcokId封装到FailureFetchResult,添加到结果队列
results.put(new FailureFetchResult(BlockId(blockId), address, e))
}
}
//通过NettyBlockTransferService#fetchBlocks进行数据拉去工作,执行数据拉取监听器
//当请求太大时,将远程数据块读取到磁盘
//由于shuffle数据已经通过网络加密和压缩,我们可以直接获取数据并将其写入文件
if (req.size > maxReqSizeShuffleToMem) {
shuffleClient.fetchBlocks(address.host, address.port, address.executorId, blockIds.toArray,
blockFetchingListener, this)
} else {
shuffleClient.fetchBlocks(address.host, address.port, address.executorId, blockIds.toArray,
blockFetchingListener, null)
}
}
}
4.2.1.3.fetchLocalBlocks-拉取本地数据
说明:
- 遍历本地数据库,将取出的数据添加到结果对象中;
- 通过BlockManager中获取数据buffer;
- 将buf的引用次数 + 1;
- 将buf封装到成功对象SuccessFetchResult中,添加到结果队列;
private[spark]
final class ShuffleBlockFetcherIterator(
context: TaskContext,
shuffleClient: ShuffleClient,
blockManager: BlockManager,
blocksByAddress: Iterator[(BlockManagerId, Seq[(BlockId, Long)])],
streamWrapper: (BlockId, InputStream) => InputStream,
maxBytesInFlight: Long,
maxReqsInFlight: Int,
maxBlocksInFlightPerAddress: Int,
maxReqSizeShuffleToMem: Long,
detectCorrupt: Boolean)
extends Iterator[(BlockId, InputStream)] with DownloadFileManager with Logging {
//拉取结果队列
private[this] val results = new LinkedBlockingQueue[FetchResult]
private[this] def fetchLocalBlocks() {
logDebug(s"Start fetching local blocks: ${localBlocks.mkString(", ")}")
val iter = localBlocks.iterator
//遍历数据块
while (iter.hasNext) {
val blockId = iter.next()
try {
//通过BlockManager中获取数据buffer
val buf = blockManager.getBlockData(blockId)
shuffleMetrics.incLocalBlocksFetched(1)
shuffleMetrics.incLocalBytesRead(buf.size)
//将buf的引用次数 + 1
buf.retain()
//将buf封装到成功对象SuccessFetchResult中,添加到结果队列
results.put(new SuccessFetchResult(blockId, blockManager.blockManagerId,
buf.size(), buf, false))
} catch {
case e: Exception =>
// If we see an exception, stop immediately.
logError(s"Error occurred while fetching local blocks", e)
//抛异常,构建失败对象FailureFetchResult,添加到结果队列
results.put(new FailureFetchResult(blockId, blockManager.blockManagerId, e))
return
}
}
}
}
4.3.ShuffleReader#read总结
说明:
- 通过SortShuffleManager(ShuffleManager的子类)对ShuffleReader进行管理;ShuffleManager提供注册ShuffleHandle能力,根据ShuffleHandle实例化并获取ShuffleReader的能力;
- 通过
ShuffleReader.read()
函数,实现读取节点(本地节点、远程节点)文本数据的功能;
数据读取流程总结:
- 从MapOutputTracker中根据shuffleId和分区获取存在数据库的每个节点,对应的数据块列表;
- 数据读取时,根据数据所在位置分为本地数据和远程节点数据,本地数据通过BlockManager获取;远程节点数据通过NettyBlockTransferService#fetchBlocks进行数据拉取;然后将数据封装到ShuffleBlockFetcherIterator迭代器(迭代器中阻塞队列缓存读取的数据结构)中返回;
- 此时相当于map端数据获取完成,接着对迭代器中获取的数据进行反序列化、数据聚合、数据排序等数据处理操作;
- 最后将数据封装到InterruptibleIterator迭代器中返回;交给后续RDD进行处理;
注意事项:
- 对于所有待处理远程数据拉取请求,都由类型为Queue的fetchRequests属性提供请求缓存功能;
- 对应所有待处理的本地数据拉取请求,都由类型为LinkedHashSet的localBlocks属性缓存所有blockId;
- 每个远端数据拉取请求能够拉取的数据量受spark.reducer.maxSizeInFlight、spark.reducer.maxBlocksInFlightPerAddress这2个参数限制;
- spark.reducer.maxSizeInFlight:
- 用于设置
shuffle read
任务的buff缓冲区大小,该缓冲区决定一次可以拉取多少数据;默认48M; - 一个请求一次可以拉取1/5个spark.reducer.maxSizeInFlight数据量的数据;默认9.6M;
- 用于设置
- spark.reducer.maxBlocksInFlightPerAddress:指定每个请求能够从远程节点拉取数据块的数量;默认Int.MaxValue;
- 一个请求一次最多可以拉取spark.reducer.maxBlocksInFlightPerAddress个数据块,默认Int.MaxValue;
- spark.reducer.maxSizeInFlight:
- 每个远程节点能个接受的数据拉取请求数也是有限制的,由spark.reducer.maxReqsInFlight、spark.reducer.maxSizeInFlight进行限制;
- 处理的请求数不能超过spark.reducer.maxReqsInFlight个,默认Int.MaxValue;
- 处理的数据量不能超过spark.reducer.maxSizeInFlight,默认48M;
5.总结
- 任务上游数据位置信息从MapOutputTracker中根据shuffleId和分区获取,返回存在数据块的每个节点,对应的数据块列表;
- shuffleMapTask任务数据的获取是一个从任务最后一个RDD沿着血缘向上追溯层层获取数据的过程,从底层RDD向上层父RDD要数据,父RDD向他的父RDD要数据,一直要到某个RDD存在缓存数据或者checkpoint数据截止,否则一直递归要到task任务的第一个RDD(ShuffleRDD),然后开始向下计算传递数据;
- task任务第一个RDD时ShuffleRDD,该RDD从远程节点或本地节点的数据文件中获取数据;这些数据文件是当前task的上游task通过ShuffleWriter#write函数写入进去的;
6.参考资料
Spark存储分析 - 存储级别
Spark Core源码精读计划#30:终于讲到的BlockManager及其管理的块读取流程