暂无图片
暂无图片
暂无图片
暂无图片
暂无图片

Spark Shuffle原理及源码解析

民生运维人 2021-11-18
847

Shuffle的产生

在Spark作业中当父RDD与子RDD的分区对应关系为多对多或者一对多的情况下会发生宽依赖,也即一个父RDD的分区需要分发到多个子RDD所在的任务中去执行,这种情况就会涉及数据的重新分布,也即产生了shuffle。

Spark算子是否引入shuffle与各算子的具体实现有关,本质上是要看父子RDD的分区器的实现是否相同,例如:在执行聚合类算子reduceByKey时判断是否会引入shuffle,需要分析父子rdd的分区器partitioner是否一致,如果不一致则创建一个ShuffleRDD作为子RDD从而产生shuffle:

def combineByKeyWithClassTag[C](
      createCombiner: V => C,
      mergeValue: (CV) => C,
      mergeCombiners: (CC) => C,
      partitioner: Partitioner,
      mapSideCombine: Boolean = true,
      serializer: Serializer = null)(implicit ct: ClassTag[C]): RDD[(KC)] = self.withScope {
    ...
    if (self.partitioner == Some(partitioner)) {
      self.mapPartitions(iter => {
        val context = TaskContext.get()
        new InterruptibleIterator(context, aggregator.combineValuesByKey(iter, context))
      }, preservesPartitioning = true)
    } else {
      new ShuffledRDD[KVC](self, partitioner)
        .setSerializer(serializer)
        .setAggregator(aggregator)
        .setMapSideCombine(mapSideCombine)
    }
  }

类似的再以join算子的实现为例,在CoGroupedRDD的getDependencies方法中遍历所有的父rdd,如果父rdd和子rdd的分区器一致则创建OneToOneDependency,否则创建ShuffleDependency并引入shuffle:

override def getDependenciesSeq[Dependency[_]] = {
  rdds.map { rdd: RDD[_] =>
    if (rdd.partitioner == Some(part)) {
      logDebug("Adding one-to-one dependency with " + rdd)
      new OneToOneDependency(rdd)
    } else {
      logDebug("Adding shuffle dependency with " + rdd)
      new ShuffleDependency[KAnyCoGroupCombiner](
        rdd.asInstanceOf[RDD[_ <: Product2[K, _]]], part, serializer)
    }
  }
}

Shuffle文件的生成

我们首先来看shuffle的第一个阶段:Shuffle Write阶段。通过Spark任务调度原理的讲解,我们知道在创建taskset的过程中,如果当前的stage是ShuffleMapStage,则创建的任务类型为ShuffleMapTask,否则task的类型为ResultTask,两种类型的Task均实现了runTask方法;如果发生了shuffle则执行ShuffleMapTask实现的runTask方法,即根据rdd、dependency、mapId等信息调用ShuffleWriteProcessor的write方法执行shuffle数据的写入:

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)

选择ShuffleWriter

在ShuffleWriteProcessor的write方法中首先通过ShuffleManager获取writer实例,然后再由相应的writer执行具体的write逻辑:

def write(
    rdd: RDD[_],
    dep: ShuffleDependency[_, _, _],
    mapId: Long,
    context: TaskContext,
    partition: Partition): MapStatus = {
  var writer: ShuffleWriter[AnyAny] = null
  try {
    val manager = SparkEnv.get.shuffleManager
    writer = manager.getWriter[AnyAny](
      dep.shuffleHandle,
      mapId,
      context,
      createMetricsReporter(context))
    writer.write(
      rdd.iterator(partition, context).asInstanceOf[Iterator[_ <: Product2[AnyAny]]])

Spark根据ShuffleHandle的不同采用相应的ShuffleWriter的实现,包括:UnsafeShuffleWriter、BypassMergeSortShuffleWriter和SortShuffleWriter三种:

override def getWriter[KV](
    handle: ShuffleHandle,
    mapId: Long,
    context: TaskContext,
    metrics: ShuffleWriteMetricsReporter): ShuffleWriter[KV] = {
  val mapTaskIds = taskIdMapsForShuffle.computeIfAbsent(
    handle.shuffleId, _ => new OpenHashSet[Long](16))
  mapTaskIds.synchronized { mapTaskIds.add(mapId) }
  val env = SparkEnv.get
  handle match {
    case unsafeShuffleHandle: SerializedShuffleHandle[K @uncheckedV @unchecked] =>
      new UnsafeShuffleWriter(
          ...
    case bypassMergeSortHandle: BypassMergeSortShuffleHandle[K @uncheckedV @unchecked] =>
      new BypassMergeSortShuffleWriter(
          ...
    case other: BaseShuffleHandle[K @uncheckedV @unchecked, _] =>
      new SortShuffleWriter(other, mapId, context, shuffleExecutorComponents)
  }

而具体的ShuffleHandle的选择是根据shuffle算子实际的partition数、是否需要执行排序或者聚合等情况来确定的:

override def registerShuffle[KVC](
    shuffleId: Int,
    dependency: ShuffleDependency[KVC]): ShuffleHandle = {
  if (SortShuffleWriter.shouldBypassMergeSort(conf, dependency)) {
    new BypassMergeSortShuffleHandle[KV](
      shuffleId, dependency.asInstanceOf[ShuffleDependency[KVV]])
  } else if (SortShuffleManager.canUseSerializedShuffle(dependency)) {
    new SerializedShuffleHandle[KV](
      shuffleId, dependency.asInstanceOf[ShuffleDependency[KVV]])
  } else {
    new BaseShuffleHandle(shuffleId, dependency)
  }
}

下面分别介绍这三种ShuffleHandle的选择逻辑:

1、BypassMergeSortShuffleHandle

BypassMergeSortShuffleHandle对应BypassMergeSortShuffleWriter,当不需要做map端的聚合,并且分区数小于SHUFFLE_SORT_BYPASS_MERGE_THRESHOLD(默认200)时采用这种方式,可以跳过在内存中排序和聚合的过程:

if (dep.mapSideCombine) {
  false
else {
  val bypassMergeThreshold: Int = conf.get(config.SHUFFLE_SORT_BYPASS_MERGE_THRESHOLD)
  dep.partitioner.numPartitions <= bypassMergeThreshold
}

BypassMergeSortShuffleWriter不需要将shuffle记录先写入内存缓存结构中,而是根据数据的key值得到reduce分区,并创建对应的DiskBlockObjectWriter对象将数据记录直接写入到各分区对应的临时文件中;最后再将不同分区的临时文件合并生产data和index文件即可。

2、SerializedShuffleHandle,该方式使用了tungsten基于内存压缩的机制,缓解shuffle过程中的内存压力从而实现shuffle加速。

采用该方式需要满足三个条件:

  if (!dependency.serializer.supportsRelocationOfSerializedObjects) {
    log.debug(s"Can't use serialized shuffle for shuffle $shufId because the serializer, " +
      s"${dependency.serializer.getClass.getName}, does not support object relocation")
    false
  } else if (dependency.mapSideCombine) {
    log.debug(s"Can't use serialized shuffle for shuffle $shufId because we need to do " +
      s"map-side aggregation")
    false
  } else if (numPartitions > MAX_SHUFFLE_OUTPUT_PARTITIONS_FOR_SERIALIZED_MODE) {
    log.debug(s"Can't use serialized shuffle for shuffle $shufId because it has more than " +
      s"$MAX_SHUFFLE_OUTPUT_PARTITIONS_FOR_SERIALIZED_MODE partitions")
    false
  } else {
    log.debug(s"Can use serialized shuffle for shuffle $shufId")
    true
  }


1)dependency的序列化器支持relocation

如果用户程序中采用DataFrame、DataSet数据模型则等底层使用SparkSQL内核,当出现shuffle的情况下,优化器在制定物理计划会构建ShuffleExchangeExec节点,并采用UnsafeRowSerializer,该序列化器的supportsRelocationOfSerializedObjects属性为true,即支持对序列化对象进行排序;另外,如果用户指定使用KryoSerializer序列化器或者记录的key和value为原生数据类型或者string类型也采用KryoSerializer序列化器,此时upportsRelocationOfSerializedObjects属性为true;否则使用默认的JavaSerializer,该属性的值为false。

2)不需要执行map端合并:

如果采用非聚合类算子例如join相关算子时对应dependency的mapSideCombine属性值为false;如果采用聚合类算子如reduceByKey、aggregateByKey、combineByKey等mapSideCombine属性为true;注意执行groupByKey算子时该属性也为false:

def groupByKey(partitioner: Partitioner): RDD[(KIterable[V])] = self.withScope {
  ...
  val bufs = combineByKeyWithClassTag[CompactBuffer[V]](
    createCombiner, mergeValue, mergeCombiners, partitioner, mapSideCombine = false)

3)shuffledependency的partition数小于MAX_SHUFFLE_OUTPUT_PARTITIONS_FOR_SERIALIZED_MODE,即16777215。

3、如果无法使用前两种ShuffleHandle,则采用BaseShuffleHandle,对应ShuffleWriter为SortShuffleWriter。

综合以上分析,Spark根据ShuffleHandle的不同而选择相应的ShuffleWriter的实现,接下来我们来详细阐述这三种ShuffleWriter中最为典型的实现方式SortShuffleWriter的执行原理,在后续的Spark内存管理的文章中我们将对UnsafeShuffleWriter以内存的角度进行阐述;而BypassMergeSortShuffleWriter则是SortShuffleWriter的特殊情况,即跳过了map排序和聚合部分。

SortShuffleWriter

SortShuffleWriter通过insertAll方法首先将参与shuffle的数据写入到shuffle缓存列表中,当缓存列表的空间增大到无法继续写入时则将数据溢写到磁盘中。

Shuffle缓存列表的实现有两种数据结构:如果参与shuffle的算子需要做聚合则将数据记录写入到数据结构PartitionedAppendOnlyMap中,该结构是一个HashMap,key为partitionId和记录的key值,并且每处理一个记录均会更新对应的key的value值;如果算子不需要做聚合则采用PartitionedPairBuffer的数据结构,并将记录的key和value顺序插入到buffer数组中:

if (shouldCombine) {
  // Combine values in-memory first using our AppendOnlyMap
  val mergeValue = aggregator.get.mergeValue
  val createCombiner = aggregator.get.createCombiner
  var kv: Product2[KV] = 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) //更新hashmap中的value值
    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]) //将记录顺序插入到buffer中
    maybeSpillCollection(usingMap = false)
  }
}

由此可见,采用PartitionedAppendOnlyMap这种数据结构可以节约内存空间、减少磁盘溢写以及shuffle拉取的网络开销,这也是reduceByKey比groupByKey性能更好的原因;同时,也是为什么shuffle在没有聚合的情况下需要采用tungsten的存储方式来进一步提升执行性能。

每次写入记录之后都会判断是否需要将内存中的数据进行溢写,主要的判断逻辑是当shuffle缓存的数据量达到当前的阈值之后尝试扩容shuffle缓存列表,当扩容之后的空间仍然不足的情况下则开始执行溢写逻辑:

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()
}

排序:

如果在shuffle依赖中指定了排序的顺序或者聚合算法则定义排序函数keyComparator:

private def comparatorOption[Comparator[K]] = {
  if (ordering.isDefined || aggregator.isDefined) {
    Some(keyComparator)
  } else {
    None
  }
}

在具有排序函数的情况下,PartitionedAppendOnlyMap和PartitionedPairBuffer分别实现了partitionedDestructiveSortedIterator函数,对数据记录首先根据分区排序,然后再根据key进行排序:

/**
 * A comparator for (Int, K) pairs that orders them both by their partition ID and a key ordering.
 */

def partitionKeyComparator[K](keyComparator: Comparator[K]): Comparator[(IntK)] =
  (a: (IntK), b: (IntK)) => {
    val partitionDiff = a._1 - b._1
    if (partitionDiff != 0) {
      partitionDiff
    } else {
      keyComparator.compare(a._2, b._2)
    }
  }

磁盘刷写:

通过前面的过程将需要溢写的数据在内存中排序并封装到一个迭代器对象inMemoryIterator中,然后再通过ExternalSorter调用spillMemoryIteratorToDisk方法将排序后的数据写到IO输出缓冲区,当达到输出缓冲区的容量上限(配置项:spark.shuffle.file.buffer,默认32K) 或者记录的个数超过SHUFFLE_SPILL_BATCH_SIZE的值(配置项:spark.shuffle.spill.batchSize,默认10000),则将数据flush到磁盘。因此如果一个作业shuffle溢写的数据量较大,可以适当调大相关配置参数从而减轻磁盘IO的性能开销:

val (blockId, file) = diskBlockManager.createTempShuffleBlock()   //创建临时溢写文件,并为其生成blockId
val writer: DiskBlockObjectWriter =
  blockManager.getDiskWriter(blockId, file, serInstance, fileBufferSize, spillMetrics) //构建fileBufferSize大小的缓冲区writer
...
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到磁盘中
          flush()
        }
 ...
  

将内存中的记录先进行排序并刷写到临时磁盘文件之后,再将该文件追加到spills列表中,spills列表是一个ArrayBuffer[SpilledFile]的数据结构,表示一个task所有的待合并文件的集合:

val spillFile = spillMemoryIteratorToDisk(inMemoryIterator)
spills += spillFile

合并溢写文件:

当所有需要shuffle的记录均处理完成并溢写之后,ExternalSorter针对每个map分区调用writePartitionedMapOutput方法将溢写到磁盘的临时文件和以及内存中数据进行归并排序,并写入到一个data文件中,具体实现过程如下:

1.根据shuffleId、mapId创建该分区溢写文件合并后的data文件,文件名为:

 name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".data"

2.首先针对每个reduce分区,遍历所有的临时溢写文件和内存中的记录,将属于该分区的记录根据key值进行聚合运算;如果需要排序,则先对记录进行归并排序再根据key值做聚合;最后生成一个(partitionId,partitionId对应的记录列表) 的二元组迭代器。

private def merge(spills: Seq[SpilledFile], inMemory: Iterator[((IntK), C)])
      : Iterator[(IntIterator[Product2[KC]])] = {
    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)
      }
    }
  }


3.遍历第2步中生成的二元组迭代器,依次为每个reduce partitionId创建一个ShufflePartitionPairsWriter对象,并将partitionId对应的所有记录的key和value值写入到在步骤1中创建的data文件中:

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)
      }
    }
  } 

需要说明的是,在将shuffle数据进行合并的过程还会累计各个patition对应数据所占用的存储空间的大小,这些信息采用partitionLengths数组进行记录,partitionLengths数组是一个下标为partitionId、值为对应分区的数据长度的长整型数组。

构建索引文件:

由于在创建的data文件的过程中还构建了partitionLengths数组,就可以方便的知道各分区的数据在data文件中的偏移量,以便于在reduce阶段快速检索data文件中的数据,避免了大量shuffle文件的全量扫描,从而提高shuffle读阶段的处理性能。接下来介绍为每个data文件构建索引文件的过程:

1.在IndexShuffleBlockResolver类的writeIndexFileAndCommit方法中,根据shuffleId、mapId即"shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".index" 作为索引文件的名称,并且该文件名也对应存储系统中的BlockId,然后通过对应executor的DiskBlockManager对象在localDir(一般是spark.local.dir配置项)目录中创建一个index文件;

def getIndexFile(
    shuffleId: Int,
    mapId: Long,
    dirs: Option[Array[String]] = None): File = {
  val blockId = ShuffleIndexBlockId(shuffleId, mapId, NOOP_REDUCE_ID)
  dirs
    .map(ExecutorDiskUtils.getFile(_, blockManager.subDirsPerLocalDir, blockId.name))
    .getOrElse(blockManager.diskBlockManager.getFile(blockId))
}

2.根据partitionLengths数组中的length值进行逐一累加计算,获得每个reduce task的数据在data文件中的起始偏移量offset,并将其记录在index文件中,用于后续快速检索对应分区的shuffle数据:

var offset = 0L
out.writeLong(offset)
for (length <- lengths) {
  offset += length
  out.writeLong(offset)
}

Reduce 阶段的数据处理

在shuffle的Map阶段也即shuffle write阶段完成了数据的溢写和合并,接下来进入shuffle的Reduce阶段也即shuffle read 阶段。

我们知道,所有RDD都会执行其compute方法,在ShuffleRDD的compute方法中会初始化一个reader对象并调用其read()方法并在实现了如上逻辑:

override def compute(split: Partition, context: TaskContext): Iterator[(KC)] = {
  val dep = dependencies.head.asInstanceOf[ShuffleDependency[KVC]]
  val metrics = context.taskMetrics().createTempShuffleReadMetrics()
  SparkEnv.get.shuffleManager.getReader(
    dep.shuffleHandle, split.index, split.index + 1, context, metrics)
    .read()
    .asInstanceOf[Iterator[(KC)]]
}

初始化BlockStoreShuffleReader对象

在reduce阶段,SortShuffleManager首先通过MapOutputTracker根据shuffleId从mapStatuses中获取blocksByAddress对象,该对象的数据结构为:Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])],表示一个reduce分区所需拉取的shuffle文件的存储信息(包括BlockManagerId以及BlockId、字节数、mapId的集合);接下来创建BlockStoreShuffleReader对象用于读取blocksByAddress中所指定的shuffle文件:

override def getReader[KC](
    handle: ShuffleHandle,
    startMapIndex: Int,
    endMapIndex: Int,
    startPartition: Int,
    endPartition: Int,
    context: TaskContext,
    metrics: ShuffleReadMetricsReporter): ShuffleReader[KC] = {
  val blocksByAddress = SparkEnv.get.mapOutputTracker.getMapSizesByExecutorId(
    handle.shuffleId, startMapIndex, endMapIndex, startPartition, endPartition)
  new BlockStoreShuffleReader(
    handle.asInstanceOf[BaseShuffleHandle[K, _, C]], blocksByAddress, context, metrics,
    shouldBatchFetch = canUseBatchFetch(startPartition, endPartition, context))
}

其中,BlockId为ShuffleBlockId的实例,其编码方式为"shuffle_" + shuffleId + "_" + mapId + "_" + reduceId,block所占字节数通过MapStatus获取:

for (part <- startPartition until endPartition) {
  val size = status.getSizeForBlock(part)
  if (size != 0) {
    splitsByAddress.getOrElseUpdate(status.location, ListBuffer()) +=
      ((ShuffleBlockId(shuffleId, status.mapId, part), size, mapIndex))
  }
}

执行read()方法

BlockStoreShuffleReader通过执行其read()方法从本地获取或者通过网络从其他节点拉取shuffle数据,并对这些数据进行进一步处理,接下来我们来看一下read()方法的具体实现:

1、获取shuffle map数据

在read()方法中首先构建了ShuffleBlockFetcherIterator实例,并通过ShuffleBlockFetcherIterator的initialize()方法来实现shuffle记录的读取:

1)调用partitionBlocksByFetchMode方法根据shuffle文件的位置信息划分为不同的队列:

根据blocksByAddress中携带的shuffle文件的地址信息,如果blockManager对应的executor与当前reduce 任务的executor一致,则将该的blockManager对应的shuffle文件存储信息放入localBlocks列表中;否则,如果blockManager所在的节点与当前reduce 任务的节点一致,则将该blockManager对应的shuffle文件存储信息放到hostLocalBlocks列表中;否则shuffle文件信息存在于远程节点中,将对应的shuffle文件存储信息放到fetchRequests队列中:

for ((address, blockInfos) <- blocksByAddress) {
  //blockManager对应的executor与当前reduce 任务的executor一致
  if (Seq(blockManager.blockManagerId.executorId, fallback).contains(address.executorId)) {
    checkBlockSizes(blockInfos)
    val mergedBlockInfos = mergeContinuousShuffleBlockIdsIfNeeded(
      blockInfos.map(info => FetchBlockInfo(info._1, info._2, info._3)), doBatchFetch)
    numBlocksToFetch += mergedBlockInfos.size
    localBlocks ++= mergedBlockInfos.map(info => (info.blockId, info.mapIndex))
    localBlockBytes += mergedBlockInfos.map(_.size).sum
  } else if (blockManager.hostLocalDirManager.isDefined &&
    address.host == blockManager.blockManagerId.host) { //blockManager所在的节点与当前reduce 任务的节点一致
    checkBlockSizes(blockInfos)
    val mergedBlockInfos = mergeContinuousShuffleBlockIdsIfNeeded(
      blockInfos.map(info => FetchBlockInfo(info._1, info._2, info._3)), doBatchFetch)
    numBlocksToFetch += mergedBlockInfos.size
    val blocksForAddress =
      mergedBlockInfos.map(info => (info.blockId, info.size, info.mapIndex))
    hostLocalBlocksByExecutor += address -> blocksForAddress
    hostLocalBlocks ++= blocksForAddress.map(info => (info._1, info._3))
    hostLocalBlockBytes += mergedBlockInfos.map(_.size).sum
  } else {  //否则shuffle文件信息存在于远程节点中
    remoteBlockBytes += blockInfos.map(_._2).sum
    collectFetchRequests(address, blockInfos, collectedRemoteRequests)
  }

值得注意的是,从远端拉取数据的情况下如果数据量太大容易导致网络阻塞,因此spark中通过targetRemoteRequestSize 来限制reduce task每次远程拉取的数据量,如果超过该阈值则将当前的block封装为一个FetchRequest并放置到collectedRemoteRequests列表中作为后续数据拉取的一个基本单元:

if (curRequestSize >= targetRemoteRequestSize || mayExceedsMaxBlocks) {
        curBlocks = createFetchRequests(curBlocks, address, isLast = false,
          collectedRemoteRequests)
...
    
private def createFetchRequests(
      curBlocks: Seq[FetchBlockInfo],
      address: BlockManagerId,
      isLast: Boolean,
      collectedRemoteRequests: ArrayBuffer[FetchRequest]): Seq[FetchBlockInfo] = {
    val mergedBlocks = mergeContinuousShuffleBlockIdsIfNeeded(curBlocks, doBatchFetch)
    numBlocksToFetch += mergedBlocks.size
    var retBlocks = Seq.empty[FetchBlockInfo]
    if (mergedBlocks.length <= maxBlocksInFlightPerAddress) {
      collectedRemoteRequests += createFetchRequest(mergedBlocks, address)
    } else {
      mergedBlocks.grouped(maxBlocksInFlightPerAddress).foreach { blocks =>
        if (blocks.length == maxBlocksInFlightPerAddress || isLast) {
          collectedRemoteRequests += createFetchRequest(blocks, address)
    ...

其中,targetRemoteRequestSize 的值为 math.max(maxBytesInFlight / 5, 1L),maxBytesInFlight 通过配置项SparkEnv.get.conf.get(config.REDUCER_MAX_SIZE_IN_FLIGHT) * 1024 * 1024(默认48M)来指定;

如果网络带宽不是瓶颈并且需要拉取的shuffle数据量较大,则可以适当调大REDUCER_MAX_SIZE_IN_FLIGHT即配置项spark.reducer.maxSizeInFlight的值,反之亦然。

2)前面的步骤根据shuffle文件存储的位置不同得到了三个请求列表,接下来分别获取各个列表中的数据:

 // Send out initial requests for blocks, up to our maxBytesInFlight
  fetchUpToMaxBytes()   //跨节点拉取数据
  ...
  // Get Local Blocks
  fetchLocalBlocks()
  ...
  if (hostLocalBlocks.nonEmpty) {
    blockManager.hostLocalDirManager.foreach(fetchHostLocalBlocks)
  }


以获取远端的shuffle文件拉取为例,ShuffleBlockFetcherIterator遍历fetchRequests,首先得到各个request对应的blockManager,然后向该blockManager发送数据拉取请求:

while (isRemoteBlockFetchable(fetchRequests)) {
  val request = fetchRequests.dequeue()
  val remoteAddress = request.address
  ...
    send(remoteAddress, request)
  }
}

实际的请求发送是在NettyBlockTransferService的fetchBlocks方法中实现的,首先创建TransportClient实例,然后由OneForOneBlockFetcher根据TransportClient实例、appId、executorId等向数据所在的BlockManager发送拉取消息FetchShuffleBlocks并处理返回的结果:

@Override
public void fetchBlocks(
    String host,
    int port,
    String execId,
    String[] blockIds,
    BlockFetchingListener listener,
    DownloadFileManager downloadFileManager) {
    ...
    RetryingBlockFetcher.BlockFetchStarter blockFetchStarter =
        (inputBlockId, inputListener) -> {
          // Unless this client is closed.
          if (clientFactory != null) {
            TransportClient client = clientFactory.createClient(host, port, maxRetries > 0);
            new OneForOneBlockFetcher(client, appId, execId,
              inputBlockId, inputListener, conf, downloadFileManager).start();

当对应BlockManager的NettyBlockRpcServer接收到FetchShuffleBlocks消息后,则根据ShuffleBlockId调用BlockManager的getLocalBlockData方法从本地的shuffle文件中读取所需的数据:

case fetchShuffleBlocks: FetchShuffleBlocks =>
  val blocks = fetchShuffleBlocks.mapIds.zipWithIndex.flatMap { case (mapId, index) =>
    if (!fetchShuffleBlocks.batchFetchEnabled) {
      fetchShuffleBlocks.reduceIds(index).map { reduceId =>
        blockManager.getLocalBlockData(
          ShuffleBlockId(fetchShuffleBlocks.shuffleId, mapId, reduceId))
      }  

值得注意的是,在getLocalBlockData方法的实现代码中我们看到了前面提到的IndexShuffleBlockResolver的实例:

override def getLocalBlockData(blockId: BlockId): ManagedBuffer = {
  if (blockId.isShuffle) {
    logDebug(s"Getting local shuffle block ${blockId}")
    try {
      shuffleManager.shuffleBlockResolver.getBlockData(blockId)
    } 

由于IndexShuffleBlockResolver对象在shuffle map阶段文件合并的过程中创建了index文件,在reduce阶段就可以根据shuffleId、mapId等信息得到具体的index文件,然后根据reduceId获取对应分区的数据长度值在index文件中的偏移量,快速地从data文件中定位到对应partition的数据:

override def getBlockData(
    blockId: BlockId,
    dirs: Option[Array[String]]): ManagedBuffer = {
  val (shuffleId, mapId, startReduceId, endReduceId) = blockId match {
    case id: ShuffleBlockId =>
      (id.shuffleId, id.mapId, id.reduceId, id.reduceId + 1)
 ...
  val indexFile = getIndexFile(shuffleId, mapId, dirs)
  ...
  val channel = Files.newByteChannel(indexFile.toPath)
  channel.position(startReduceId * 8L)
  val in = new DataInputStream(Channels.newInputStream(channel))
  try {
    val startOffset = in.readLong()
    channel.position(endReduceId * 8L)
    val endOffset = in.readLong()
    val actualPosition = channel.position()
    val expectedPosition = endReduceId * 8L + 8
    if (actualPosition != expectedPosition) {
      throw new Exception(s"SPARK-22982: Incorrect channel position after index file reads: " +
        s"expected $expectedPosition but actual position was $actualPosition.")
    }
    new FileSegmentManagedBuffer(
      transportConf,
      getDataFile(shuffleId, mapId, dirs),
      startOffset,
      endOffset - startOffset)
  }

2、执行聚合

如果指定了聚合函数则调用聚合器(Aggregator)的combine CombinersByKey方法在reduce端对数据进行聚合:

val aggregatedIter: Iterator[Product2[KC]] = if (dep.aggregator.isDefined) {
  if (dep.mapSideCombine) {
    // We are reading values that are already combined
    val combinedKeyValuesIterator = interruptibleIter.asInstanceOf[Iterator[(KC)]]
    dep.aggregator.get.combineCombinersByKey(combinedKeyValuesIterator, context)
  } else {
    // We don't know the value type, but also don't care -- the dependency *should*
    // have made sure its compatible w/ this aggregator, which will convert the value
    // type to the combined type C
    val keyValuesIterator = interruptibleIter.asInstanceOf[Iterator[(KNothing)]]
    dep.aggregator.get.combineValuesByKey(keyValuesIterator, context)
  }


3、执行排序

如果需要根据key做排序(例如sortByKey算子),则调用ExternalSorter的insertAll方法对数据进行缓存,当缓存空间无法扩容时则先在内存中排序然后执行溢写,这个过程和map阶段insertAll方法类似,reduce阶段的输出又可以作为下一个shuffle map阶段或者是action的数据源:

// Sort the output if there is a sort ordering defined.
val resultIter = dep.keyOrdering match {
  case Some(keyOrd: Ordering[K]) =>
    // Create an ExternalSorter to sort the data.
    val sorter =
      new ExternalSorter[KCC](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[KC], Iterator[Product2[KC]]](sorter.iterator, sorter.stop())

总结

Shuffle的处理过程较为复杂,并且由于其排序、聚合、磁盘溢写以及数据分发等过程必然会产生CPU、内存、磁盘IO以及网络通信等方面的性能开销,我们在实际的业务开发中需要尽量避免shuffle,或者通过数据过滤和中间结果缓存等方式尽量减少shuffle带来的性能影响。

下面根据本文的内容,Shuffle的原理主要总结如下:

1、Shuffle的产生从根本上是由父子RDD的分区器是否一致决定的,分区器不同则必然产生任务之间的数据分发;

2、Shuffle的过程主要分为map和reduce两个阶段,也即shuffle write和shuffle read阶段:

在shuffle write阶段,根据ShuffleHandle的不同,shuffle写磁盘的过程将采用不同的ShuffleWriter的实现类,本文详细介绍了其中最经典的实现方式SortShuffleWriter,该模式通过PartitionedAppendOnlyMap数据结构在map端将key值相同的数据聚合之后再进行排序和溢写,采用该结构可以减少数据记录占用的内存空间从而提升shuffle的执行性能;BypassMergeSortShuffleWriter 则是跳过了内存合并和排序的过程,直接将shuffle数据溢写到对应分区的临时文件中;而采用UnsafeShuffleWriter可以利用到Tungsten内存模式的红利,通过字节数组来组织数据记录,不仅减少了内存空间的占用,而且大幅减少了数据对象的创建从而减轻JVM的运行压力。

在shuffle read阶段,首先根据shuffleId从mapStatuses中得到对应的MapStatus列表,然后结合reduceId获得reduce任务对应的所有shuffle文件的存储信息,并根据文件所在的存储位置将shuffle记录分配到不同的列表中并分别执行数据拉取;如果定义了合并和排序算法则先在内存中进行合并和排序之后再溢写到磁盘中,否则直接将该分区的数据写入对应的磁盘文件中,并作为下一个shuffle read阶段或者action算子的输入。

作者简介

焦媛,主要负责民生银行Hadoop大数据平台的生产运维工作,并负责HDFS和Spark相关开源产品的技术支持,以及Spark云原生技术的支持和推广工作。


文章转载自民生运维人,如果涉嫌侵权,请发送邮件至:contact@modb.pro进行举报,并提供相关证据,一经查实,墨天轮将立刻删除相关内容。

评论