目录
sorter.writePartitionedMapOutput
背景
由于spark shuffle 机制和hadoop的MR shuffle非常相似,我先回顾一下MR Shuffle
MR Shuffle
hadoop
打上分区标签,写入到内存 kvBuffer 100M
100M使用达到80%的时候,就会对80%的数据进行排序,排完顺序后spill溢写为一个小文件
hadoopMR的流程
-
Map阶段根据逻辑切片形成对应个数的task
-
task读取split中的数据,给数据添加一个分区的的标记,再写入到kvBuffer
-
kvBuffer慢80%的时候先进行排序(按照分区和key),然后spill溢写到磁盘小文件spill,当task执行完所有溢写后,会进行merge(归并)形成一个大文件
-
Reduce阶段会调一个接口去上游拉取指定的分区数据,假如reduceTask1会拉取 分区1,3的数据
-
拉取数据过程实际上是读取文件的过程,先读取到缓存(内存)中,缓存不够再也会写到磁盘中,当缓冲中的数据慢之后会做一个sort&merge,然后写到小文件中
-
最后merge形成一个大文件,大文件是按照分区和key有序
-
reduce逻辑需要计算的时候,假如统计partition1的数据,和partition3的数据,只需要扫描一次这个大文件,就可以计算出所有的结果
spark shuffle进化
Spark的shuffle演进历史
-
Spark 0.8及以前 Hash Based Shuffle
-
Spark 0.8.1 为Hash Based Shuffle引入File Consolidation机制
-
Spark 0.9 引入ExternalAppendOnlyMap
-
Spark 1.1 引入Sort Based Shuffle,但默认仍为Hash Based Shuffle
-
Spark 1.2 默认的Shuffle方式改为Sort Based Shuffle
-
Spark 1.4 引入Tungsten-Sort Based Shuffle
-
Spark 1.6 Tungsten-sort并入Sort Based Shuffle
-
Spark 2.0 Hash Based Shuffle退出历史舞台
-
本人主要学习spark3.0.1中的 SortShuffleManager
SortShuffleManager
为了更好地解决HashShuffleManager的问题,Spark 参考了 MapReduce 中 Shuffle 的处理方式,引入基于排序的 Shuffle 写操作机制。总体上看来 Sort Shuffle 解决了 Hash Shuffle 的所有弊端,但是因为需要其 Shuffle 过程需要对记录进行排序,所以在性能上有所损失。
-
SortShuffleManager的运行机制主要分成三种,普通运行机制、bypass运行机制、Unsafe运行机制
普通运行机制
-
在普通模式下,每个task当中处理的数据,会先写入一个内存数据结构当中,内存数据结构是Map或者Array,根据不同的shuffle算子,选用不同的数据结构。如果是reduceByKey这类聚合shuffle算子,那么就会选用Map数据结构,如果是join这种普通shuffle算子,那么就会选用Array数据结构。
-
每写入一条数据,判断内存阈值,达到阈值,溢写到磁盘,清空内存结构数据。
-
溢写之前,会根据key对内存数据结构进行排序(为归并排序提供前提条件),排序之后分批次写入磁盘,每批次默认写入10000条,使用java的BufferedOutputStream来实现的,可以减少磁盘IO次数,提升性能。
-
task多次溢写,形成多个小文件,小文件最终进行合并(归并排序),就是merge过程,此时将之前所有的溢写文件全部读取出来,然后依次写入最终的磁盘文件当中形成一个大文件,为了解决大文件分配到各个下游task的数据标识问题,还会写入一份索引文件,索引文件标识了下游每个task当汇总所属数据的start offset以及end offset.
-
SortShuffleManager由于有一个磁盘文件merge的过程,因此大大减少了文件数量。比如一个stage有50个task,总共有10个Executor,每个Executor执行5个task,由于每个task最终只有一个磁盘文件,因此此时每个Executor上只有5个磁盘文件,所有Executor只有50个磁盘文件。
byPass运行机制
-
byPass机制的触发条件:
-
shuffle map task的数量 小于spark.shuffle.sort.bypassMergeThreshold参数的值(默认200)
-
不是聚合类的shuffle算子(例如reduceByKey或者groupBy等)
-
此时task会为每个下游task都创建一个临时磁盘文件,对数据按照key进行hash取值,然后将对应的数据写入到对应的磁盘文件。最终进行磁盘文件的合并,并创建索引文件确定最后大的磁盘文件里面的数据属于哪一个下游的reducetask。
Unsafe机制
-
序列化器是否支持Relocation,如kyro序列化器
-
不是聚合类的shuffle算子(例如reduceByKey或者groupBy等)
-
分区数 > (1 << 24) - 1 + 1 则不行; // 16777215+1
普通的运行机制
嘻嘻嘻,三个机制有点相似,没有的周末我只有一点点时间,所以只看了其中一个机制的原理
SortShuffleWriter流程
SparkTask 的执行入口是 Executor 接收到 Driver 发送过来的 LaunchTask 消息,然后封装了一个TaskRunner 的线任务然后提交给 Executor JVM 进程中的-个线程池执行,最终,运行的是Task.run,这个方法的内部是运行: ShuffleMapTask.runTask0 或者 ResultTask.runTask0。Task 有两种类型: ShufileMaplask和 ResulTask,我们重点关注 ShuffleMaplask 的 ShufleWrite,其实 Resuktask 的 ShufilewWrite 和 ShufleMapTask是一样的,只不过是数据写出逻辑不一样
初始化shuffleBlockResolver
org.apache.spark.shuffle.sort.io.LocalDiskShuffleExecutorComponents#initializeExecutor
blockResolver = new IndexShuffleBlockResolver(sparkConf, blockManager);
注册ShuffleHandle
org.apache.spark.shuffle.sort.SortShuffleManager#registerShuffle
override def registerShuffle[K, V, C](
shuffleId: Int,
dependency: ShuffleDependency[K, V, C]): ShuffleHandle = {
if (SortShuffleWriter.shouldBypassMergeSort(conf, dependency)) {
// If there are fewer than spark.shuffle.sort.bypassMergeThreshold partitions and we don't
// need map-side aggregation, then 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 BypassMergeSortShuffleHandle[K, V](
shuffleId, dependency.asInstanceOf[ShuffleDependency[K, V, V]])
} else if (SortShuffleManager.canUseSerializedShuffle(dependency)) {
// Otherwise, try to buffer map outputs in a serialized form, since this is more efficient:
new SerializedShuffleHandle[K, V](
shuffleId, dependency.asInstanceOf[ShuffleDependency[K, V, V]])
} else {
// Otherwise, buffer map outputs in a deserialized form:
new BaseShuffleHandle(shuffleId, dependency)
}
根据不同的shuffleHandle选用不同的ShuffleWriter
org.apache.spark.shuffle.sort.SortShuffleManager#getWriter
override def getWriter[K, V](
handle: ShuffleHandle,
mapId: Long,
context: TaskContext,
metrics: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = {
val mapTaskIds = taskIdMapsForShuffle.computeIfAbsent(
handle.shuffleId, _ => new OpenHashSet[Long](16))
mapTaskIds.synchronized { mapTaskIds.add(context.taskAttemptId()) }
val env = SparkEnv.get
handle match {
case unsafeShuffleHandle: SerializedShuffleHandle[K @unchecked, V @unchecked] =>
new UnsafeShuffleWriter(
env.blockManager,
context.taskMemoryManager(),
unsafeShuffleHandle,
mapId,
context,
env.conf,
metrics,
shuffleExecutorComponents)
case bypassMergeSortHandle: BypassMergeSortShuffleHandle[K @unchecked, V @unchecked] =>
new BypassMergeSortShuffleWriter(
env.blockManager,
bypassMergeSortHandle,
mapId,
env.conf,
metrics,
shuffleExecutorComponents)
case other: BaseShuffleHandle[K @unchecked, V @unchecked, _] =>
new SortShuffleWriter(
shuffleBlockResolver, other, mapId, context, shuffleExecutorComponents)
}
}
ShuffleMapTask通过getWriter选择调用Write来溢写数据
org.apache.spark.scheduler.ShuffleMapTask#runTask
override def runTask(context: TaskContext): MapStatus = {
// Deserialize the RDD using the broadcast variable.
val threadMXBean = ManagementFactory.getThreadMXBean
val deserializeStartTimeNs = System.nanoTime()
val deserializeStartCpuTime = if (threadMXBean.isCurrentThreadCpuTimeSupported) {
threadMXBean.getCurrentThreadCpuTime
} else 0L
val ser = SparkEnv.get.closureSerializer.newInstance()
val rddAndDep = ser.deserialize[(RDD[_], ShuffleDependency[_, _, _])](
ByteBuffer.wrap(taskBinary.value), Thread.currentThread.getContextClassLoader)
_executorDeserializeTimeNs = System.nanoTime() - deserializeStartTimeNs
_executorDeserializeCpuTime = if (threadMXBean.isCurrentThreadCpuTimeSupported) {
threadMXBean.getCurrentThreadCpuTime - deserializeStartCpuTime
} else 0L
val rdd = rddAndDep._1
val dep = rddAndDep._2
// While we use the old shuffle fetch protocol, we use partitionId as mapId in the
// ShuffleBlockId construction.
val mapId = if (SparkEnv.get.conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)) {
partitionId
} else context.taskAttemptId()
dep.shuffleWriterProcessor.write(rdd, dep, mapId, context, partition)
}
SortShuffleWriter 写解析
我要弄的问题有4个
问题1:什么时候、什么位置、写内存
问题2:什么时候、什么位置、spill磁盘
问题3:spill磁盘前要排序
问题4:小文件是如何做merge的
带着问题看源码
入口
org.apache.spark.shuffle.sort.SortShuffleWriter#write
override def write(records: Iterator[Product2[K, V]]): Unit = {
sorter = if (dep.mapSideCombine) {
new ExternalSorter[K, V, C](
context, dep.aggregator, Some(dep.partitioner), dep.keyOrdering, 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](
context, aggregator = None, Some(dep.partitioner), ordering = None, dep.serializer)
}
sorter.insertAll(records)
writeMetrics.incWriteDurationTime(sorter.dealRecordsTimeNs)
val start = System.nanoTime()
// 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).
val mapOutputWriter = shuffleExecutorComponents.createMapOutputWriter(
dep.shuffleId, mapId, dep.partitioner.numPartitions)
sorter.writePartitionedMapOutput(dep.shuffleId, mapId, mapOutputWriter)
val partitionLengths = mapOutputWriter.commitAllPartitions()
mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths, mapId)
write
sorter.insertAll(records)
哈哈哈,为了不迷路,这个流程图我会重复的看
org.apache.spark.util.collection.ExternalSorter#insertAll
顾名思义把所有内容写到磁盘中去
但是一次写不完啊,内存有限制,所有一次写一个小文件
所以insertAll执行完后:存在很多小文件和一个正在写的内存
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
val update = (hadValue: Boolean, oldValue: C) => {
if (hadValue) mergeValue(oldValue, kv._2) else createCombiner(kv._2)
}
while (records.hasNext) {
addElementsRead()
kv = records.next()
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)
}
}
}
org.apache.spark.util.collection.ExternalSorter#maybeSpillCollection
检查是否要溢写,要则溢写,溢写的同时,新建一块缓存
private def maybeSpillCollection(usingMap: Boolean): Unit = {
var estimatedSize = 0L
if (usingMap) {
estimatedSize = map.estimateSize()
if (maybeSpill(map, estimatedSize)) {
map = new PartitionedAppendOnlyMap[K, C]
}
} else {
estimatedSize = buffer.estimateSize()
if (maybeSpill(buffer, estimatedSize)) {
buffer = new PartitionedPairBuffer[K, C]
}
}
if (estimatedSize > _peakMemoryUsedBytes) {
_peakMemoryUsedBytes = estimatedSize
}
}
如何溢写的吗
org.apache.spark.util.collection.Spillable#maybeSpill
我用脑子想了一下,这个方式应该要实现两个功能
第一是排序,按照传入的排序器排序(也就是分区和key)
第二是flush刷写数据到磁盘,一次性刷写10000条数据
看看源码是不是这样子的
protected def maybeSpill(collection: C, currentMemory: Long): Boolean = {
var shouldSpill = false
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 = acquireMemory(amountToRequest)
myMemoryThreshold += granted
// If 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
shouldSpill = currentMemory >= myMemoryThreshold
}
shouldSpill = shouldSpill || _elementsRead > numElementsForceSpillThreshold
// Actually spill
if (shouldSpill) {
_spillCount += 1
logSpillage(currentMemory)
spill(collection)
_elementsRead = 0
_memoryBytesSpilled += currentMemory
releaseMemory()
}
shouldSpill
}
两种情况要检查是否要刷写
-
读取的元素的个数是32的倍数&&map中使用的内存是大于myMemoryThreshold=5MB
-
不过这个会慢慢增大myMemoryThreshold += granted,acquireMemory(amountToRequest)兄弟们,这里面获取内存的逻辑非常复杂,就是有一种情况是获取不到足够的内存的时候,就会shouldSpill=true
-
-
另一种情况是_elementsRead > numElementsForceSpillThreshold() 超过整数最大值Integer.MAX_VALUE,这种情况应该比较少有吧
再看看spill逻辑
org.apache.spark.util.collection.ExternalSorter#spill
/**
* Spill our in-memory collection to a sorted file that we can merge later.
* We add this file into `spilledFiles` to find it later.
*
* @param collection whichever collection we're using (map or buffer)
*/
override protected[this] def spill(collection: WritablePartitionedPairCollection[K, C]): Unit = {
val inMemoryIterator = collection.destructiveSortedWritablePartitionedIterator(comparator)
val spillFile = spillMemoryIteratorToDisk(inMemoryIterator)
spills += spillFile
}
nice 就是这里,先排序,再写到磁盘
溢写val inMemoryIterator = collection.destructiveSortedWritablePartitionedIterator(comparator)
/**
* Iterate through the data and write out the elements instead of returning them. Records are
* returned in order of their partition ID and then the given comparator.
* This may destroy the underlying collection.
*/
def destructiveSortedWritablePartitionedIterator(keyComparator: Option[Comparator[K]])
: WritablePartitionedIterator = {
val it = partitionedDestructiveSortedIterator(keyComparator)
new WritablePartitionedIterator {
private[this] var cur = if (it.hasNext) it.next() else null
def writeNext(writer: PairsWriter): Unit = {
writer.write(cur._1._2, cur._2)
cur = if (it.hasNext) it.next() else null
}
def hasNext(): Boolean = cur != null
def nextPartition(): Int = cur._1._1
}
}
val it = partitionedDestructiveSortedIterator(keyComparator)
org.apache.spark.util.collection.PartitionedAppendOnlyMap#partitionedDestructiveSortedIterator
对,就是这个方法,返回有序的数据
def partitionedDestructiveSortedIterator(keyComparator: Option[Comparator[K]])
: Iterator[((Int, K), V)] = {
val comparator = keyComparator.map(partitionKeyComparator).getOrElse(partitionComparator)
destructiveSortedIterator(comparator)
}
/**
* Return an iterator of the map in sorted order. This provides a way to sort the map without
* using additional memory, at the expense of destroying the validity of the map.
*/
def destructiveSortedIterator(keyComparator: Comparator[K]): Iterator[(K, V)] = {
destroyed = true
// Pack KV pairs into the front of the underlying array
var keyIndex, newIndex = 0
while (keyIndex < capacity) {
if (data(2 * keyIndex) != null) {
data(2 * newIndex) = data(2 * keyIndex)
data(2 * newIndex + 1) = data(2 * keyIndex + 1)
newIndex += 1
}
keyIndex += 1
}
assert(curSize == newIndex + (if (haveNullValue) 1 else 0))
new Sorter(new KVArraySortDataFormat[K, AnyRef]).sort(data, 0, newIndex, keyComparator)
new Iterator[(K, V)] {
var i = 0
var nullValueReady = haveNullValue
def hasNext: Boolean = (i < newIndex || nullValueReady)
def next(): (K, V) = {
if (nullValueReady) {
nullValueReady = false
(null.asInstanceOf[K], nullValue)
} else {
val item = (data(2 * i).asInstanceOf[K], data(2 * i + 1).asInstanceOf[V])
i += 1
item
}
}
}
}
瞄一下返回的结果是一个迭代器WritablePartitionedIterator,nice,下面是重点啊,按照分区的id和比较器进行排序
-
Records are returned in order of their partition ID and then the given comparator.
val spillFile = spillMemoryIteratorToDisk(inMemoryIterator)
这里把有序的数据 spill到磁盘
/**
* Spill contents of in-memory iterator to a temporary file on disk.
*/
private[this] def spillMemoryIteratorToDisk(inMemoryIterator: WritablePartitionedIterator)
: SpilledFile = {
// 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
val spillMetrics: ShuffleWriteMetrics = new ShuffleWriteMetrics
val writer: DiskBlockObjectWriter =
blockManager.getDiskWriter(blockId, file, serInstance, fileBufferSize, spillMetrics)
// 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 committed at the end of this process.
def flush(): Unit = {
val segment = writer.commitAndGet()
batchSizes += segment.length
_diskBytesSpilled += segment.length
objectsWritten = 0
}
var success = false
try {
while (inMemoryIterator.hasNext) {
val partitionId = inMemoryIterator.nextPartition()
require(partitionId >= 0 && partitionId < numPartitions,
s"partition Id: ${partitionId} should be in the range [0, ${numPartitions})")
inMemoryIterator.writeNext(writer)
elementsPerPartition(partitionId) += 1
objectsWritten += 1
if (objectsWritten == serializerBatchSize) {
flush()
}
}
if (objectsWritten > 0) {
flush()
writer.close()
} else {
writer.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
writer.revertPartialWritesAndClose()
if (file.exists()) {
if (!file.delete()) {
logWarning(s"Error deleting ${file}")
}
}
}
}
SpilledFile(file, blockId, batchSizes.toArray, elementsPerPartition)
}
到这里,insertAll就结束了,是否有几个小文件和一个正在写的内存呢? 是的
写完之后得开始小文件合并了吧
sorter.writePartitionedMapOutput
def writePartitionedMapOutput(
shuffleId: Int,
mapId: Long,
mapOutputWriter: ShuffleMapOutputWriter): Unit = {
var nextPartitionId = 0
//合并内存中的数据
if (spills.isEmpty) {
// Case where we only have in-memory data
val collection = if (aggregator.isDefined) map else buffer
//常规操作,先对内存中的文件进行一个排序
val it = collection.destructiveSortedWritablePartitionedIterator(comparator)
while (it.hasNext()) {
val partitionId = it.nextPartition()
var partitionWriter: ShufflePartitionWriter = null
var partitionPairsWriter: ShufflePartitionPairsWriter = null
TryUtils.tryWithSafeFinally {
partitionWriter = mapOutputWriter.getPartitionWriter(partitionId)
val blockId = ShuffleBlockId(shuffleId, mapId, partitionId)
partitionPairsWriter = new ShufflePartitionPairsWriter(
partitionWriter,
serializerManager,
serInstance,
blockId,
context.taskMetrics().shuffleWriteMetrics)
while (it.hasNext && it.nextPartition() == partitionId) {
it.writeNext(partitionPairsWriter)
}
} {
if (partitionPairsWriter != null) {
partitionPairsWriter.close()
}
}
nextPartitionId = partitionId + 1
}
} else {
//合并小文件,这里才会用到归并排序,这么说是先基于一个文件进行归并排序吗
// We must perform merge-sort; get an iterator by partition and write everything directly.
for ((id, elements) <- this.partitionedIterator) {
val blockId = ShuffleBlockId(shuffleId, mapId, id)
var partitionWriter: ShufflePartitionWriter = null
var partitionPairsWriter: ShufflePartitionPairsWriter = null
TryUtils.tryWithSafeFinally {
partitionWriter = mapOutputWriter.getPartitionWriter(id)
partitionPairsWriter = new ShufflePartitionPairsWriter(
partitionWriter,
serializerManager,
serInstance,
blockId,
context.taskMetrics().shuffleWriteMetrics)
if (elements.hasNext) {
for (elem <- elements) {
partitionPairsWriter.write(elem._1, elem._2)
}
}
} {
if (partitionPairsWriter != null) {
partitionPairsWriter.close()
}
}
nextPartitionId = id + 1
}
}
划重点
(id, elements) <- this.partitionedIterator
/**
* Return an iterator over all the data written to this object, grouped by partition and
* aggregated by the requested aggregator. For each partition we then have an iterator over its
* contents, and these are expected to be accessed in order (you can't "skip ahead" to one
* partition without reading the previous one). Guaranteed to return a key-value pair for each
* partition, in order of partition ID.
*
* For now, we just merge all the spilled files in once pass, but this can be modified to
* support hierarchical merging.
* Exposed for testing.
*/
def partitionedIterator: Iterator[(Int, Iterator[Product2[K, C]])] = {
val usingMap = aggregator.isDefined
val collection: WritablePartitionedPairCollection[K, C] = if (usingMap) map else buffer
if (spills.isEmpty) {
// Special case: if we have only in-memory data, we don't need to merge streams, and perhaps
// we don't even need to sort by anything other than partition ID
if (ordering.isEmpty) {
// The user hasn't requested sorted keys, so only sort by partition ID, not key
groupByPartition(destructiveIterator(collection.partitionedDestructiveSortedIterator(None)))
} else {
// We do need to sort by both partition ID and key
groupByPartition(destructiveIterator(
collection.partitionedDestructiveSortedIterator(Some(keyComparator))))
}
} else {
// Merge spilled and in-memory data
merge(spills.toSeq, destructiveIterator(
collection.partitionedDestructiveSortedIterator(comparator)))
}
}
merge
把一个file和另外一个inMemory归并
/**
* Merge a sequence of sorted files, giving an iterator over partitions and then over elements
* inside each partition. This can be used to either write out a new file or return data to
* the user.
*
* Returns an iterator over all the data written to this object, grouped by partition. For each
* partition we then have an iterator over its contents, and these are expected to be accessed
* in order (you can't "skip ahead" to one partition without reading the previous one).
* Guaranteed to return a key-value pair for each partition, in order of partition ID.
*/
private def merge(spills: Seq[SpilledFile], inMemory: Iterator[((Int, K), C)])
: Iterator[(Int, Iterator[Product2[K, C]])] = {
val readers = spills.map(new SpillReader(_))
val inMemBuffered = inMemory.buffered
(0 until numPartitions).iterator.map { p =>
val inMemIterator = new IteratorForPartition(p, inMemBuffered)
val iterators = readers.map(_.readNextPartition()) ++ Seq(inMemIterator)
if (aggregator.isDefined) {
// Perform partial aggregation across partitions
(p, mergeWithAggregation(
iterators, aggregator.get.mergeCombiners, keyComparator, ordering.isDefined))
} else if (ordering.isDefined) {
// No aggregator given, but we have an ordering (e.g. used by reduce tasks in sortByKey);
// sort the elements without trying to merge them
(p, mergeSort(iterators, ordering.get))
} else {
(p, iterators.iterator.flatten)
}
}
}
到这里位置就是溢写完毕了
溢写完毕之后不得加一个索引吗,来了,就是以下的方法
mapOutputWriter.commitAllPartitions
给数据添加index
org.apache.spark.shuffle.IndexShuffleBlockResolver#writeIndexFileAndCommit
这里不展开了
好了,兄弟们,找到四个问题的答案了吗
哈哈第二次写博客,非常菜鸡,凑合着看