ESS 服务启动
Executor 向 ESS 注册
- Executor 的初始化
- Spark on Yarn 的 Executor 的启动类是
org.apache.spark.executor.YarnCoarseGrainedExecutorBackend
。 - 在
run()
方法中初始化SparkConf
,SparkEnv
, 然后创建一个名字为 Executor 的CoarseGrainedExecutorBackend
来接收 RPC 请求。 - 因为
CoarseGrainedExecutorBackend
是IsolatedRpcEndpoint
, 所以启动的时候通过向自身发送OnStart
事件来调用onStart()
方法完成初始化. - 向 Driver Endpoint 发送
RegisterExecutor
事件进行注册 - 向自身发送 RegisteredExecutor 事件
- 通过
new Executor()
初始化Executor
, 再向 Driver 发送LaunchedExecutor
事件
- Spark on Yarn 的 Executor 的启动类是
- Executor 初始化时,调用 BlockManager.initialize(appId) 初始化 BlockManager
- 如果开启 ExternalShuffleService 且 BlockManager 不是 Driver,Registering executor with local external shuffle service. registerWithExternalShuffleServer()
BlockManager.registerWithExternalShuffleServer()
blockStoreClient.asInstanceOf[ExternalBlockStoreClient].registerWithShuffleServer(shuffleServerId.host, shuffleServerId.port, shuffleServerId.executorId, shuffleConfig)
// shuffleServerId 的 host 是当前主机, port 是 SHUFFLE_SERVICE_PORT 7337
- BlockManager 向 ESS 发送 RegisterExecutor 事件
public RegisterExecutor(
String appId,
String execId,
ExecutorShuffleInfo executorInfo) {
this.appId = appId;
this.execId = execId;
this.executorInfo = executorInfo;
}
- ESS 接收 RegisterExecutor 消息,将 Executor 信息保存到 BlockManager 的 ConcurrentMap<AppExecId, ExecutorShuffleInfo> executors; 中
try {
RegisterExecutor msg = (RegisterExecutor) msgObj;
checkAuth(client, msg.appId);
blockManager.registerExecutor(msg.appId, msg.execId, msg.executorInfo);
mergeManager.registerExecutor(msg.appId, msg.executorInfo);
callback.onSuccess(ByteBuffer.wrap(new byte[0]));
} finally {
responseDelayContext.stop();
}
Map Task 写Shuffle 数据
这里我们忽略 Shuffle 写,排序,spill 等详细过程,主要看生成最终 Shuffle File 的代码。
在 ShuffleWriteProcessor.write()
方法中先通过 ShuffleManager.getWriter() 获取一个 writer,然后调用 writer.write() 方法写Shuffle 数据。
shuffleExecutorComponents.createMapOutputWriter()
返回LocalDiskShuffleMapOutputWriter
来实现 Shuffle 数据写本地
ShuffleMapOutputWriter createMapOutputWriter()
ShuffleWriteProcessor.write()
writer = manager.getWriter[Any, Any]()
writer.write(
rdd.expansionLimitedIterator(partition, context).
asInstanceOf[Iterator[_ <: Product2[Any, Any]]])
最后 writer.write(iterator)
方法我们看 UnsafeShuffleWriter 的实现,就是遍历迭代器,然后写Shuffle数据。最后 closeAndWriteOutput()
合并最终输出文件。
final ShuffleMapOutputWriter mapWriter = shuffleExecutorComponents.createMapOutputWriter(shuffleId, mapId, partitioner.numPartitions());
创建 Shuffle File 的 writer,写最终的Shuffle 文件。
Shuffle文件也是通过 BlockManager 进行管理的,DataFile 的 BlockId 是 ShuffleDataBlockId(shuffleId, mapId, NOOP_REDUCE_ID)
然后通过 DiskBlockManager.getFile(blockId.name)
获取最终文件。
DiskBlockManager
管理的文件目录结构:
根目录 rootDirs: Array[String] = Utils.getConfiguredLocalDirs(conf)
, 如果是 on Yarn 模式是 conf.getenv("LOCAL_DIRS")
。
一级目录是 val localDirs: Array[File] = rootDirs.map {rootDir => Utils.createDirectory(rootDir, "blockmgr") }
第二级目录是 new File(localDirs(dirId), "%02x".format((hash / localDirs.length) % subDirsPerLocalDir))
例如:
find /hadoop/*/yarn/local/usercache/b_carmel/appcache/application_1651420012128_1874 |grep shuffle_82624_221586707
/hadoop/7/yarn/local/usercache/b_carmel/appcache/application_1651420012128_1874/blockmgr-4d42fc44-d86e-4e9d-b0f1-58f984738fe0/0a/shuffle_82624_221586707_0.index
/hadoop/8/yarn/local/usercache/b_carmel/appcache/application_1651420012128_1874/blockmgr-db467cd8-eec1-4039-b15c-4945a8edcba2/3a/shuffle_82624_221586707_0.data
向 ESS 获取 Shuffle 数据
- Shuffle Read还是从
ShuffleManager.getReader()
开始,然后在 read() 方法内创建 ShuffleBlockFetcherIterator , 然后在initialize()
创建 FetchRequest,在 sendRequest 方法中创建连接,请求数据shuffleClient.fetchBlocks(address.host, address.port, address.executorId, blockIds.toArray, blockFetchingListener, null)
- 在
OneForOneBlockFetcher
内发送OpenBlocks
或者FetchShuffleBlocks
或者FetchShuffleBlockChunks
RPC请求。 - 在
ExternalBlockHandler
中接收到请求后先通过streamManager.registerStream()
创建一个streamId
然后返回new StreamHandle(streamId, numBlockIds)
(OneForOneStreamManager)streamManager.registerStream()
(ConcurrentHashMap<Long, StreamState>)streams.put(streamId++, StreamState)
new StreamState(appId, correlatedId, buffers, channel) // buffers 为读取shuffle结果的迭代器
客户端接收服务端返回的 StreamHandle
,根据是否有 downloadFileManager 也就是是否需要把shuffle文件写本地,有两种获取数据方式:
for (int i = 0; i < streamHandle.numChunks; i++) {
if (downloadFileManager != null) {
client.stream(OneForOneStreamManager.genStreamChunkId(streamHandle.streamId, i),
new DownloadCallback(i));
} else {
client.fetchChunk(streamHandle.streamId, i, chunkCallback);
}
}
-
第一种,如果是小的Shuffle数据,客户端在
client.fetchChunk(streamHandle.streamId, i, chunkCallback);
中向服务器发送ChunkFetchRequest
请求; -
服务端
TransportRequestHandler
接收ChunkFetchRequest
请求,因为在之前的注册 Stream 的时候就已经通过blockManager.getLocalBlockData(blockId)
获取了本地Block 数据,此时将对应 Block 数据对应的ManagedBuffer
返回给客户端。在def getLocalBlockData(blockId: BlockId)
中对于 shuffle block 会调用shuffleManager.shuffleBlockResolver.getBlockData(blockId)
,解析接收到 shuffle blockId, 读取对应的 Index 文件, 找到对应 Data文件的 startOffset 和 endOffset,创建对应的 FileSegmentManagedBuffer 然后返回数据。 -
第二种,如果请求的数据比较大,会通过
client.stream(OneForOneStreamManager.genStreamChunkId(streamHandle.streamId, i), new DownloadCallback(i));
方法中发送StreamRequest
请求; -
服务端
TransportRequestHandler
接收StreamRequest
请求, 通过streamManager.openStream(req.streamId)
获取数据,返回new StreamResponse(req.streamId, buf.size(), buf)
给客户端 -
openStream()
方法调用了上面的ManagedBuffer getChunk(long streamId, int chunkIndex)
方法,返回对应 Shuffle 数据的 Buf, 不同点是客户端在发送请求的时候就会把这个Stream 请求放到streamCallbacks
中 -
当客户端接收到
StreamResponse
后会从streamCallbacks
中找到对应 callback, 创建StreamInterceptor
接收数据并将数据写到文件中。在StreamInterceptor
中不断调用 callback 的onData()
和onComplete()
不断接收数据。
streamManager.getChunk(msg.streamChunkId.streamId, msg.streamChunkId.chunkIndex)
ShuffleManagedBufferIterator.next()
(ExternalShuffleBlockResolver)blockManager.getBlockData(appId, execId, shuffleId, mapIds[mapIdx], reduceIds[mapIdx][reduceIdx]);