Apache Spark源码走读之20 -- ShuffleMapTask计算结果的保存与读取

时间:2022-08-24 01:13:45

欢迎转载,转载请注明出处,徽沪一郎。

概要

ShuffleMapTask的计算结果保存在哪,随后Stage中的task又是如何知道从哪里去读取的呢,这个过程一直让我困惑不已。

用比较通俗一点的说法来解释一下Shuffle数据的写入和读取过程

  1. 每一个task负责处理一个特定的data partition
  2. task在初始化的时候就已经明确处理结果可能会产生多少个不同的data partition
  3. 利用partitioner函数,task将处理结果存入到不同的partition,这些数据存放在当前task执行的机器上
  4. 假设当前是stage 2有两个task, stage 2可能输出4个不同的data partition, task 0和task 1各自运行于不同的机器上,task 0中的部分处理结果会存入到data partition 0,task 1的部分处理结果也可能存入到data partition 0.
  5. 由于stage 2产生了4个不同的data partition, 后续stage 1中的task个数就为4. task 0 就负责读取data partition 0的数据,对于(stage1, task0)来说,所要读取的data partition 0的内容由task 0和task 1中的partition 0共同组成。
  6. 现在问题的关键转换成为(stage_1, task_0)如何知道(stage_2, task_x)有没有相应的输出是属于data partition 0的呢?这个问题的解决就是MapStatus
  7. 每一个ShuffleMapTask在执行结束,都会上报一个MapStatus,在MapStatus中会反应出朝哪些data partition写入了数据,写入了数据则size为非零值,否则为零值
  8. (stage_1,task_0)会去获取stage_2中所有task的MapStatus,以判定(stage_2, task_x)产生的数据中有自己需要读入的内容
  9. 假设(stage_1,task_0)知道(stage_2, task_0)生成了data partition 0中的数据,于是去(stage_2, task_0)运行时的机器去获取具体的数据,如果恰巧这个时候远端机器已经挂掉了,获取失败,怎么办?
  10. 上报异常,由DAGScheduler重新调度(stage_2,task_0),重新生成所需要的数据。
  11. Spark不像Hadoop中的MapReduce有一个明显的combine阶段,在spark中combine过程有两次调用,一是Shuffle数据写入过程,另一个是Shuffle数据读取过程。

如果能够明白上述的过程,并对应到相应的代码,那就无须看下述的详细解释了。

好了,让我们开始代码跟踪吧。

数据写入过程

Apache Spark源码走读之20 -- ShuffleMapTask计算结果的保存与读取

数据写入动作最原始的触发点是ShuffleMapTask.runTask函数,看一看源码先。

  override def runTask(context: TaskContext): MapStatus = {
metrics = Some(context.taskMetrics)
var writer: ShuffleWriter[Any, Any] = null
try {
val manager = SparkEnv.get.shuffleManager
writer = manager.getWriter[Any, Any](dep.shuffleHandle, partitionId, context)
writer.write(rdd.iterator(split, context).asInstanceOf[Iterator[_
if (writer != null) {
writer.stop(success = false)
}
throw e
} finally {
context.executeOnCompleteCallbacks()
}
}

managerGetWriter返回的是HashShuffleWriter,所以调用过程是ShuffleMapTask.runTask->HashShuffleWriter.write->BlockObjectWriter.write. 注意dep.mapSideCombine这一分支判断。ReduceByKey(_ + _)中的(_ + _)在此处被执行一次,另一次执行是在read过程。

  override def write(records: Iterator[_ <: Product2[K, V]]): Unit = {
val iter = if (dep.aggregator.isDefined) {
if (dep.mapSideCombine) {
dep.aggregator.get.combineValuesByKey(records, context)
} else {
records
}
} else if (dep.aggregator.isEmpty && dep.mapSideCombine) {
throw new IllegalStateException("Aggregator is empty for map-side combine")
} else {
records
} for (elem <- iter) {
val bucketId = dep.partitioner.getPartition(elem._1)
shuffle.writers(bucketId).write(elem)
}

HashShuffleWriter.write中主要处理两件事

  1. 判断是否需要进行聚合,比如<hello,1>和<hello,1>都要写入的话,那么先生成<hello,2>然后再进行后续的写入工作
  2. 利用Partitioner函数来决定<k,val>写入到哪一个文件中

Partitioner是在什么时候注入的,RDD抽象类中,Partitioner为空?以reduceByKey为例,HashPartitioner会在后面combineByKey的代码创建ShuffledRDD的时候作为ShuffledRDD的构造函数传入。

  def reduceByKey(func: (V, V) => V, numPartitions: Int): RDD[(K, V)] = {
reduceByKey(new HashPartitioner(numPartitions), func)
}

Stage在创建的时候通过构造函数入参明确需要从多少Partition读取数据,生成的Partition会有多少。看一看Stage的构造函数,读取的分区数目由RDD.partitions.size决定,输出的partitions由shuffleDep决定。

private[spark] class Stage(
val id: Int,
val rdd: RDD[_],
val numTasks: Int,
val shuffleDep: Option[ShuffleDependency[_, _, _]], // Output shuffle if stage is a map stage
val parents: List[Stage],
val jobId: Int,
val callSite: CallSite)
extends Logging {
val isShuffleMap = shuffleDep.isDefined
val numPartitions = rdd.partitions.size
val outputLocs = Array.fill[List[MapStatus]](numPartitions)(Nil)
var numAvailableOutputs = 0
private var nextAttemptId = 0

回到数据写入的问题上来,结果写入时的一个主要问题就是已经知道shuffle_id, map_id和要写入的elem,如何找到对应的写入文件。每一个临时文件由三元组(shuffle_id,map_id,reduce_id)来决定,当前已经知道了两个,还剩下一下reduce_id待确定。

reduce_id是使用partitioner计算出来的结果,输入的是elem的键值。也就是dep.partitioner.getPartition(elem._1)。 根据计算出来的bucketid找到对应的writer,然后真正写入。

在HashShuffleWriter.write中使用到的shuffle由ShuffleBlockManager的forMapTask函数生成,注意forMapTask中产生writers的代码逻辑。

每个writer分配一下文件, 文件名由三元组(shuffle_id,map_id,reduce_id)组成,如果知道了这个三元组就可以找到对应的文件。

如果consolidation没有打开,那么在一个task中,有多少个输出的partition就会有多少个中间文件。

      val writers: Array[BlockObjectWriter] = if (consolidateShuffleFiles) {
fileGroup = getUnusedFileGroup()
Array.tabulate[BlockObjectWriter](numBuckets) { bucketId =>
val blockId = ShuffleBlockId(shuffleId, mapId, bucketId)
blockManager.getDiskWriter(blockId, fileGroup(bucketId), serializer, bufferSize)
}
} else {
Array.tabulate[BlockObjectWriter](numBuckets) { bucketId =>
val blockId = ShuffleBlockId(shuffleId, mapId, bucketId)
val blockFile = blockManager.diskBlockManager.getFile(blockId)
// Because of previous failures, the shuffle file may already exist on this machine.
// If so, remove it.
if (blockFile.exists) {
if (blockFile.delete()) {
logInfo(s"Removed existing shuffle file $blockFile")
} else {
logWarning(s"Failed to remove existing shuffle file $blockFile")
}
}
blockManager.getDiskWriter(blockId, blockFile, serializer, bufferSize)
}
}

getFile负责将三元组(shuffle_id,map_id,reduce_id)映射到文件名

def getFile(filename: String): File = {
// Figure out which local directory it hashes to, and which subdirectory in that
val hash = Utils.nonNegativeHash(filename)
val dirId = hash % localDirs.length
val subDirId = (hash / localDirs.length) % subDirsPerLocalDir // Create the subdirectory if it doesn't already exist
var subDir = subDirs(dirId)(subDirId)
if (subDir == null) {
subDir = subDirs(dirId).synchronized {
val old = subDirs(dirId)(subDirId)
if (old != null) {
old
} else {
val newDir = new File(localDirs(dirId), "%02x".format(subDirId))
newDir.mkdir()
subDirs(dirId)(subDirId) = newDir
newDir
}
}
} new File(subDir, filename)
} def getFile(blockId: BlockId): File = getFile(blockId.name)

产生的文件在哪呢,如果没有更改默认的配置,生成的目录结构类似于下

/tmp/spark-local-20140723092540-7f24
/tmp/spark-local-20140723092540-7f24/0d
/tmp/spark-local-20140723092540-7f24/0d/shuffle_0_0_1
/tmp/spark-local-20140723092540-7f24/0d/shuffle_0_1_0
/tmp/spark-local-20140723092540-7f24/0c
/tmp/spark-local-20140723092540-7f24/0c/shuffle_0_0_0
/tmp/spark-local-20140723092540-7f24/0e
/tmp/spark-local-20140723092540-7f24/0e/shuffle_0_1_1

当所有的数据写入文件并提交以后,还需要生成MapStatus汇报给driver application. MapStatus在哪生成的呢?commitWritesAndBuildStatus就干这活。

调用关系HashShuffleWriter.stop->commitWritesAndBuildStatus

private def commitWritesAndBuildStatus(): MapStatus = {
// Commit the writes. Get the size of each bucket block (total block size).
var totalBytes = 0L
var totalTime = 0L
val compressedSizes = shuffle.writers.map { writer: BlockObjectWriter =>
writer.commit()
writer.close()
val size = writer.fileSegment().length
totalBytes += size
totalTime += writer.timeWriting()
MapOutputTracker.compressSize(size)
} // Update shuffle metrics.
val shuffleMetrics = new ShuffleWriteMetrics
shuffleMetrics.shuffleBytesWritten = totalBytes
shuffleMetrics.shuffleWriteTime = totalTime
metrics.shuffleWriteMetrics = Some(shuffleMetrics) new MapStatus(blockManager.blockManagerId, compressedSizes)
}

compressedSize是一个非常让人疑惑的地方,原因慢慢道来,先看一下MapStatus的构造函数

class MapStatus(var location: BlockManagerId, var compressedSizes: Array[Byte])

compressedSize是一个byte数组,每一个byte反应了该partiton中的数据大小。如Array(0)=128就表示在data partition 0中有128byte数据。

问题的问题是一个byte只能表示255,如果超过255怎么办呢?

当当当,数学闪亮登场了,注意到compressSize没,通过转换将2^8变换为1.1^256。一下子由255byte延伸到近35G.

看一看这神奇的compressSize函数吧,只是聊聊几行代码而已。

  def compressSize(size: Long): Byte = {
if (size == 0) {
0
} else if (size <= 1L) {
1
} else {
math.min(255, math.ceil(math.log(size) / math.log(LOG_BASE)).toInt).toByte
}
}

ShuffleMapTask运行结束时,会将MapStatus结果封装在StatusUpdate消息中汇报给SchedulerBackend, 由DAGScheduler在handleTaskCompletion函数中将MapStatus加入到相应的Stage。这一过程略过,不再详述。

MapOutputTrackerMaster会保存所有最新的MapStatus.

只画张图来表示存储之后的示意。

Apache Spark源码走读之20 -- ShuffleMapTask计算结果的保存与读取

数据读取过程

ShuffledRDD.compute函数是读取过程的触发点。

  override def compute(split: Partition, context: TaskContext): Iterator[P] = {
val dep = dependencies.head.asInstanceOf[ShuffleDependency[K, V, C]]
SparkEnv.get.shuffleManager.getReader(dep.shuffleHandle, split.index, split.index + 1, context)
.read()
.asInstanceOf[Iterator[P]]
}

shuffleManager.getReader返回的是HashShuffleReader,所以看一看HashShuffleReader中的read函数的具体实现。

read函数处理逻辑中需要注意到一点即combine过程有可能会被再次执行。注意dep.aggregator.isDefined这一分支判断。ReduceByKey(_ + _)中的(_ + _)在此处被执行。

override def read(): Iterator[Product2[K, C]] = {
val iter = BlockStoreShuffleFetcher.fetch(handle.shuffleId, startPartition, context,
Serializer.getSerializer(dep.serializer)) if (dep.aggregator.isDefined) {
if (dep.mapSideCombine) {
new InterruptibleIterator(context, dep.aggregator.get.combineCombinersByKey(iter, context))
} else {
new InterruptibleIterator(context, dep.aggregator.get.combineValuesByKey(iter, context))
}
} else if (dep.aggregator.isEmpty && dep.mapSideCombine) {
throw new IllegalStateException("Aggregator is empty for map-side combine")
} else {
iter
}
}

一路辗转,终于来到了读取过程中非常关键的所在BlockStoreShuffleFetcher。

BlockStoreShuffleFetcher需要回答如下问题

  1. 所要获取的mapid的mapstatus的内容是什么
  2. 根据获得的mapstatus去相应的blockmanager获取具体的数据
 val blockManager = SparkEnv.get.blockManager

  val startTime = System.currentTimeMillis
val statuses = SparkEnv.get.mapOutputTracker.getServerStatuses(shuffleId, reduceId)
logDebug("Fetching map output location for shuffle %d, reduce %d took %d ms".format(
shuffleId, reduceId, System.currentTimeMillis - startTime)) val splitsByAddress = new HashMap[BlockManagerId, ArrayBuffer[(Int, Long)]]
for (((address, size), index)
(address, splits.map(s => (ShuffleBlockId(shuffleId, s._1, reduceId), s._2)))
}
val blockFetcherItr = blockManager.getMultiple(blocksByAddress, serializer)
val itr = blockFetcherItr.flatMap(unpackBlock)

一个ShuffleMapTask会生成一个MapStatus,MapStatus中含有当前ShuffleMapTask产生的数据落到各个Partition中的大小。如果大小为0,则表示该分区没有数据产生。MapStatus中另一个重要的成员变量就是BlockManagerId,该变量表示目标数据在哪个BlockManager当中。

MapoutputTrackerMaster拥有最新的MapStatus信息,为了执行效率,MapoutputTrackerWorker会定期更新数据到本地,所以MapoutputTracker先从本地查找,如果找不到再从MapoutputTrackerMaster上同步最新数据。

索引即是reduceId,如果array(0) == 0,就表示上一个ShuffleMapTask中生成的数据中没有任意的内容可以作为reduceId为0的ResultTask的输入。如果不能理解,返回仔细看一下MapStatus的结构图。

BlockManager.getMultiple用于读取BlockManager中的数据,根据配置确定生成tNettyBlockFetcherIterator还是BasicBlockFetcherIterator。

如果所要获取的文件落在本地,则调用getLocal读取,否则发送请求到远端blockmanager。看一下BlockFetcherIterator的initialize函数

    override def initialize() {
// Split local and remote blocks.
val remoteRequests = splitLocalRemoteBlocks()
// Add the remote requests into our queue in a random order
fetchRequests ++= Utils.randomize(remoteRequests) // Send out initial requests for blocks, up to our maxBytesInFlight
while (!fetchRequests.isEmpty &&
(bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) {
sendRequest(fetchRequests.dequeue())
} val numFetches = remoteRequests.size - fetchRequests.size
logInfo("Started " + numFetches + " remote fetches in" + Utils.getUsedTimeMs(startTime)) // Get Local Blocks
startTime = System.currentTimeMillis
getLocalBlocks()
logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms")
}

至此,数据读取的正常流程讲述完毕。

数据读取异常

如果数据读取中碰到异常怎么办?比如,

  1. 已知(stage_2,task_0)产生的parition_0的数据在机器m1, 当前任务在m2执行,于是从m2向m1发起远程获取请求,如果m2中拥有目标数据的JVM进程异常退出,则相应的目标数据无法获取。

如果无法获取目标数据,就会上报FetchFailedException.

    def unpackBlock(blockPair: (BlockId, Option[Iterator[Any]])) : Iterator[T] = {
val blockId = blockPair._1
val blockOption = blockPair._2
blockOption match {
case Some(block) => {
block.asInstanceOf[Iterator[T]]
}
case None => {
blockId match {
case ShuffleBlockId(shufId, mapId, _) =>
val address = statuses(mapId.toInt)._1
throw new FetchFailedException(address, shufId.toInt, mapId.toInt, reduceId)
case _ =>
throw new SparkException(
"Failed to get block " + blockId + ", which is not a shuffle block")
}
}
}
}

FetchFailedExecption会被包装在StatutsUpdate上报给SchedulerBackend,然后一路处理下去,最终将丢失目标数据的归属Task重新提交。比如当前是(stage_1, task_0),需要读取(stage_2, task_1)产生的目标数据,但是对应的目标数据丢失,这个时候就需要将(stage_2, task_1)重新提交运行。

注意DAGScheduler中的FetchFailed处理分支,一路跟踪下去就会看到任务被重新提交了

  case FetchFailed(bmAddress, shuffleId, mapId, reduceId) =>
// Mark the stage that the reducer was in as unrunnable
val failedStage = stageIdToStage(task.stageId)
runningStages -= failedStage
// TODO: Cancel running tasks in the stage
logInfo("Marking " + failedStage + " (" + failedStage.name +
") for resubmision due to a fetch failure")
// Mark the map whose fetch failed as broken in the map stage
val mapStage = shuffleToMapStage(shuffleId)
if (mapId != -1) {
mapStage.removeOutputLoc(mapId, bmAddress)
mapOutputTracker.unregisterMapOutput(shuffleId, mapId, bmAddress)
}
logInfo("The failed fetch was from " + mapStage + " (" + mapStage.name +
"); marking it for resubmission")
if (failedStages.isEmpty && eventProcessActor != null) {
// Don't schedule an event to resubmit failed stages if failed isn't empty, because
// in that case the event will already have been scheduled. eventProcessActor may be
// null during unit tests.
import env.actorSystem.dispatcher
env.actorSystem.scheduler.scheduleOnce(
RESUBMIT_TIMEOUT, eventProcessActor, ResubmitFailedStages)
}
failedStages += failedStage
failedStages += mapStage
// TODO: mark the executor as failed only if there were lots of fetch failures on it
if (bmAddress != null) {
handleExecutorLost(bmAddress.executorId, Some(task.epoch))
}

文件清除

生成的中间数据是在什么时候被清除的呢?

当Driver Application退出的时候,该Application生成的临时文件将会被一一清除,注意是application结束生命,不是job。一个application可以包含一至多个job。

实验

以local-cluster方式运行spark-shell,观察/tmp/spark-local*目录下的文件变化,具体指令如下

MASTER=local-cluster[2,2,512] bin/spark-shell
#进入spark-shell之后,输入
sc.textFile("README.md").flatMap(_.split(" ")).map(w=>(w,1)).reduceByKey(_ + _)

小结

Shuffle数据的写入和读取是Spark Core这一部分最为复杂的内容,彻底了解该部分内容才能深刻意识到Spark实现的精髓所在。