本篇文章以RDD.aggregateByKey引起的SortShuffleWriter为例说明Shuffle map端的原理和实现,为了便于说明问题这里的所有执行流程都是默认执行流程
为了便于说明问题,本文中Stage1是shuffle map操作所在的Stage,Stage2是shuffle reduce操作所在的Stage
Spark Shuffle map操作可以用如上时序图表示。(时序图可放大显示)
如上的时序图所示,Shuffle任务通过调用ShuffleMapTask.runTask执行Shuffle的map操作,将分区的数据Shuffle到本地文件,代码如下:
override def runTask(context: TaskContext): MapStatus = {
// Deserialize the RDD using the broadcast variable.
val deserializeStartTime = System.currentTimeMillis()
val ser = SparkEnv.get.closureSerializer.newInstance()
val (rdd, dep) = ser.deserialize[(RDD[_], ShuffleDependency[_, _, _])](
ByteBuffer.wrap(taskBinary.value), Thread.currentThread.getContextClassLoader)
_executorDeserializeTime = System.currentTimeMillis() - deserializeStartTime
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)
/*
* 对ShuffledRDD依赖的RDD1进行shuffle map操作,把RDD1的数据先combine,然后写入到shuffle 文件
* */
writer.write(rdd.iterator(partition, context).asInstanceOf[Iterator[_ <: Product2[Any, Any]]])//rdd.iterator读取并处理数据,把处理结果返回
/*
* 返回一个数组,map操作的分区数据shuffle map后会把数据Shuffle到reduce操作的每个分区,每个分区一部分数据
* 数组里的一个元素是map操作的分区数据shuffle map后在reduce操作的一个分区的数据量
* */
return writer.stop(success = true).get
} catch {
case e: Exception =>
try {
if (writer != null) {
writer.stop(success = false)
}
} catch {
case e: Exception =>
log.debug("Could not stop writer", e)
}
throw e
}
}
默认情况下,Spark使用SortShuffleWriter.write将数据写入到Shuffle文件。
SortShuffleWriter.write做2件工作:
1. 在Map端对数据进行分区内的合并
2. 在Map端对合并后一个分区的数据根据Key进行排序
SortShuffleWriter.write方法的源码如下:
override def write(records: Iterator[Product2[K, V]]): Unit = {
<pre name="code" class="java">override def write(records: Iterator[Product2[K, V]]): Unit = {
/*
* aggregateByKey默认情况下它的ShuffleDependency
* */
sorter = if (dep.mapSideCombine) {
require(dep.aggregator.isDefined, "Map-side combine without Aggregator specified!")
new ExternalSorter[K, V, C](
dep.aggregator, Some(dep.partitioner), dep.keyOrdering, dep.serializer)//这几个参数在创建ShuffledRDD传入,创建ShuffleRDD的这些参数在这里用到了
} else if (SortShuffleWriter.shouldBypassMergeSort(
SparkEnv.get.conf, dep.partitioner.numPartitions, aggregator = None, keyOrdering = None)) {
// If there are fewer than spark.shuffle.sort.bypassMergeThreshold partitions and we don't
// need local aggregation and sorting, write numPartitions files directly and just concatenate
// them at the end. This avoids doing serialization and deserialization twice to merge
// together the spilled files, which would happen with the normal code path. The downside is
// having multiple files open at a time and thus more memory allocated to buffers.
new BypassMergeSortShuffleWriter[K, V](SparkEnv.get.conf, blockManager, dep.partitioner,
writeMetrics, Serializer.getSerializer(dep.serializer))
} else {
// In this case we pass neither an aggregator nor an ordering to the sorter, because we don't
// care whether the keys get sorted in each partition; that will be done on the reduce side
// if the operation being run is sortByKey.
new ExternalSorter[K, V, V](
aggregator = None, Some(dep.partitioner), ordering = None, dep.serializer)
}
/*
* 根据ShuffledRDD的ShuffleHandler对paritition的数据进行处理
* 1. 将partition的数据存放到ExternalSorter.map中,ExternalSorter.map是一个PartitionedAppendOnlyMap对象
* 2. 如果partition的数据量太大,超过了允许的最大内存,则将ExternalSorter.map中的数据进行排序,然后将这些数据spill到磁盘文件
* */
sorter.insertAll(records)
// Don't bother including the time to open the merged output file in the shuffle write time,
// because it just opens a single file, so is typically too fast to measure accurately
// (see SPARK-3570).
//返回输出的文件对象,文件名根据shuffleid、mapId、reduceId命名,文件路径在BlockManager的DiskStore存储文件的位置
val outputFile = shuffleBlockResolver.getDataFile(dep.shuffleId, mapId)
//blockId和outputFile的文件名生成算法一样,传入的参数也一样,所以blockId和outputFile的文件名相同
val blockId = ShuffleBlockId(dep.shuffleId, mapId, IndexShuffleBlockResolver.NOOP_REDUCE_ID)
/*
* sorter.writePartitionedFile将Stage1一个partition(标记为partition1)的每条数据的Key根据ShuffledRDD的dep.partitioner进行运算,运算出
* 每个Key在Stage2的ShuffledRDD的partition(标记为partition2),将属于partition2的所有数据根据排序算法排序后,顺序连续写到outputFile的某个位置
* 返回一个数组,数组中的一个元素表示每个partition2的数据写到outputFile中的起始地址和这个partition2的数据长度
* */
val partitionLengths = sorter.writePartitionedFile(blockId, context, outputFile)
/*
* 将Shuffle map后Stage2每个partition在outputFile的起始地址记录到index文件中
* */
shuffleBlockResolver.writeIndexFile(dep.shuffleId, mapId, partitionLengths)
/*
* mapStatus是ShuffleMapTask的返回值
* */
mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths)
}
SortShuffleWriter.write方法里面使用到了表示本次Shuffle操作数据结构BaseShuffleHandle,从BaseShuffleHandle里面拿到本次Shuffle操作所需要的依赖关系ShuffleDependency。ShuffledRDD通过使用ShuffleDependency数据结构,对Stage1 RDD分区的数据进行Shuffle write操作。ShuffleDependency提供了如下几个关键数据:
1. Aggregator对象 Aggregator对象描述了相同Key的数据如何合并,包括分区内的合并和分区间的合并,在shuffle map阶段只执行分区内的合并
2. Partitioner对象 描述了ShuffledRDD是如何分区的,如果没有指定分区信息,默认情况是HashPartitioner
3. Ordering对象 描述了Key是如何排序的,aggregateByKey方法不会设置这一项
4. Serializer对象 描述了分区的数据是如何序列化的。在2个地方用到,第一地方是将Shuffle map结束的时候将它的执行结果写到shuffle文件,这个文件在节点的本地磁盘;第二个地方是Stage1一个分区的数据太大,存储在PartitionedAppendOnly中的数据占有空间超过了能申请的内存,则需要将这些数据spill到磁盘
SortShuffleWriter.write方法首先根据ShuffleDependency创建ExternalSorter对象,然后执行ExternalSotrer.insertAll方法,这个方法的作用是:
1. 将partition的数据存放到ExternalSorter.map中,ExternalSorter.map是一个PartitionedAppendOnlyMap对象2. 如果partition的数据量太大,超过了允许的最大内存,则将ExternalSorter.map中的数据进行排序,然后将这些数据spill到磁盘文件
ExternalSorter.insertAll方法定义如下:
override def insertAll(records: Iterator[Product2[K, V]]): Unit = {
// TODO: stop combining if we find that the reduction factor isn't high
val shouldCombine = aggregator.isDefined
if (shouldCombine) {
// Combine values in-memory first using our AppendOnlyMap
val mergeValue = aggregator.get.mergeValue
val createCombiner = aggregator.get.createCombiner
var kv: Product2[K, V] = null
/*
* update方法的返回值是mergeValue方法的返回值。在执行combineByKey的时候,mergeValue就是它的mergeValue方法
* */
val update = (hadValue: Boolean, oldValue: C) => {
if (hadValue) mergeValue(oldValue, kv._2) else createCombiner(kv._2)
}
while (records.hasNext) {
addElementsRead()
kv = records.next()
/*
*先对key根据partition算法进行运算,计算出key在Stage2的partition index,getPartition返回这个index
* 然后调用update方法,将combine后的值放到map中
* changeValue方法的定义在SizeTrackingAppendOnlyMap类中,它是PartitionedAppendOnlyMap类的父类
* SizeTrackingAppendOnlyMap类的父类是AppendOnlyMap类,它的data成员是一个数组,最终存储了update后的value
* map中存储的数据元素的Key是一个二元tuple,tuple._1是这个Key 在Stage2的partition index,tuple._2是Key;Value是相同Key的值合并后的值
* */
map.changeValue((getPartition(kv._1), kv._1), update)
maybeSpillCollection(usingMap = true)
}
} else {
// Stick values into our buffer
while (records.hasNext) {
addElementsRead()
val kv = records.next()
buffer.insert(getPartition(kv._1), kv._1, kv._2.asInstanceOf[C])
maybeSpillCollection(usingMap = false)
}
}
}
ExternalSorter.insertAll首先调用PartitionedAppendOnlyMap.changeValue方法逐条合并相同Key的Value,把合并后的结果存储在了PartitionedAppendOnlyMap.data数组(data数组的定义在它的父类AppendOnlyMap中)。
然后调用ExternalSorter.maybeSpillCollection检查PartitionedAppendOnlyMap存储数据占有空间是否超过了阈值,如果超过了阈值就调用ShuffleMemoryManager.tryToAcquire方法申请内存,申请内存的大小是当前存储数据的内存,也就是加倍内存。如果申请内存失败,则需要将当前PartitionedAppendOnlyMap中的数据spill到磁盘。这个功能的具体实现在Spillable.maybeSpill方法(Spillable方法是ExternalSorter的父类),它的定义如下:
protected def maybeSpill(collection: C, currentMemory: Long): Boolean = {
if (elementsRead % 32 == 0 && currentMemory >= myMemoryThreshold) {
// Claim up to double our current memory from the shuffle memory pool
val amountToRequest = 2 * currentMemory - myMemoryThreshold
val granted = shuffleMemoryManager.tryToAcquire(amountToRequest)
myMemoryThreshold += granted
if (myMemoryThreshold <= currentMemory) {//加上新申请到的内存还是不能满足当前的内存需求,需要将数据spill到磁盘
// We were granted too little memory to grow further (either tryToAcquire returned 0,
// or we already had more memory than myMemoryThreshold); spill the current collection
_spillCount += 1
logSpillage(currentMemory)
spill(collection)
_elementsRead = 0
// Keep track of spills, and release memory
_memoryBytesSpilled += currentMemory
releaseMemoryForThisThread()
return true
}
}
false
}
ExternalSorter.spill方法负责将PartitionedAppendOnlyMap中的数据spill到磁盘,这个方法往磁盘写入的时候默认情况下每10000条数据往磁盘写入一次,由于shuffle map操作的分区可能很大,这个分区可能多次spill到磁盘,并且把这些磁盘文件信息保存到ExternalSorter.spills这个ArrayBuffer中,ExternalSorter.spill定义如下:
override protected[this] def spill(collection: WritablePartitionedPairCollection[K, C]): Unit = {
// Because these files may be read during shuffle, their compression must be controlled by
// spark.shuffle.compress instead of spark.shuffle.spill.compress, so we need to use
// createTempShuffleBlock here; see SPARK-3426 for more context.
val (blockId, file) = diskBlockManager.createTempShuffleBlock()
// These variables are reset after each flush
var objectsWritten: Long = 0
var spillMetrics: ShuffleWriteMetrics = null
var writer: BlockObjectWriter = null
def openWriter(): Unit = {
assert (writer == null && spillMetrics == null)
spillMetrics = new ShuffleWriteMetrics
writer = blockManager.getDiskWriter(blockId, file, serInstance, fileBufferSize, spillMetrics)
}
openWriter()
// List of batch sizes (bytes) in the order they are written to disk
val batchSizes = new ArrayBuffer[Long]
// How many elements we have in each partition
val elementsPerPartition = new Array[Long](numPartitions)
// Flush the disk writer's contents to disk, and update relevant variables.
// The writer is closed at the end of this process, and cannot be reused.
def flush(): Unit = {
val w = writer
writer = null
w.commitAndClose()
_diskBytesSpilled += spillMetrics.shuffleBytesWritten
batchSizes.append(spillMetrics.shuffleBytesWritten)
spillMetrics = null
objectsWritten = 0
}
var success = false
try {
val it = collection.destructiveSortedWritablePartitionedIterator(comparator)
while (it.hasNext) {
val partitionId = it.nextPartition()//返回这个Element的partition index
/*
* 将这个it Element的key和value写入到上面openWriter内部方法调用时候创建file中,在这里不把Stage2 paritition index写入到file中。
* 在这里不记录partition index是因为把partition index记录到了elementsPerPartition中,而elementPerPartition是创建SpillFile对象的一个参数
* */
it.writeNext(writer)
elementsPerPartition(partitionId) += 1
objectsWritten += 1
if (objectsWritten == serializerBatchSize) {//默认每10000条数据往disk写入一次
flush()
openWriter()
}
}
if (objectsWritten > 0) {
flush()
} else if (writer != null) {
val w = writer
writer = null
w.revertPartialWritesAndClose()
}
success = true
} finally {
if (!success) {
// This code path only happens if an exception was thrown above before we set success;
// close our stuff and let the exception be thrown further
if (writer != null) {
writer.revertPartialWritesAndClose()
}
if (file.exists()) {
file.delete()
}
}
}
/*
* 一个分区在执行ExternalSorter.insertAll执行的时候,可能会spill多次到disk,每次spill都会写入一个file,表示这些file的数据结构都保存在spills中
* */
spills.append(SpilledFile(file, blockId, batchSizes.toArray, elementsPerPartition))
}
SortShuffleWriter.write方法通过调用ExternalSorter.insertAll方法将数据进行分区内的合并之后调用ExternalSorter.writePartitionedFile方法将ExternalSorter.insertAll产生的结果先排序,然后写入到Shuffle map产生的Shuffle文件。需要注意这里写入的Shuffle文件跟spill产生的文件是不同的,这里写入的shuffle文件是为了shuffle的reduce操作使用,而spill产生的文件只是一个中间临时文件,为了写入这个shuffle文件。
在对数据进行排序的时候,发生spill和没有发生spill处理流程是不同的,
1. 没有发生spill,数据存储在了PartitionedAppendOnlyMap,直接调用PartitionedAppendOnlyMap.destructiveSortedWritablePartitionedIterator对PartitionedAppendOnlyMap中的数据根据在Stage2的partition index和数据项的Key进行排序,然后将结果写入shuffle文件
2. 发生了spill,数据存储到了spill后的disk文件,把这些disk文件merge,然后返回排好序的数据,写入到shuffle文件。
ExternalSorter.writePartitionedFile代码如下:
override def writePartitionedFile(
blockId: BlockId,
context: TaskContext,
outputFile: File): Array[Long] = {
// Track location of each range in the output file
val lengths = new Array[Long](numPartitions)
if (spills.isEmpty) {
// Case where we only have in-memory data
val collection = if (aggregator.isDefined) map else buffer
//返回的it已经根据Stage2的partition index和Key进行排序了
val it = collection.destructiveSortedWritablePartitionedIterator(comparator)
while (it.hasNext) {
val writer = blockManager.getDiskWriter(blockId, outputFile, serInstance, fileBufferSize,
context.taskMetrics.shuffleWriteMetrics.get)
val partitionId = it.nextPartition()
//由于it已经根据Stage2 partition index进行排序了,当循环结束的位置是it.nextPartition() != partitionId
while (it.hasNext && it.nextPartition() == partitionId) {
it.writeNext(writer)
}
writer.commitAndClose()
/*
* 返回一个FileSegment对象,这个对象表示Stage2 一个partition的数据写到outputFile中的起始地址和这个partition的数据长度
*
* */
val segment = writer.fileSegment()
lengths(partitionId) = segment.length
}
} else {
// We must perform merge-sort; get an iterator by partition and write everything directly.
/*
* 合并后Stage1一个partition数据太大,数据spill到了disk,对这些spill到disk的文件先合并,然后再写到Shuffle map文件
* */
for ((id, elements) <- this.partitionedIterator) {
if (elements.hasNext) {
val writer = blockManager.getDiskWriter(blockId, outputFile, serInstance, fileBufferSize,
context.taskMetrics.shuffleWriteMetrics.get)
for (elem <- elements) {
writer.write(elem._1, elem._2)
}
writer.commitAndClose()
val segment = writer.fileSegment()
lengths(id) = segment.length
}
}
}
context.taskMetrics.incMemoryBytesSpilled(memoryBytesSpilled)
context.taskMetrics.incDiskBytesSpilled(diskBytesSpilled)
/*
* lengths的数组索引是在Stage2的partition index, 元素value是一个FileSegment.length,
* lengths数组的一个元素表示Stage1一个partition的数据经过Shuffle之后Stage2一个partition的数据长度
*lengths数组表示Stage1每个partition的数据经过Shuffle之后Stage2每个partition的数据长度
* */
lengths
}
在这里要注意ExternalSorter.writePartitionedFile方法的返回值,它的含义是:lengths数组表示Stage1每个partition的数据经过Shuffle之后Stage2每个partition的数据长度。这个返回值会最终返回给Driver,由Driver把它记录到MapOutputTrackerMaster对象,用于启动Stage2。举例来说,假设Stage1处理它的第二个partition,这个partition的数据Shuffle到了Stage2 3个partition中,那么返回结果可以用如下图表示:
SortShuffleWriter.write方法调用ExternalSorter.writePartitionedFile将Shuffle map的结果写入文件之后,再调用IndexShuffleBlockResolver.writeIndexFile将ExternalSorter.writePartitionedFile方法的返回值(也就是Stage1每个partition的数据经过Shuffle之后Stage2每个partition的数据长度)写入到index文件,到此Shuffle map操作部分完成。
ExternalSorter.writePartitionedFile生成的Shuffle map的数据结果写入的文件和IndexShuffleBlockResolver.writeIndexFile生成的文件之间的关系可以用如下图表示:
这两个文件的关系在shuffle reduce操作的时候会用到
总结一下:Spark shuffle map操作将Stage1阶段的一个分区的每条数据,对Key进行运算,计算出一个Key在Stage2的分区id,然后根据分区id将shuffle map结果写到数据存储文件,相同分区id的数据写到了一起。每个ShuffleMapTask任务将Stage1的一个分区写到了一个shuffle文件