1. 概述
shuffle可以说是spark中的难点,本篇文章主要讲解shuffle过程中的一些原理,提纲如下:
- shuffle write过程
- shuffle read过程
- shuffle优化
2. shuffle write 过程
上面的图描述了整个shuffle write的整个流程,描述如下:
- 当遇到action算子,提交任务时,
DAGScheduler
按ShuffleDependency
划分stage,除了最后的Stage为ResultStage
之外,其余的stage都是ShuffleMapStage
DAGScheduler
在创建ShuffleMapStage
时,将该shuffle以(shuffleId,ShuffleStatus)的形式注册到MapOutputTrackerMaster
的变量shuffleStatuses
中,ShuffleStatus
可以看作是以mapId为下标构成的MapStatus数组ShuffleMapStage
会构建由一个个ShuffleMapTask
组合而成的TaskSet
,将ShuffleMapTask
提交到executor上执行,最中调用ShuffleMapTask
的runTask
方法来进行shuffle write- shuffle write结束后,将写出的结果以文件的形式保存到executor节点的
BlockManager
中,任务执行成功后,将MapStatus
封装为DirectTaskResult
给driver端的DriverEndpoint
DriverEndpoint
收到executor发过来的StatusUpdate
事件,最后调用DAGScheduler
将返回的MapStatus
按shuffleId、mapId存入到MapOutputTrackerMaster
中
返回MapStatus
时需要处理返回给driver的结果,逻辑在TaskRunner
的run方法中,这里顺便记录一下:
- 假设结果序列化后的结果为S,如果S > spark.driver.maxResultSize(默认1g),则结果会被丢弃,返回元数据
new IndirectTaskResult[Any](TaskResultBlockId(taskId), resultSize)
- 否则如果S > spark.task.maxDirectResultSize(默认1M),结果被保留到该任务所在的
BlockManager
中,并通知给driver端的BlockManagerMaster
,并返回元数据new IndirectTaskResult[Any](TaskResultBlockId(taskId), resultSize)
- 否则,直接返回序列化后的结果
2.1 ShuffleWriter的选取
shuffle过程通过类ShuffleManager
进行管理,在2.x只有一个实现类SortShuffleManager
,它决定了如何读与写shuffle数据。shuffle的写通过ShuffleWriter
进行抽象,有三个实现类:
ShuffleWriter实现类 | 采用条件 |
---|---|
BypassMergeSortShuffleWriter | 如果map端不用合并(比如groupByKey算子),且分区数不大于spark.shuffle.sort.bypassMergeThreshold (默认200) |
UnsafeShuffleWriter | serializer的supportsRelocationOfSerializedObjects 属性为true;且不能mapSideCombine,因为序列化的数据无法combine;reduce数不能超过2^24 |
SortShuffleWriter | 不能使用BypassMergeSortShuffleWriter与UnsafeShuffleWriter的最后选择 |
2.1.1 BypassMergeSortShuffleWriter
过程概述:
- 每个reduce分区构建一个DiskObjectWriter
- 每个分区写入到一个文件中
- 将每个分区的文件合并为一个大的文件
- 生成一个针对合并文件的索引文件
优点:
- 不需要排序,只需要序列化写,相对于其它需要排序的writer,不存在溢写临时文件,少了合并溢写文件的过程;由于写的过程是先将key/value序列化为字节数组,所以强烈建议使用KryoSerializer
缺点:
- 对每一个
ShuffleMapTask
,每一个reduce分区对应一个DiskObjectWriter,相当于每一个分区文件都有一个缓冲区,其中缓冲区由spark.shuffle.file.buffer
配置决定,默认为32k
,极端情况下需要 map个数 * reduce个数 * spark.shuffle.file.buffer KB的内存,增大了内存负载- 虽然最后文件的个数是reduce的个数,但中间过程中,最大的文件个数为
map个数
*reduce个数
,增加了文件系统负载
2.1.2 SortShuffleWriter
过程如上,补充一些点:
- shuffle通过
ExternalSorter
对数据进行排序与溢写- 如果需要在map端合并,
ExternalSorter
使用PartitionedAppendOnlyMap
作为缓冲区,否则使用PartitionedPairBuffer
作为缓冲区;两者内部的数据结构一致,都使用一个数组来存储数据,key与value都放在数组中,key放在偶数位(2*pos)
,value放在key的下一个索引位置(2*pos+1)
,只是在插入数据的时候,前者使用hash+开放定址法来存放数据,后者直接追加- 图中排序的
Key
实际的类型为(Int, K),前者表示分区,后者为实际的Key类型。排序规则为如下:3.1 先按reduce分区升序排序
3.2 如果为同一个分区,且需要mapSideCombinie,则还需对K进行排序,如果对应的ShuffleDependency的keyOrdering不为None
则使用keyOrdering
进行升序排序,否则使用K的hashcode进行升序排序
- 使用执行内存,其溢写规则为:
4.1 如果缓冲区的数据条数超过阈值(
spark.shuffle.spill.numElementsForceSpillThreshold
,默认是Long.MAX_VALUE),则溢写
4.2 通过采样估算当前缓存中的数据大小curMemory
,如果大于阈值myMemoryThreshold
(初始值由spark.shuffle.spill.initialMemoryThreshold配置,默认为5 * 1024 * 1024),则会向TaskMemoryManager申请执行内存,参试将myMemoryThreshold
变为curMemory
的2倍,如果申请到的内存+原myMemoryThreshold<= curMemory
,也要进行溢写
- 索引文件就是按reduce分区顺序写的每个reduce分区数据的大小,从0开始
- 最终文件个数为
map任务数*2
优点:
- 一个task一个缓冲区,能更精细的管理内存使用
缺点:
- 需要对数据进行排序
- 可能需要归并排序。由于归并排序是针对文件的,这些文件中的数据在写入时是序列化过,此时归并排序又需要反序列化回来进行比较,增大了cpu负载
2.1.3 UnsafeShuffleWriter
在讲解之前,需要一些知识储备。由于UnsafeShuffleWriter
的一大特点是操作序列化数据(字节数组),spark为了更高效的管理内存,spark引入了Page的概念,Page代表一块连续的内存,具体实现为MemoryBlock
// spark.memory.offHeap.enabled为true时,使用堆外内存
public class MemoryBlock extends MemoryLocation {
Object obj; // 使用堆外内存时为null,否则为long[]
long offset; // 使用堆外内存时为绝对地址,否则为第一个元素到long数组开始地址的偏移量
private final long length; // 不可变,代表的page的大小,单位为字节
}
有了存放数据的容器,那么一条序列化的数据(下面称为记录)是怎么在MemoryBlock中存放的?上图
- 有两个
MemoryBlock
,一个用来排序与索引数据,称为pointerArray,另一个用来存放数据,称为page。- 对于pointerArray,用一个long型整数来表示一条记录,前24位为reduce分区,接下来的13位为在
pageTable
中的索引,后面的27位为数据在page中的偏移量(单位为字节)。这就是为什么reduce的分区数不能超过2^24- 对于page,一条记录由两部分组成,为记录大小(4B)+记录本身
- pointerArray的工作模式为使用reduce分区来排序,然后通过索引定位到page,然后通过偏移量获取数据大小,这样就可以获取整条记录,从而对记录进行排序了
那什么情况下,serializer的方法supportsRelocationOfSerializedObjects
返回true,它需要满足:
- 对serializer S,假设序列化对象A得到字节数组[A],放入到缓存区
- 接着序列化对象B得到字节数组[B],放入到缓存区,这时缓存区的布局为[A][B]
- 对于S,对该缓存区进行反序列化读,会得到A、B
- 但是如果将缓存区布局调换为[B][A],当该serializer也能反序列化读得到B、A,则满足
JavaSerializer
不能,KryoSerializer
可以
为什么要有这个特性?因为序列化记录在写入文件时需要按照reduceId排序,假设有2条记录A、B,写入到page中的顺序为AB,排序后变为了BA,写入到文件中,在reduce端被反序列化时,也要正确的得到B、A
有了上面的铺垫,那整个写过程就比较好理解了
上图概述了整个写过程,写过程细节较多,我将一些核心的点记录下来,供大家在深入源码时参考:
- 排序:pointerArray被用来排序,排序规则为reduce分区升序排列
- 溢写:在溢写前,需要 排序,这样每个reduce分区的数据就是连续的。溢写的内容为记录本身,每一次溢写会生成一个文件,使用SpillInfo来记录,里面有一个数组记录本次溢写每个reduce分区的数据大小,下标即是reduceId,这样合并时,顺序读取每个分区的数据,然后依次拼接就行了。溢写阈值:
2.1 插入到pointerArray的记录超过
spark.shuffle.spill.numElementsForceSpillThreshold
(默认Integer.MAX_VALUE)
2.2 插入到pointerArray的记录大小超过了Page的最大值(即long[]的最大长度(2^31 - 1)
*8
)
2.3 当获取的内存小于请求的内存时,详见TaskMemoryManager
的acquireExecutionMemory
优点:
- 数据被序列化后大小变小,能更好的利用内存,相较于
SortShuffleWriter
,合并溢写文件时不需要反序列化
- 排序更快,数组排序能利用cpu的缓存更快的排序
2.1.4 总结
map端其实不管key是否需要排序,是否聚合与排序取决于是否需要mapSideCombine
3. shuffle read 过程
shuffle读由ShuffledRDD
的compute
方法触发,ShuffleReader只有一个实现BlockStoreShuffleReader
。在介绍以前,也需要一些铺垫
3.1 铺垫一: BlockManager介绍
BlockManager
管理所有需要保留在该Executor的数据,他是消耗存储内存的唯一用户。BlockManager
使用NettyBlockTransferService
通过TransportServer
向外部提供数据下载服务,也使用NettyBlockTransferService
通过TransportClient
连接到外部的NettyBlockTransferService
获取数据,比如shuffle数据、broadcast数据。在图中的inbound(ChannelInboundHandler)用来处理从外面来的数据,outbound(ChannelOutboundHandler)用来处理出去的数据。整个shuffle数据的读取就是在这个链路中处理的。其中比较主要的是TransportChannelHandler
,它持有的TransportResponseHandler
用来处理接收的response信息,TransportRequestHandler
用来处理接收的request信息。
3.2 铺垫二: MapStatus
MapStatus表示的是ShuffleMapTask执行的位置与输出文件每个分区的大小,在实现中,该分区大小并不是实际的大小(被特别处理了,减少返回给driver的数据量),作用是用来限定reduce端每一次请求的数据量。真正的数据大小需要通过索引文件来获得
3.3 获取所有shuffle输出的所在位置
MapOutputTrackerWorker
通过MapOutputTrackerMasterEndpoint
向MapOutputTrackerMaster
获取本次shuffleId对应的所有的MapStatus
(数组),然后根据MapStatus
的executorId_划分出远程请求与本地请求。对于远程请求,也做了流控,每一个远程请求获取的量保持在spark.reducer.maxSizeInFlight(默认48M)/5
左右。本地的数据就直接向BlockManager
要就行了。
shuffle数据的存储有固定的规范,对于数据文件,规范为shuffle_${shuffleId}_${mapId }_ 0.data,索引文件为shuffle_${shuffleId}_${mapId }_ 0.index,这样在请求shuffle数据时,通过shuffleId与mapId即可获取数据文件,其中,shuffleId由ShuffleDependency提供,mapId即为MapStatus数组的下标。整个过程为先通过reduceId在索引文件中获取偏移量与长度,然后从数据文件中获取数据
3.4 获取远程的shuffle文件
这个过程由ShuffleBlockFetcherIterator
的initialize
方法执行,整个过程异步执行(netty特性),在使用数据的时候通过将获取的数据放入到LinkedBlockingQueue
,将异步转成同步。整个过程有两个阶段:
- 数据准备阶段
- 数据传输阶段
3.4.1 数据准备阶段
如图:
- 客户端发送
OpenBlocks
被TransportRequestHandler
接收,NettyBlockRpcServer
通过索引文件获取reduce分区的偏移量与大小,然后与数据文件构造成FileSegmentManagedBuffer
,然后注册到OneForOneStreamManager
,返回StreamId
- 服务端将
StreamId
与该id中包含的block数构造成StreamHandle
返回给客户端
3.4.2 数据传输阶段
如果OpenBlocks
中所有block的大小之和超过了spark.maxRemoteBlockSizeFetchToMem(默认Long.MaxValue),接收到StreamHandle
后,开始去请求数据,会将接收到的数据放入到文件中,否则放入到内存里。最后的结果被放到results:LinkedBlockingQueue
中,供ShuffleBlockFetcherIterator
的next
方法调用
需要指出的是,传输过程还是比较复杂的,涉及到了很深的netty知识。当接收数据使用默认配置时,基本不会使用文件去接收,而是放在内存里,因为数据的大小不会超过Long.MaxValue
,那么spark是怎样保证不OOM的?在ShuffleBlockFetcherIterator
的方法fetchUpToMaxBytes
中,限制了每次去拉取的数据的大小尽量不超过spark.reducer.maxSizeInFlight(默认48M)。 当map端数据倾斜时,可能会导致Container因为内存超过阈值而被杀掉的情况,spark为了避免这个情况,在TransportFrameDecoder
中限制了数据大小不能超过Int.MAX_VALUE
,此时需要设置参数spark.maxRemoteBlockSizeFetchToMem到小于Int.MAX_VALUE
3.5 合并数据
如果数据需要聚合(对应的ShuffleDependency的aggregator可用),如groupByKey
,则需要将所有的数据反序列化后添加到ExternalAppendOnlyMap
进行聚合(与SortShuffleWriter中的PartitionedAppendOnlyMap
类似,key没有reduceId):
- 先往内存中写,如果内存不足则溢写,溢写时对key进行排序
- 然后使用堆(
PriorityQueue
)对溢写的文件与未溢写的内存数据进行归并排序,key相同的还需要合并- 排序规则: 实现类
HashComparator
,使用hashcode进行排序
需要注意的是,由于排序使用的是key的hashcode,在合并时需要将所有hashcode相同的KV全部加载到内存中,然后通过equals来合并,所以如果hashcode冲突大时,会影响性能
3.6 排序数据
如果数据需要排序(对应的ShuffleDependency的keyOrdering可用),如sortByKey
、repartitionAndSortWithinPartitions
,则还需要使用ExternalSorter
(在SortShuffleWriter
已讲过了)对合并后的数据进行排序
3.7 总结
在看代码的过程中,思考过为什么在数据传输的阶段没有控制内存使用的代码,如果shuffle数据量比较大,内存不会不够用吗?原来数据不是一次传输过来的,除了在构造ShuffleBlockFetcherIterator
时,会去获取数据(此时数据量已被spark.reducer.maxSizeInFlight(默认48M)控制),余下的获取数据的都需要调用next
方法才行,且每次去获取的数据量也被spark.reducer.maxSizeInFlight(默认48M)控制,而下游又在消费数据,这样每次放在内存的数据并不是所有recuce的数据。
还有一个问题,为什么排序数据不与合并数据合并在一起?从实现来看,map端数据排不排序与keyOrdering可用没有关系,只与需不需要mapSideCombine有关,而排序数据必须要求map端的数据有序,这个要求map端不能保证(BypassMergeSortShuffleWriter
与UnsafeShuffleWriter
都不能)。小伙伴们可能觉得这会不会有性能问题,因为如果即需要合并又需要排序,那数据需要溢写2次,从代码上看,这种算子很少,笔者还没有碰到过。
4. 优化
- 可以看到,序列化存在于write与read,所以高效的serializer对提升shuffle的速度有帮助,建议使用KryoSerializer
- 当数据量较大时,会使用
DiskBlockObjectWriter
多次进行溢写,该写buffer的大小由spark.shuffle.file.buffer
决定,默认为32K
,可以根据executor使用的内存大小来调整该值,以减少写入次数,提升IO效率- spark.reducer.maxSizeInFlight:在shuffle read的获取远程数据阶段,为了避免spark占用过大的网络流量,对每次请求的数据量进行了限制,该数据量由该参数决定,默认为48M,如果集群使用了万兆网卡或集群网络负载较低,可以增大该值,以增大每次连接获取的数据量,减少获取数据的时间
- spark.maxRemoteBlockSizeFetchToMem:默认是
Long.MaxValue
,当map端出现数据倾斜导致输出的属于某个分区的数据过大(超过2g)时,需要将该值设置的小一些,比如小于2g的某个值,以避免出现错误Too large frame