Spark-ShuffleWriter-UnsafeShuffleWriter
一、上下文
《Spark-ShuffleWriter》中对ShuffleWriter的获取、分类和写入做了简单的分析,下面我们对其中的UnsafeShuffleWriter做更详细的学习
二、构建UnsafeShuffleWriter
public UnsafeShuffleWriter(
BlockManager blockManager,
TaskMemoryManager memoryManager,
SerializedShuffleHandle<K, V> handle,
long mapId,
TaskContext taskContext,
SparkConf sparkConf,
ShuffleWriteMetricsReporter writeMetrics,
ShuffleExecutorComponents shuffleExecutorComponents) throws SparkException {
final int numPartitions = handle.dependency().partitioner().numPartitions();
//如果分区数 大于 2的24次方(16777215) 会直接报异常
if (numPartitions > SortShuffleManager.MAX_SHUFFLE_OUTPUT_PARTITIONS_FOR_SERIALIZED_MODE()) {
throw new IllegalArgumentException(
"UnsafeShuffleWriter can only be used for shuffles with at most " +
SortShuffleManager.MAX_SHUFFLE_OUTPUT_PARTITIONS_FOR_SERIALIZED_MODE() +
" reduce partitions");
}
this.blockManager = blockManager;
this.memoryManager = memoryManager;
this.mapId = mapId;
final ShuffleDependency<K, V, V> dep = handle.dependency();
this.shuffleId = dep.shuffleId();
this.serializer = dep.serializer().newInstance();
this.partitioner = dep.partitioner();
this.writeMetrics = writeMetrics;
this.shuffleExecutorComponents = shuffleExecutorComponents;
this.taskContext = taskContext;
this.sparkConf = sparkConf;
//默认开启零拷贝
this.transferToEnabled = sparkConf.getBoolean("spark.file.transferTo", true);
//spark.shuffle.sort.initialBufferSize 默认值 4096
//排序缓冲区大小
this.initialSortBufferSize =
(int) (long) sparkConf.get(package$.MODULE$.SHUFFLE_SORT_INIT_BUFFER_SIZE());
//spark.shuffle.file.buffer 默认值 32k
//每个随机文件输出流的内存缓冲区大小,它减少了在创建中间shuffle文件时进行的磁盘查找和系统调用的数量
this.inputBufferSizeInBytes =
(int) (long) sparkConf.get(package$.MODULE$.SHUFFLE_FILE_BUFFER_SIZE()) * 1024;
open();
}
static final int DEFAULT_INITIAL_SER_BUFFER_SIZE = 1024 * 1024;
private void open() throws SparkException {
assert (sorter == null);
//初始化排序器 专门用于基于排序的Shuffle
//将记录数据添加到数据页,当所有记录都已插入时或当达到当前线程的shuffle内存限制时
//使用ShuffleInemorySorter将内存中的数据按照分区ID进行排序
//然后将排序后的记录作为序列化的压缩流写入单个或多个输出文件
//这与 SortShuffleWriter 中使用的 ExternalSorter 不同
//此排序器不会合并其溢出文件。具体的合并时由UnsafeShuffleWriter做的(使用一个专门的合并过程来避免额外的序列化/反序列化)
sorter = new ShuffleExternalSorter(
memoryManager,
blockManager,
taskContext,
initialSortBufferSize,
partitioner.numPartitions(),
sparkConf,
writeMetrics);
//默认serBuffer = 1M
//MyByteArrayOutputStream 是 ByteArrayOutputStream 的子类
//缓冲区在写入数据时会自动增长
//此类中的方法可以在流关闭后调用,而不会生成IOException。
serBuffer = new MyByteArrayOutputStream(DEFAULT_INITIAL_SER_BUFFER_SIZE);
//初始化序列化输出流
serOutputStream = serializer.serializeStream(serBuffer);
}
三、将数据插入排序器
1、将单条数据序列化并写入缓存
while (records.hasNext()) {
//将该ShuffleMapTask中迭代器中的数据一条一条放入排序器
insertRecordIntoSorter(records.next());
}
void insertRecordIntoSorter(Product2<K, V> record) throws IOException {
assert(sorter != null);
final K key = record._1();
//计算分区
final int partitionId = partitioner.getPartition(key);
//将此字节数组输出流的计数字段重置为零,以便丢弃输出流中当前累积的所有输出。输出流可以再次使用,重用已分配的缓冲区空间。
serBuffer.reset();
//将kv数据写入serBuffer
serOutputStream.writeKey(key, OBJECT_CLASS_TAG);
serOutputStream.writeValue(record._2(), OBJECT_CLASS_TAG);
serOutputStream.flush();
//获取serBuffer中的数据长度
final int serializedRecordSize = serBuffer.size();
assert (serializedRecordSize > 0);
//向shuffle sorter写入一条数据
//Platform.BYTE_ARRAY_OFFSET = 16 为 字节数组在内存中存放时的长度
//最终会调用 JDK的 native 方法 即 Unsafe的arrayBaseOffset()
sorter.insertRecord(
serBuffer.getBuf(), Platform.BYTE_ARRAY_OFFSET, serializedRecordSize, partitionId);
}
2、构建ShuffleExternalSorter
final class ShuffleExternalSorter extends MemoryConsumer implements ShuffleChecksumSupport {
static final int DISK_WRITE_BUFFER_SIZE = 1024 * 1024;
private final int numPartitions;
private final TaskMemoryManager taskMemoryManager;
private final BlockManager blockManager;
private final TaskContext taskContext;
private final ShuffleWriteMetricsReporter writeMetrics;
//当内存中有这么多元素时,强制溢写
private final int numElementsForSpillThreshold;
//使用DiskBlockObjectWriter溢写使用的缓冲区大小
private final int fileBufferSizeBytes;
//将排序后的记录写入磁盘文件时使用的缓冲区大小
private final int diskWriteBufferSize;
//保存正在排序的记录的内存页。
//溢写时,此列表中的页面会被释放回收
//如果在TaskMemoryManager本身中维护了一个可重复使用的页面池,可以不释放回收
private final LinkedList<MemoryBlock> allocatedPages = new LinkedList<>();
private final LinkedList<SpillInfo> spills = new LinkedList<>();
// 溢写后重置这些变量
@Nullable private ShuffleInMemorySorter inMemSorter;
@Nullable private MemoryBlock currentPage = null;
private long pageCursor = -1;
//每个分区的校验和计算器。禁用Shuffle校验和时为空。
private final Checksum[] partitionChecksums;
ShuffleExternalSorter(
TaskMemoryManager memoryManager,
BlockManager blockManager,
TaskContext taskContext,
int initialSize,
int numPartitions,
SparkConf conf,
ShuffleWriteMetricsReporter writeMetrics) throws SparkException {
super(memoryManager,
(int) Math.min(PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES, memoryManager.pageSizeBytes()),
memoryManager.getTungstenMemoryMode());
this.taskMemoryManager = memoryManager;
this.blockManager = blockManager;
this.taskContext = taskContext;
this.numPartitions = numPartitions;
// spark.shuffle.file.buffer 默认 32K
//每个随机文件输出流的内存缓冲区大小
//这些缓冲区减少了在创建中间洗牌文件时进行的磁盘查找和系统调用的数量。
this.fileBufferSizeBytes =
(int) (long) conf.get(package$.MODULE$.SHUFFLE_FILE_BUFFER_SIZE()) * 1024;
//spark.shuffle.spill.numElementsForceSpillThreshold 默认值 Integer.MAX_VALUE
//这意味着我们永远不会强制排序器溢写,直到我们达到一些限制,比如排序器中指针数组的最大页面大小限制。
this.numElementsForSpillThreshold =
(int) conf.get(package$.MODULE$.SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD());
this.writeMetrics = writeMetrics;
//spark.shuffle.sort.useRadixSort 默认 true
//是否使用基数排序对内存分区ID进行排序。基数排序要快得多,但在添加指针时需要额外的内存作为保留内存
this.inMemSorter = new ShuffleInMemorySorter(
this, initialSize, (boolean) conf.get(package$.MODULE$.SHUFFLE_SORT_USE_RADIXSORT()));
//已经用了多少内存
this.peakMemoryUsedBytes = getMemoryUsage();
//spark.shuffle.spill.diskWriteBufferSize 默认 1024 * 1024 即 1M
//将排序后的记录写入磁盘文件时使用的缓冲区大小
this.diskWriteBufferSize =
(int) (long) conf.get(package$.MODULE$.SHUFFLE_DISK_WRITE_BUFFER_SIZE());
//分区校验和
this.partitionChecksums = createPartitionChecksums(numPartitions, conf);
}
}
3、向排序器插入数据
public void insertRecord(Object recordBase, long recordOffset, int length, int partitionId)
throws IOException {
//..........省略..............
//检查是否有足够的空间在排序指针数组中插入额外的记录,
//如果需要额外的空间,则扩展数组。如果无法获得所需的空间,则内存中的数据将溢写到磁盘
growPointerArrayIfNecessary();
//当运行JVM时,其中有sun的Unsafe包可用,并且底层系统具有未对齐的访问能力 uaoSize = 4
//否则 uaoSize = 8
final int uaoSize = UnsafeAlignedOffset.getUaoSize();
//需要4或8个字节来存储记录长度
final int required = length + uaoSize;
//分配更多内存以插入其他记录。这将向内存管理器请求额外的内存,如果无法获得所请求的内存,则会溢写
acquireNewPageIfNecessary(required);
assert(currentPage != null);
final Object base = currentPage.getBaseObject();
//给定一个内存页和该页内的偏移量,将此地址编码为64位长。只要相应的页面未被释放,此地址将保持有效。
final long recordAddress = taskMemoryManager.encodePageNumberAndOffset(currentPage, pageCursor);
//将这条已经序列化并写入缓存的数据索引放到内存页中
UnsafeAlignedOffset.putSize(base, pageCursor, length);
//内存页游标向右移动 4个 字节
pageCursor += uaoSize;
//将缓存中的这条数据复制到 堆外
Platform.copyMemory(recordBase, recordOffset, base, pageCursor, length);
//内存页游标 向右移动 数据长度
pageCursor += length;
//将记录地址和分区id打包 并 插入数组中,将来排序只排序索引即可
//数组中只有一部分将用于存储指针,其余部分将保留为临时缓冲区用于排序。
inMemSorter.insertRecord(recordAddress, partitionId);
}
private void growPointerArrayIfNecessary() throws IOException {
assert(inMemSorter != null);
if (!inMemSorter.hasSpaceForAnotherRecord()) {
long used = inMemSorter.getMemoryUsage();
LongArray array;
try {
// 可能引发溢写
//调用父类即MemoryConsumer 的 allocateArray()
array = allocateArray(used / 8 * 2);
} catch (TooLargePageException e) {
//.........
}
// 检查是否触发溢写
if (inMemSorter.hasSpaceForAnotherRecord()) {
freeArray(array);
} else {
inMemSorter.expandPointerArray(array);
}
}
}
4、MemoryConsumer分配LongArray
支持溢写的TaskMemoryManager内存消耗者(仅仅支持Tungsten memory)
public abstract class MemoryConsumer {
protected final TaskMemoryManager taskMemoryManager;
public LongArray allocateArray(long size) {
long required = size * 8L;
//最终调用TaskMemoryManager 来申请page page的类型是 MemoryBlock
MemoryBlock page = taskMemoryManager.allocatePage(required, this);
if (page == null || page.size() < required) {
throwOom(page, required);
}
used += required;
return new LongArray(page);
}
}
TaskMemoryManager
负责管理单个任务分配的内存。这里会触发Spark 的钨丝内存分配
此类中的大部分复杂性涉及将堆外地址编码为64位长。在堆外模式下,内存可以直接用64位长进行寻址。在堆上模式下,内存由基对象引用和该对象内的64位偏移量的组合来寻址。当我们想将指向数据结构的指针存储在其他结构中时,是一个问题,例如哈希映射或排序缓冲区中的记录指针。即使我们决定使用128位来寻址内存,我们也不能只存储基对象的地址,因为当堆因GC而重新组织时,它不能保证保持稳定。
相反,我们使用以下方法将记录指针编码为64位长:对于堆外模式,只存储原始地址,对于堆上模式,使用地址的高位13位存储“页码”,低位51位存储此页内的偏移量。这些页码用于索引到MemoryManager内的“页表”数组中,以检索基本对象。
这使我们能够处理8192(2的13次方)页。在堆上模式下,最大页面大小受到 long[] 数组最大大小的限制,使我们能够寻址8192*(2^31-1)*8字节,约为140TB的内存。
public class TaskMemoryManager {
//最大 page 大小原则上应该是 2的51次方,但是堆上模式中受到 long [] 中可存的数据量限制,因此
//最大 page 大小为 2的31次方 -1 * 8 byte 大概是 17G
public static final long MAXIMUM_PAGE_SIZE_BYTES = ((1L << 31) - 1) * 8L;
public MemoryBlock allocatePage(long size, MemoryConsumer consumer) {
assert(consumer != null);
assert(consumer.getMode() == tungstenMemoryMode);
//申请的page大小限制 最大约为 17G
if (size > MAXIMUM_PAGE_SIZE_BYTES) {
throw new TooLargePageException(size);
}
//为内存消费者 可分配的执行内存 的大小
long acquired = acquireExecutionMemory(size, consumer);
if (acquired <= 0) {
return null;
}
final int pageNumber;
synchronized (this) {
pageNumber = allocatedPages.nextClearBit(0);
if (pageNumber >= PAGE_TABLE_SIZE) {
releaseExecutionMemory(acquired, consumer);
throw new IllegalStateException(
"Have already allocated a maximum of " + PAGE_TABLE_SIZE + " pages");
}
allocatedPages.set(pageNumber);
}
MemoryBlock page = null;
try {
//调用钨丝内存分配器取分配内存,有两种模式:
// 1、ON_HEAP -> HeapMemoryAllocator
// 2、OFF_HEAP -> UnsafeMemoryAllocator
//这也是一个调优点,你可以选择堆上分配,也可以选择堆外分配,默认堆上分配
//我们单独用一篇博客来分析Spark 的 钨丝内存分配
page = memoryManager.tungstenMemoryAllocator().allocate(acquired);
} catch (OutOfMemoryError e) {
logger.warn("Failed to allocate a page ({} bytes), try again.", acquired);
// 实际上没有足够的内存,这意味着实际的可用内存比MemoryManager想象的要小,我们应该保留获得的内存。
synchronized (this) {
acquiredButNotUsed += acquired;
allocatedPages.clear(pageNumber);
}
// this could trigger spilling to free some pages.
return allocatePage(size, consumer);
}
page.pageNumber = pageNumber;
pageTable[pageNumber] = page;
if (logger.isTraceEnabled()) {
logger.trace("Allocate page number {} ({} bytes)", pageNumber, acquired);
}
return page;
}
}
5、排序器中的数据长相
final class ShuffleInMemorySorter {
//存放数据地址和分区id的数组
//该数组与原生JVM数组相比有以下特点:
// 1、支持同时使用on-heap 和 off-heap
// 2、没有绑定检查,因此在关闭断言时可能会使JVM进程崩溃
//单个元素长度为 64 字节
private LongArray array;
public void insertRecord(long recordPointer, int partitionId) {
if (!hasSpaceForAnotherRecord()) {
throw new IllegalStateException("There is no space for new record");
}
array.set(pos, PackedRecordPointer.packPointer(recordPointer, partitionId));
pos++;
}
}
final class PackedRecordPointer {
public static long packPointer(long recordPointer, int partitionId) {
assert (partitionId <= MAXIMUM_PARTITION_ID);
// 注意,如果没有字节对其,一个内存页 可以寻址 128M字节数据 也就是 2的27次方
final long pageNumber = (recordPointer & MASK_LONG_UPPER_13_BITS) >>> 24;
final long compressedAddress = pageNumber | (recordPointer & MASK_LONG_LOWER_27_BITS);
// | 前 24字节 | 13 字节 - 27字节 |
// | 分区id | 内存页编号 - 页内偏移 |
//这也是为什么该UnsafeShuffleWriter只能支持 2 的 24次方 个分区的原因了
return (((long) partitionId) << 40) | compressedAddress;
}
}
我们用图来描述下,排序器中的数据是如何存储的
四、溢写磁盘并释放内存
void closeAndWriteOutput() throws IOException {
assert(sorter != null);
updatePeakMemoryUsed();
serBuffer = null;
serOutputStream = null;
//SpillInfo 是由 ShuffleExternalSorter 写入的数据块的元数据
//关闭排序器,并缓存中的数据进行排序并进行溢写
final SpillInfo[] spills = sorter.closeAndGetSpills();
try {
//将零个或多个溢出文件合并在一起,根据溢出数量和IO压缩编解码器选择最快的合并策略。
partitionLengths = mergeSpills(spills);
} finally {
//..............
}
mapStatus = MapStatus$.MODULE$.apply(
blockManager.shuffleServerId(), partitionLengths, mapId);
}
//-----------------------------
final class ShuffleExternalSorter extends MemoryConsumer implements ShuffleChecksumSupport {
public SpillInfo[] closeAndGetSpills() throws IOException {
if (inMemSorter != null) {
// 对内存中的记录进行排序,并将排序后的记录写入磁盘文件。
writeSortedFile(true);
//释放内存页和数据块占用的内存
freeMemory();
//释放内存页指针数组结构所用的内存
inMemSorter.free();
inMemSorter = null;
}
return spills.toArray(new SpillInfo[spills.size()]);
}
}
五、向调度器报告摘要信息
ShuffleMapTask向调度器返回的结果。包括任务存储shuffle文件的块管理器地址,以及每个reducer的输出大小,以便传递给reduce任务。
当调度器发现这个ShuffleMapTask执行完成,就会执行下一个ShuffleMapTask或者ResultTask