Spark中Push Based Shuffle实现

Push Based Shuffle 概述

正常的 Shuffle 会将Map结果写本地一个data文件,然后还有一个index文件存储每个reduce数据的索引。
Pushed Shuffle在Map完成后,将同一个Reduce数据推送到同一个ESS,写到同一个data文件,并用一个index文件存储Map的索引。
此外还有一个Meta文件,Server端不断接受Pushed 数据,当Server端接收到的数据大于Chunk size时,会将一批Map 数据当作一个Chunk来处理,并记录在Meta文件中。
Reduce获取数据的时候,按Chunk来进行获取数据。
通过 RoaringBitmap mapTracker 来跟踪所有接收到的Map,然后返回给Driver,Reduce会优先从ESS 获取shuffle数据,还没有Pushed的Shuffle数据,从原始Map output获取。另外一个是Reduce在根据Chunk获取数据的时候,根据meta 中的RoaringBitmap chunkTracker,就知道来一次Chunk数据就获取了多少个Map的输出。这两个信息必须保持强一致性。

Push Based Shuffle 代码实现

Submit Shuffle Stage

  • 通过 DAGScheduler.submitMissingTasks() 提交 Stage 执行,在 Submit 之前会选择好 PushMergerLocations getAndSetShufflePushMergerLocations(stage)

Map 推送数据

  • ShuffleWriteProcessor.write() 方法调用 ShuffleManager.getWriter().write() 方法写 Map计算结果
  • 通过 ShuffleBlockPusher.initiateBlockPush() 方法开始推送 Shuffle 数据
  • 通过 BlockManager.ExternalBlockStoreClient.pushBlocks() 方法推送数据到指定的 ESS
  • 推送数据使用 RetryingBlockTransferor(OneForOneBlockPusher) 进行包装
  • 通过 TransportClient.uploadStream() 发送 UploadStream 请求
  • 当Map 数据推送完成后,会向Driver 发送 ShufflePushCompletion(shuffleId, shuffleMergeId, mapIndex) 消息。当 Driver 端收集的已完成 Map 个数 / Stage 总 Map 个数 > spark.shuffle.push.minCompletedPushRatio 会立即发起 finalizeShuffleMerge 过程,无需任何等待。

Push Stage Finalized

  • DAGScheduler 等待所有 ShuffleMapTask 全部完成后,调用 finalizeShuffleMerge() 方法,向所有 MergerLocs 发送 FinalizeShuffleMerge RPC 请求
  • 接收从 ESS 返回的 MergeStatuses ,更新到 mapOutputTracker.registerMergeResults()
  • MergeStatuses 结构
/*
* int shuffleId,
* int shuffleMergeId,
* RoaringBitmap[] bitmaps 用于校验 map 之后的数据是否已经存在
* int[] reduceIds: r0  r1  r2  r3
* long[]   sizes:  5   8   9   11
*/
new MergeStatuses(shuffleId, shuffleMergeId,
        bitmaps.toArray(new RoaringBitmap[bitmaps.size()]), Ints.toArray(reduceIds),
        Longs.toArray(sizes))

Reduce 获取 Shuffle 数据

  • ShuffledRowRDD.compute() 在计算的时候会创建一个 Reader,再调用 reader.read() 来读取 Shuffle 数据
  • SortShuffleManager.getReader() 调用 mapOutputTracker.getPushBasedShuffleMapSizesByExecutorId() 获取需要读取的Shuffle Block
  • 如果 Executor 端如果不存在 Shuffle 信息,会向 Master 发送 GetMapAndMergeResultStatuses RPC,请求Shuffle信息,然后在 convertMapStatuses() 方法中合并 mapStatuses 和 mergeStatuses,分别对应 ShuffleBlockIdShuffleMergedBlockId 两种 Block 类型
  • BlockStoreShuffleReader.read() 方法创建 new ShuffleBlockFetcherIterator() 对象获取 Shuffle 数据,对于读取到的Shuffle数据,根据需要设置处理Shuffle数据的 Aggregator 和 Sort
  • ShuffleBlockFetcherIterator 以迭代器的方式读取 Shuffle 数据,通过其内部的 fetchBlock()
  • 创建 FetchRequest 。 对于 ShuffleBlockId 集合会创建 FetchRequest(address, blocks, forMergedMetas = false) 请求,而对于 ShuffleMergedBlockId 会创建 FetchRequest(address, blocks, forMergedMetas = true) 请求
  • 对于 forMergedMetas = false 的 FetchRequest 请求, 直接调用 sendRequest() 方法, 通过 shuffleClient.fetchBlocks() 获取 Shuffle 数据。
  • 对于 forMergedMetas = true 的 FetchRequest 请求, 先调用 shuffleClient.getMergedBlockMeta(): (shuffleId: Int, shuffleMergeId: Int, reduceId: Int, meta: MergedBlockMeta) merged shuffle meta 数据。根据 meta 创建一组 ShuffleBlockChunkId 再重新调用 createFetchRequests() 创建获取Chunk数据的请求

其他说明

  • 为了防止Speculative Task 重复推送数据,ESS 使用RoaringBitmap mapTracker检查数据是否已经被推送过,当收到重复推送的数据时,直接忽略重复推送的数据
  • Spark Job 在Yarn 上运行失败后会进行重试,每次重试会有一个 ApplicationMaster.getAttemptId, 会被转换成 shuffleMergeId, 当ESS收到新的 shuffleMergeId 数据时,会清理旧的 shuffleMergeId 的数据。
  • Map 的 Shuffle 数据是否推送成功无关紧要,只要保证 finalize stage 时,元数据和 Shuffle Data 数据一致即可,对于没有上传到ESS 的Block 数据,会从原始的 Map 进行获取
  • 如果Map Output 数据特别大,比如数据倾斜,对于大于 spark.shuffle.push.maxBlockSizeToPush (1M) 数据的 Block 从原始的 Map 进行获取

举例

这里以一个普通CTAS的Query 的两个State为例

22/05/06 19:30:07,565 INFO [dag-scheduler-event-loop] scheduler.DAGScheduler:57 : Submitting ShuffleMapStage 0 (MapPartitionsRDD[3] at run at AccessController.java:0), which has no missing parents for job Some(0) and jobGroup eb5c5fde-ff39-454b-9b79-31cd80d42a64
22/05/06 19:30:44,999 INFO [dag-scheduler-event-loop] scheduler.DAGScheduler:57 : ShuffleMapStage 0 (run at AccessController.java:0) finished in 36.844 s

22/05/06 19:30:47,925 INFO [dag-scheduler-event-loop] scheduler.DAGScheduler:57 : Submitting ResultStage 2 (MapPartitionsRDD[5] at run at AccessController.java:0), which has no missing parents for job Some(1) and jobGroup eb5c5fde-ff39-454b-9b79-31cd80d42a64
22/05/06 19:33:03,544 INFO [dag-scheduler-event-loop] scheduler.DAGScheduler:57 : ResultStage 2 (run at AccessController.java:0) finished in 135.521 s

正常的Query Map Stage 用时 36秒,Reduce Stage 用时 135秒

22/05/06 19:53:23,462 INFO [dag-scheduler-event-loop] scheduler.DAGScheduler:57 : Submitting ShuffleMapStage 0 (MapPartitionsRDD[3] at run at AccessController.java:0), which has no missing parents for job Some(0) and jobGroup 9cd73d60-1d78-4be0-898f-9b574bfc0e31
22/05/06 19:54:05,803 INFO [dag-scheduler-event-loop] scheduler.DAGScheduler:57 : ShuffleMapStage 0 (run at AccessController.java:0) scheduled for finalizing shuffle merge in 10 s
22/05/06 19:54:15,806 INFO [shuffle-merge-finalizer-0] scheduler.DAGScheduler:57 : ShuffleMapStage 0 (run at AccessController.java:0) finalizing the shuffle merge with registering merge results set to true
22/05/06 19:54:26,280 INFO [dag-scheduler-event-loop] scheduler.DAGScheduler:57 : ShuffleMapStage 0 (run at AccessController.java:0) finished in 62.291 s

22/05/06 19:54:28,316 INFO [dag-scheduler-event-loop] scheduler.DAGScheduler:57 : Submitting ResultStage 2 (MapPartitionsRDD[5] at run at AccessController.java:0), which has no missing parents for job Some(1) and jobGroup 9cd73d60-1d78-4be0-898f-9b574bfc0e31
22/05/06 19:56:39,603 INFO [dag-scheduler-event-loop] scheduler.DAGScheduler:57 : ResultStage 2 (run at AccessController.java:0) finished in 125.790 s

开启PushBased的Query Map Stage 用时 62秒,Reduce Stage 用时 125秒.
其中Map Stage包含 spark.shuffle.push.results.timeout = 10sspark.shuffle.push.finalize.timeout = 10s 两个等待时间

  • spark.shuffle.push.finalize.timeout : 在发送finalizeShuffleMerge 请求之前,等待 inFlight Block 传输完成并被merge的等待时间
  • spark.shuffle.push.results.timeout : 向所有 ESS 发送 finalizeShuffleMerge 请求,最长等待时间
  • 0
    点赞
  • 0
    收藏
    觉得还不错? 一键收藏
  • 0
    评论
评论
添加红包

请填写红包祝福语或标题

红包个数最小为10个

红包金额最低5元

当前余额3.43前往充值 >
需支付:10.00
成就一亿技术人!
领取后你会自动成为博主和红包主的粉丝 规则
hope_wisdom
发出的红包
实付
使用余额支付
点击重新获取
扫码支付
钱包余额 0

抵扣说明:

1.余额是钱包充值的虚拟货币,按照1:1的比例进行支付金额的抵扣。
2.余额无法直接购买下载,可以购买VIP、付费专栏及课程。

余额充值