Flink在两个Task之间建立Netty连接进行数据传输,每一个Task会分配两个缓冲池,一个用于输出数据,一个用于接收数据。当一个Task的缓冲池用尽之后,网络连接就处于阻塞状态,上游Task无法产出数据,下游Task无法接收数据,也就是我们所说的“反压”状态。这是一种非常自然的“反压”的机制,但是过程也相对比较粗暴。由于TaskManager之间的网络连接是由不同Task复用的,一旦网络处于阻塞状态,所有Task都无法向TCP连接中写入数据或者从中读取数据,即便其它Task关联的缓冲池仍然存在空余。此外,由于网络发生了阻塞,诸如CheckpointBarrier等事件也无法在Task之间进行流转。
在Flink V1.5版之前,其实Flink并没有刻意做上述所说的动态反馈。那么问题来了,没有做上述的动态反馈机制,Flink难道不怕数据丢失或者上游和下游的一些Buffer把内存撑爆吗?当然不怕了,因为Flink已经依赖其他机制来实现了所谓的动态反馈。如下图所示,对于一个Flink任务,动态反馈可以抽象成以下两个阶段:
1、跨Task,动态反馈如何从下游Task的Receive Buffer反馈给上游Task的Send Buffer
2、Task内,动态反馈如何从内部的 Send Buffer 反馈给内部的 Receive Buffer
先了解一下Flink的TaskManager之间网络传输的数据流向:
图中,我们可以看到 TaskManager A 给 TaskManager B 发送数据,TaskManager A 做为 Producer,TaskManager B 做为 Consumer。Producer 端的 Operator 实例会产生数据,最后通过网络发送给 Consumer 端的 Operator 实例。Producer 端 Operator 实例生产的数据首先缓存到 TaskManager 内部的 NetWork Buffer。NetWork 依赖 Netty 来做通信,Producer 端的 Netty 内部有 ChannelOutbound Buffer,Consumer 端的 Netty 内部有 ChannelInbound Buffer。Netty 最终还是要通过 Socket 发送网络请求,Socket 这一层也会有 Buffer,Producer 端有 Send Buffer,Consumer 端有 Receive Buffer。
总结一下,现在有两个 TaskManager A、B,TaskManager A 中 Producer Operator 处理完的数据由 TaskManager B 中 Consumer Operator 处理。那么 Producer Operator 处理完的数据是怎么到达 Consumer Operator 的?首先 Producer Operator 从自己的上游或者外部数据源读取到数据后,对一条条的数据进行处理,处理完的数据首先输出到 Producer Operator 对应的 NetWork Buffer 中。Buffer 写满或者超时后,就会触发将 NetWork Buffer 中的数据拷贝到 Producer 端 Netty 的 ChannelOutbound Buffer,之后又把数据拷贝到 Socket 的 Send Buffer 中,这里有一个从用户态拷贝到内核态的过程,最后通过 Socket 发送网络请求,把 Send Buffer 中的数据发送到 Consumer 端的 Receive Buffer。数据到达 Consumer 端后,再依次从 Socket 的 Receive Buffer 拷贝到 Netty 的 ChannelInbound Buffer,再拷贝到 Consumer Operator 的 NetWork Buffer,最后 Consumer Operator 就可以读到数据进行处理了。这就是两个 TaskManager 之间的数据传输过程,我们可以看到发送方和接收方各有三层的 Buffer。
了解了数据传输流程,我们再具体了解一下跨 TaskManager 的反压过程,如下图所示,Producer 端生产数据速率为 2,Consumer 消费数据速率为 1。持续下去,下游消费较慢,Buffer 容量又是有限的,那 Flink 反压是怎么做的?
上面介绍后,我们知道每个 Operator 计算数据时,输出和输入都有对应的 NetWork Buffer,这个 NetWork Buffer 对应到 Flink 就是图中所示的 ResultSubPartition 和 InputChannel。ResultSubPartition 和 InputChannel 都是向 LocalBufferPool 申请 Buffer 空间,然后 LocalBufferPool 再向 NetWork BufferPool 申请内存空间。这里,NetWork BufferPool 是 TaskManager 内所有 Task 共享的 BufferPool,TaskManager 初始化时就会向堆外内存申请 NetWork BufferPool。LocalBufferPool 是每个 Task 自己的 BufferPool,假如一个 TaskManager 内运行着 5 个 Task,那么就会有 5 个 LocalBufferPool,但 TaskManager 内永远只有一个 NetWork BufferPool。Netty 的 Buffer 也是初始化时直接向堆外内存申请内存空间。虽然可以申请,但是必须明白内存申请肯定是有限制的,不可能无限制的申请,我们在启动任务时可以指定该任务最多可能申请多大的内存空间用于 NetWork Buffer。
继续分析该场景, Producer 端生产数据速率为2,Consumer 端消费数据速率为1。数据从 Task A 的 ResultSubPartition 按照上面的流程最后传输到 Task B 的 InputChannel 供 Task B 读取并计算。持续一段时间后,由于 Task B 消费比较慢,导致 InputChannel 被占满了,所以 InputChannel 向 LocalBufferPool 申请新的 Buffer 空间,LocalBufferPool 分配给 InputChannel 一些 Buffer。
再持续一段时间后,InputChannel 重复向 LocalBufferPool 申请 Buffer 空间,导致 LocalBufferPool 也满了,所以 LocalBufferPool 向 NetWork BufferPool 申请 Buffer 空间,NetWork BufferPool 给 LocalBufferPool 分配 Buffer。
再持续下去,NetWork BufferPool 满了,或者说 NetWork BufferPool 不能把自己的 Buffer 全分配给 Task B 对应的 LocalBufferPool ,因为 TaskManager 上一般会运行了多个 Task,每个 Task 只能使用 NetWork BufferPool 中的一部分。所以,可以认为 Task B 把自己可以使用的 InputChannel 、 LocalBufferPool 和 NetWork BufferPool 都用完了。此时 Netty 还想把数据写入到 InputChannel,但是发现 InputChannel 满了,所以 Socket 层会把 Netty 的 autoRead disable,Netty 不会再从 Socket 中去读消息。可以看到下图中多个 ❌,表示 Buffer 已满,数据已经不能往下游写了,发生了阻塞。
由于 Netty 不从 Socket 的 Receive Buffer 读数据了,所以很快 Socket 的 Receive Buffer 就会变满,TCP 的 Socket 通信有动态反馈的流控机制,会把容量为0的消息反馈给上游发送端,所以上游的 Socket 就不会往下游再发送数据 。
Task A 持续生产数据,发送端 Socket 的 Send Buffer 很快被打满,所以 Task A 端的 Netty 也会停止往 Socket 写数据。
接下来,数据会在 Netty 的 Buffer 中缓存数据,但 Netty 的 Buffer 是无界的。但可以设置 Netty 的高水位,即:设置一个 Netty 中 Buffer 的上限。所以每次 ResultSubPartition 向 Netty 中写数据时,都会检测 Netty 是否已经到达高水位,如果达到高水位就不会再往 Netty 中写数据,防止 Netty 的 Buffer 无限制的增长。
接下来,数据会在 Task A 的 ResultSubPartition 中累积,ResultSubPartition 满了后,会向 LocalBufferPool 申请新的 Buffer 空间,LocalBufferPool 分配给 ResultSubPartition 一些 Buffer。
持续下去 LocalBufferPool 也会用完,LocalBufferPool 再向 NetWork BufferPool 申请 Buffer。
然后 NetWork BufferPool 也会用完,或者说 NetWork BufferPool 不能把自己的 Buffer 全分配给 Task A 对应的 LocalBufferPool ,因为 TaskManager 上一般会运行了多个 Task,每个 Task 只能使用 NetWork BufferPool 中的一部分。此时,Task A 已经申请不到任何的 Buffer 了,Task A 的 Record Writer 输出就被 wait ,Task A 不再生产数据。
通过上述的这个流程,来动态反馈,保障各个 Buffer 都不会因为数据太多导致内存溢出。上面描述了整个阻塞的流程,当下游 Task B 持续消费,Buffer 的可用容量会增加,所有被阻塞的数据通道会被一个个打开,之后 Task A 又可以开始正常的生产数据了。
之前介绍,Task 之间的数据传输可能存在上游的 Task A 和下游的 Task B 运行在同一台节点的情况,整个流程与上述类似,只不过由于 Task A 和 B 运行在同一个 JVM,所以不需要网络传输的环节,Task B 的 InputChannel 会直接从 Task A 的 ResultSubPartition 读取数据。
假如 Task A 的下游所有 Buffer 都占满了,那么 Task A 的 Record Writer 会被 block,Task A 的 Record Reader、Operator、Record Writer 都属于同一个线程,所以 Task A 的 Record Reader 也会被 block。
然后可以把这里的 Task A 类比成上面所说的 Task B,Task A 上游持续高速率发送数据到 Task A 就会导致可用的 InputChannel、 LocalBufferPool 和 NetWork BufferPool 都会被用完。然后 Netty 、Socket 同理将压力传输到 Task A 的上游。
假设 Task A 的上游是 Task X,那么 Task A 将压力反馈给 Task X 的过程与 Task B 将压力反馈给 Task A 的过程是一样的。整个 Flink 的反压是从下游往上游传播的,一直传播到 Source Task,Source Task 有压力后,会降低从外部组件中读取数据的速率,例如:Source Task 会降低从 Kafka 中读取数据的速率,来降低整个 Flink Job 中缓存的数据,从而降低负载。
看着挺完美的反压机制,其实是有问题的。如下图所示,我们的任务有4个 SubTask,SubTask A 是 SubTask B的上游,即 SubTask A 给 SubTask B 发送数据。Job 运行在两个 TaskManager中, TaskManager 1 运行着 SubTask A.1 和 SubTask A.2, TaskManager 2 运行着 SubTask B.3 和 SubTask B.4。现在假如由于CPU共享或者内存紧张或者磁盘IO瓶颈造成 SubTask B.4 遇到瓶颈、处理速率有所下降,但是上游源源不断地生产数据,所以导致 SubTask A.2 与 SubTask B.4 产生反压。
这里需要明确一点:不同 Job 之间的每个(远程)网络连接将在 Flink 的网络堆栈中获得自己的TCP通道。但是,如果同一 Task 的不同 SubTask 被安排到同一个TaskManager,则它们与其他 TaskManager 的网络连接将被多路复用并共享一个TCP信道以减少资源使用。例如,图中的 A.1 -> B.3、A.1 -> B.4、A.2 -> B.3、A.2 -> B.4 这四条将会多路复用共享一个 TCP 信道。
现在 SubTask B.3 并没有压力,从上面跨 TaskManager 的反压流程,我们知道当上图中 SubTask A.2 与 SubTask B.4 产生反压时,会把 TaskManager1 端该任务对应 Socket 的 Send Buffer 和 TaskManager2 端该任务对应 Socket 的 Receive Buffer 占满,多路复用的 TCP 通道已经被占住了,会导致 SubTask A.1 和 SubTask A.2 要发送给 SubTask B.3 的数据全被阻塞了,从而导致本来没有压力的 SubTask B.3 现在接收不到数据了。所以,Flink 1.5 版之前的反压机制会存在当一个 Task 出现反压时,可能导致其他正常的 Task 接收不到数据。
为了解决上述问题,Flink1.5重构了网络栈,引入了“基于信用值的流量控制算法”(Credit-basedFlowControl),确保TaskManager之间的网络连接始终不会处于阻塞状态。Credit-basedFlowControl的思路其实也比较简单,它是在接收端和发送端之间建立一种类似“信用评级”的机制,发送端向接收端发送的数据永远不会超过接收端的信用值的大小。在Flink这里,信用值就是接收端可用的Buffer的数量,这样就可以保证发送端不会向TCP连接中发送超出接收端缓冲区可用容量的数据。相比于之前所有的InputChannel共享同一个本地缓冲池的方式,在重构网络栈之后,Flink会为每一个InputChannel分配一批独占的缓冲(exclusivebuffers),而本地缓冲池中的buffer则作为流动的(floatingbuffers),可以被所有的InputChannel使用。
Credit-basedFlowControl的具体机制为:
基本示例如下:
如下图所示,反压机制作用于 Flink 的应用层,即在 ResultSubPartition 和 InputChannel 这一层引入了反压机制。每次上游 SubTask A.2 给下游 SubTask B.4 发送数据时,会把 Buffer 中的数据和上游 ResultSubPartition 堆积的数据量 Backlog size发给下游,下游会接收上游发来的数据,并向上游反馈目前下游现在的 Credit 值,Credit 值表示目前下游可以接收上游的 Buffer 量,1 个Buffer 等价于 1 个 Credit 。
例如,上游 SubTask A.2 发送完数据后,还有 5 个 Buffer 被积压,那么会把发送数据和 Backlog size = 5 一块发送给下游 SubTask B.4,下游接受到数据后,知道上游积压了 5 个Buffer,于是向 Buffer Pool 申请 Buffer,由于容量有限,下游 InputChannel 目前仅有 2 个 Buffer 空间,所以,SubTask B.4 会向上游 SubTask A.2 反馈 Channel Credit = 2。然后上游下一次最多只给下游发送 2 个 Buffer 的数据,这样每次上游发送的数据都是下游 InputChannel 的 Buffer 可以承受的数据量,所以通过这种反馈策略,保证了不会在公用的 Netty 和 TCP 这一层数据堆积而影响其他 SubTask 通信。
1、初始化:首先,在向NetworkEnvironment注册的时候,会为InputGate分配本地缓冲池,还会为RemoteInputChannel分配独占的buffer:
class NetworkEnvironment {
public void setupInputGate(SingleInputGate gate) throws IOException {
BufferPool bufferPool = null;
int maxNumberOfMemorySegments;
try {
if (config.isCreditBased()) { //使用 Credit-based Flow Control
//本地缓冲池使用的 buffer 数量,如果是 bounded,则缓冲池的大小最大为 taskmanager.network.memory.floating-buffers-per-gate
maxNumberOfMemorySegments = gate.getConsumedPartitionType().isBounded() ?
config.floatingNetworkBuffersPerGate() : Integer.MAX_VALUE;
// assign exclusive buffers to input channels directly and use the rest for floating buffers
// 独占的buffer,不包含在分配的 LocalBufferPool 中
gate.assignExclusiveSegments(networkBufferPool, config.networkBuffersPerChannel());
bufferPool = networkBufferPool.createBufferPool(0, maxNumberOfMemorySegments);
} else {
maxNumberOfMemorySegments = gate.getConsumedPartitionType().isBounded() ?
gate.getNumberOfInputChannels() * config.networkBuffersPerChannel() +
config.floatingNetworkBuffersPerGate() : Integer.MAX_VALUE;
bufferPool = networkBufferPool.createBufferPool(gate.getNumberOfInputChannels(),
maxNumberOfMemorySegments);
}
//分配 LocalBufferPool 本地缓冲池,这是所有 channel 共享的
gate.setBufferPool(bufferPool);
} catch (Throwable t) {
if (bufferPool != null) {
bufferPool.lazyDestroy();
}
ExceptionUtils.rethrowIOException(t);
}
}
}
class SingleInputGate {
public void assignExclusiveSegments(NetworkBufferPool networkBufferPool, int networkBuffersPerChannel) throws IOException {
checkState(this.isCreditBased, "Bug in input gate setup logic: exclusive buffers only exist with credit-based flow control.");
checkState(this.networkBufferPool == null, "Bug in input gate setup logic: global buffer pool has" +
"already been set for this input gate.");
this.networkBufferPool = checkNotNull(networkBufferPool);
this.networkBuffersPerChannel = networkBuffersPerChannel;
synchronized (requestLock) {
for (InputChannel inputChannel : inputChannels.values()) {
if (inputChannel instanceof RemoteInputChannel) {
//RemoteInputChannel 请求独占的 buffer
((RemoteInputChannel) inputChannel).assignExclusiveSegments(
networkBufferPool.requestMemorySegments(networkBuffersPerChannel));
}
}
}
}
}
2、RemoteInputChannel管理可用buffer:在RemoteInputChannel内部使用AvailableBufferQueue来管理所有可用的buffer:
class RemoteInputChannel extends InputChannel implements BufferRecycler, BufferListener {
//可用的 buffer 队列,包含 exclusive + floating
/** The available buffer queue wraps both exclusive and requested floating buffers. */
private final AvailableBufferQueue bufferQueue = new AvailableBufferQueue();
//分配独占的 buffer
void assignExclusiveSegments(List segments) {
this.initialCredit = segments.size();
this.numRequiredBuffers = segments.size();
synchronized (bufferQueue) {
for (MemorySegment segment : segments) {
// 注意这个 NetworkBuffer 的回收器是 RemoteInputChannel 自身
bufferQueue.addExclusiveBuffer(new NetworkBuffer(segment, this), numRequiredBuffers);
}
}
}
// 独占的 buffer 释放后会直接被 RemoteInputChannel 回收
/**
* Exclusive buffer is recycled to this input channel directly and it may trigger return extra
* floating buffer and notify increased credit to the producer.
*
* @param segment The exclusive segment of this channel.
*/
@Override
public void recycle(MemorySegment segment) {
int numAddedBuffers;
synchronized (bufferQueue) {
// Similar to notifyBufferAvailable(), make sure that we never add a buffer
// after releaseAllResources() released all buffers (see below for details).
if (isReleased.get()) { // 如果这个channle已经被释放
try {
// 这个MemorySegment会被归还给NetworkBufferPool
inputGate.returnExclusiveSegments(Collections.singletonList(segment));
return;
} catch (Throwable t) {
ExceptionUtils.rethrow(t);
}
}
// 重新加入到AvailableBufferQueue中
numAddedBuffers = bufferQueue.addExclusiveBuffer(new NetworkBuffer(segment, this), numRequiredBuffers);
}
if (numAddedBuffers > 0 && unannouncedCredit.getAndAdd(numAddedBuffers) == 0) {
notifyCreditAvailable();
}
}
/**
* Manages the exclusive and floating buffers of this channel, and handles the
* internal buffer related logic.
*/
private static class AvailableBufferQueue {
/** The current available floating buffers from the fixed buffer pool. */
private final ArrayDeque floatingBuffers; // 这部分是流动的
/** The current available exclusive buffers from the global buffer pool. */
private final ArrayDeque exclusiveBuffers; // 这部分是独占的
AvailableBufferQueue() {
this.exclusiveBuffers = new ArrayDeque<>();
this.floatingBuffers = new ArrayDeque<>();
}
/**
* Adds an exclusive buffer (back) into the queue and recycles one floating buffer if the
* number of available buffers in queue is more than the required amount.
*
* @param buffer The exclusive buffer to add
* @param numRequiredBuffers The number of required buffers
*
* @return How many buffers were added to the queue
*/
//添加一个独占的buffer,如果当前可用的 buffer 总量超出了要求的数量,则向本地缓冲池归还一个流动的buffer
//返回值是新增的 buffer 数量
int addExclusiveBuffer(Buffer buffer, int numRequiredBuffers) {
exclusiveBuffers.add(buffer);
if (getAvailableBufferSize() > numRequiredBuffers) {
Buffer floatingBuffer = floatingBuffers.poll();
floatingBuffer.recycleBuffer(); //加一个,归还一个,相当于没加
return 0;
} else {
return 1;
}
}
void addFloatingBuffer(Buffer buffer) { //添加一个流动的buffer
floatingBuffers.add(buffer);
}
/**
* Takes the floating buffer first in order to make full use of floating
* buffers reasonably.
*
* @return An available floating or exclusive buffer, may be null
* if the channel is released.
*/
@Nullable
Buffer takeBuffer() { //优先取流动的buffer
if (floatingBuffers.size() > 0) {
return floatingBuffers.poll();
} else {
return exclusiveBuffers.poll();
}
}
int getAvailableBufferSize() {
return floatingBuffers.size() + exclusiveBuffers.size();
}
}
}
3、请求远端子分区:RemoteInputChannel请求远端的ResultSubpartition,会创建一个PartitionRequestClient,并通过Netty发送PartitionRequest请求,这时会带上当前InputChannel的id和初始的credit信息:
class RemoteInputChannel extends InputChannel implements BufferRecycler, BufferListener {
public void requestSubpartition(int subpartitionIndex) throws IOException, InterruptedException {
//REMOTE,需要网络通信,使用 Netty 建立网络
//通过 ConnectionManager 来建立连接:创建 PartitionRequestClient,通过 PartitionRequestClient 发起请求
if (partitionRequestClient == null) {
// Create a client and request the partition
partitionRequestClient = connectionManager.createPartitionRequestClient(connectionId);
//请求分区,通过 netty 发起请求
partitionRequestClient.requestSubpartition(partitionId, subpartitionIndex, this, 0);
}
}
}
public class PartitionRequestClient {
public ChannelFuture requestSubpartition(
final ResultPartitionID partitionId,
final int subpartitionIndex,
final RemoteInputChannel inputChannel,
int delayMs) throws IOException {
//向 NetworkClientHandler 注册当前 RemoteInputChannel
//单个 Task 所有的 RemoteInputChannel 的数据传输都通过这个 PartitionRequestClient 处理
clientHandler.addInputChannel(inputChannel);
//PartitionRequest封装了请求的 sub-partition 的信息,当前 input channel 的 ID,以及初始 credit
final PartitionRequest request = new PartitionRequest(
partitionId, subpartitionIndex, inputChannel.getInputChannelId(), inputChannel.getInitialCredit());
final ChannelFutureListener listener = new ChannelFutureListener() {
@Override
public void operationComplete(ChannelFuture future) throws Exception {
if (!future.isSuccess()) {
//如果请求发送失败,要移除当前的 inputChannel
clientHandler.removeInputChannel(inputChannel);
SocketAddress remoteAddr = future.channel().remoteAddress();
inputChannel.onError(
new LocalTransportException(
String.format("Sending the partition request to '%s' failed.", remoteAddr),
future.channel().localAddress(), future.cause()
));
}
}
};
//通过 netty 发送请求
if (delayMs == 0) {
ChannelFuture f = tcpChannel.writeAndFlush(request);
f.addListener(listener);
return f;
} else {
final ChannelFuture[] f = new ChannelFuture[1];
tcpChannel.eventLoop().schedule(new Runnable() {
@Override
public void run() {
f[0] = tcpChannel.writeAndFlush(request);
f[0].addListener(listener);
}
}, delayMs, TimeUnit.MILLISECONDS);
return f[0];
}
}
}
4、生产端的处理流程
生产者端即ResultSubpartition一侧,在网络通信中对应NettyServer。NettyServer有两个重要的ChannelHandler,即PartitionRequestServerHandler和PartitionRequestQueue。其中,PartitionRequestServerHandler负责处理消费端通过PartitionRequestClient发送的PartitionRequest和AddCredit等请求;PartitionRequestQueue则包含了一个可以从中读取数据的NetworkSequenceViewReader队列,它会监听NettyChannel的可写入状态,一旦可以写入数据,就使用NetworkSequenceViewReader来消费ResultSubpartition写入的Buffer数据,将其写入到NettyChannel。
首先,当NettyServer接收到PartitionRequest消息后,PartitionRequestServerHandler会创建一个NetworkSequenceViewReader对象,请求创建ResultSubpartitionView,并将NetworkSequenceViewReader保存在PartitionRequestQueue中。PartitionRequestQueue会持有所有请求消费数据的RemoteInputChannel的ID和NetworkSequenceViewReader之间的映射关系。
其中ResultSubpartitionView用来消费ResultSubpartition中的数据,并在ResultSubpartition中有数据可用时获得提醒;NetworkSequenceViewReader则相当于对ResultSubpartition的一层包装,她会按顺序为读取的每一个buffer分配一个序列号,并且记录了接收数据的RemoteInputChannel的ID。在使用Credit-basedFlowControl的情况下,NetworkSequenceViewReader的具体实现对应为CreditBasedSequenceNumberingViewReader。CreditBasedSequenceNumberingViewReader同时还实现了BufferAvailabilityListener接口,因而可以作为PipelinedSubpartitionView的回调对象。
class PartitionRequestServerHandler extends SimpleChannelInboundHandler {
@Override
protected void channelRead0(ChannelHandlerContext ctx, NettyMessage msg) throws Exception {
try {
Class> msgClazz = msg.getClass();
if (msgClazz == PartitionRequest.class) {
//Server 端接收到 client 发送的 PartitionRequest
PartitionRequest request = (PartitionRequest) msg;
try {
NetworkSequenceViewReader reader;
if (creditBasedEnabled) {
reader = new CreditBasedSequenceNumberingViewReader(
request.receiverId,
request.credit,
outboundQueue);
} else {
reader = new SequenceNumberingViewReader(
request.receiverId,
outboundQueue);
}
//通过 ResultPartitionProvider(实际上就是 ResultPartitionManager)创建 ResultSubpartitionView
//在有可被消费的数据产生后,PartitionRequestQueue.notifyReaderNonEmpty 会被回调,进而在 netty channelPipeline 上触发一次 fireUserEventTriggered
reader.requestSubpartitionView(
partitionProvider,
request.partitionId,
request.queueIndex);
//通知 PartitionRequestQueue 创建了一个 NetworkSequenceViewReader
outboundQueue.notifyReaderCreated(reader);
} catch (PartitionNotFoundException notFound) {
respondWithError(ctx, notFound, request.receiverId);
}
}
......
} catch (Throwable t) {
respondWithError(ctx, t);
}
}
}
class CreditBasedSequenceNumberingViewReader implements BufferAvailabilityListener, NetworkSequenceViewReader {
private final InputChannelID receiverId; //对应的 RemoteInputChannel 的 ID
private final PartitionRequestQueue requestQueue;
//消费 ResultSubpartition 的数据,并在 ResultSubpartition 有数据可用时获得通知
private volatile ResultSubpartitionView subpartitionView;
//numCreditsAvailable的值是消费端还能够容纳的buffer的数量,也就是允许生产端发送的buffer的数量
private int numCreditsAvailable;
private int sequenceNumber = -1; //序列号,自增
//创建一个 ResultSubpartitionView,用于读取数据,并在有数据可用时获得通知
@Override
public void requestSubpartitionView(
ResultPartitionProvider partitionProvider,
ResultPartitionID resultPartitionId,
int subPartitionIndex) throws IOException {
synchronized (requestLock) {
if (subpartitionView == null) {
this.subpartitionView = partitionProvider.createSubpartitionView(
resultPartitionId,
subPartitionIndex,
this);
} else {
throw new IllegalStateException("Subpartition already requested");
}
}
}
//读取数据
@Override
public BufferAndAvailability getNextBuffer() throws IOException, InterruptedException {
BufferAndBacklog next = subpartitionView.getNextBuffer(); //读取数据
if (next != null) {
sequenceNumber++; //序列号
//要发送一个buffer,对应的 numCreditsAvailable 要减 1
if (next.buffer().isBuffer() && --numCreditsAvailable < 0) {
throw new IllegalStateException("no credit available");
}
return new BufferAndAvailability(
next.buffer(), isAvailable(next), next.buffersInBacklog());
} else {
return null;
}
}
//是否还可以消费数据:
// 1. ResultSubpartition 中有更多的数据
// 2. credit > 0 或者下一条数据是事件(事件不需要消耗credit)
@Override
public boolean isAvailable() {
// BEWARE: this must be in sync with #isAvailable(BufferAndBacklog)!
return hasBuffersAvailable() &&
//要求 numCreditsAvailable > 0 或者是 Event
(numCreditsAvailable > 0 || subpartitionView.nextBufferIsEvent());
}
boolean hasBuffersAvailable() {
return subpartitionView.isAvailable();
}
//和上面 isAvailable() 是等价的
private boolean isAvailable(BufferAndBacklog bufferAndBacklog) {
// BEWARE: this must be in sync with #isAvailable()!
return bufferAndBacklog.isMoreAvailable() &&
(numCreditsAvailable > 0 || bufferAndBacklog.nextBufferIsEvent());
}
//在 ResultSubparition 中有数据时会回调该方法
@Override
public void notifyDataAvailable() {
//告知 PartitionRequestQueue 当前 ViewReader 有数据可读
requestQueue.notifyReaderNonEmpty(this);
}
}
PartitionRequestQueue负责将ResultSubparition中的数据通过网络发送给RemoteInputChannel。在PartitionRequestQueue中保存了所有的NetworkSequenceViewReader和InputChannelID之间的映射关系,以及一个ArrayDeque
class PartitionRequestQueue extends ChannelInboundHandlerAdapter {
/** The readers which are already enqueued available for transferring data. */
private final ArrayDeque availableReaders = new ArrayDeque<>();
/** All the readers created for the consumers' partition requests. */
private final ConcurrentMap allReaders = new ConcurrentHashMap<>();
//添加新的 NetworkSequenceViewReader
public void notifyReaderCreated(final NetworkSequenceViewReader reader) {
allReaders.put(reader.getReceiverId(), reader);
}
//通知 NetworkSequenceViewReader 有数据可读取
void notifyReaderNonEmpty(final NetworkSequenceViewReader reader) {
//触发一次用户自定义事件
ctx.executor().execute(() -> ctx.pipeline().fireUserEventTriggered(reader));
}
//自定义用户事件的处理
@Override
public void userEventTriggered(ChannelHandlerContext ctx, Object msg) throws Exception {
// The user event triggered event loop callback is used for thread-safe
// hand over of reader queues and cancelled producers.
if (msg instanceof NetworkSequenceViewReader) {
//NetworkSequenceViewReader有数据可读取,加入队列中
enqueueAvailableReader((NetworkSequenceViewReader) msg);
} else if (msg.getClass() == InputChannelID.class) {
// 对应的 RemoteInputChannel 请求取消消费
// Release partition view that get a cancel request.
InputChannelID toCancel = (InputChannelID) msg;
if (released.contains(toCancel)) {
return;
}
// Cancel the request for the input channel
int size = availableReaders.size();
for (int i = 0; i < size; i++) {
NetworkSequenceViewReader reader = pollAvailableReader();
if (reader.getReceiverId().equals(toCancel)) {
reader.releaseAllResources();
markAsReleased(reader.getReceiverId());
} else {
registerAvailableReader(reader);
}
}
allReaders.remove(toCancel);
} else {
ctx.fireUserEventTriggered(msg);
}
}
//加入队列
private void enqueueAvailableReader(final NetworkSequenceViewReader reader) throws Exception {
if (reader.isRegisteredAsAvailable() || !reader.isAvailable()) {
//已经被注册到队列中,或者暂时没有 buffer 或没有 credit 可用
return;
}
boolean triggerWrite = availableReaders.isEmpty();
registerAvailableReader(reader);
if (triggerWrite) {
//如果这是队列中第一个元素,调用 writeAndFlushNextMessageIfPossible 发送数据
writeAndFlushNextMessageIfPossible(ctx.channel());
}
}
}
PartitionRequestQueue会监听NettyChannel的可写入状态,当Channel可写入时,就会从availableReaders队列中取出NetworkSequenceViewReader,读取数据并写入网络。可写入状态是Netty通过水位线进行控制的,NettyServer在启动的时候会配置水位线,如果Netty输出缓冲中的字节数超过了高水位值,我们会等到其降到低水位值以下才继续写入数据。通过水位线机制确保不往网络中写入太多数据。
class PartitionRequestQueue extends ChannelInboundHandlerAdapter {
@Override
public void channelWritabilityChanged(ChannelHandlerContext ctx) throws Exception {
//当前channel的读写状态发生变化
writeAndFlushNextMessageIfPossible(ctx.channel());
}
private void writeAndFlushNextMessageIfPossible(final Channel channel) throws IOException {
if (fatalError || !channel.isWritable()) {
//如果当前不可写入,则直接返回
return;
}
BufferAndAvailability next = null;
try {
while (true) {
//取出一个 reader
NetworkSequenceViewReader reader = pollAvailableReader();
if (reader == null) {
return;
}
next = reader.getNextBuffer();
if (next == null) {
//没有读到数据
if (!reader.isReleased()) {
//还没有释放当前 subpartition,继续处理下一个 reader
continue;
}
markAsReleased(reader.getReceiverId());
//出错了
Throwable cause = reader.getFailureCause();
if (cause != null) {
ErrorResponse msg = new ErrorResponse(
new ProducerFailedException(cause),
reader.getReceiverId());
ctx.writeAndFlush(msg);
}
} else {
// 读到了数据
if (next.moreAvailable()) {
//这个 reader 还可以读到更多的数据,继续加入队列
registerAvailableReader(reader);
}
BufferResponse msg = new BufferResponse(
next.buffer(),
reader.getSequenceNumber(),
reader.getReceiverId(),
next.buffersInBacklog());
// 向 client 发送数据,发送成功之后通过 writeListener 的回调触发下一次发送
channel.writeAndFlush(msg).addListener(writeListener);
return;
}
}
} catch (Throwable t) {
if (next != null) {
next.buffer().recycleBuffer();
}
throw new IOException(t.getMessage(), t);
}
}
private class WriteAndFlushNextMessageIfPossibleListener implements ChannelFutureListener {
@Override
public void operationComplete(ChannelFuture future) throws Exception {
try {
if (future.isSuccess()) {
//发送成功,再次尝试写入
writeAndFlushNextMessageIfPossible(future.channel());
} else if (future.cause() != null) {
handleException(future.channel(), future.cause());
} else {
handleException(future.channel(), new IllegalStateException("Sending cancelled by user."));
}
} catch (Throwable t) {
handleException(future.channel(), t);
}
}
}
}
在Credit-basedFlowControl算法中,每发送一个buffer就会消耗一点credit,在消费端有空闲buffer可用时会发送AddCrdit消息。
class PartitionRequestServerHandler extends SimpleChannelInboundHandler {
@Override
protected void channelRead0(ChannelHandlerContext ctx, NettyMessage msg) throws Exception {
......
if (msgClazz == AddCredit.class) {
//增加 credit
AddCredit request = (AddCredit) msg;
outboundQueue.addCredit(request.receiverId, request.credit);
}
......
}
}
class RequestReaderQueue extends ChannelInboundHandlerAdapter {
void addCredit(InputChannelID receiverId, int credit) throws Exception {
if (fatalError) {
return;
}
NetworkSequenceViewReader reader = allReaders.get(receiverId);
if (reader != null) {
//增加 credit
reader.addCredit(credit);
//因为增加了credit,可能可以继续处理数据,因此把 reader 加入队列
enqueueAvailableReader(reader);
} else {
throw new IllegalStateException("No reader for receiverId = " + receiverId + " exists.");
}
}
}
5、消费端处理流程:消费端即RemoteInputChannel一侧,在网络通信中对应NettyClient。同样地,我们从ChannelHandler作为入口进行分析。
public interface NetworkClientHandler extends ChannelHandler {
void addInputChannel(RemoteInputChannel inputChannel) throws IOException;
void removeInputChannel(RemoteInputChannel inputChannel);
void cancelRequestFor(InputChannelID inputChannelId);
//通知有新的的 credit 可用
void notifyCreditAvailable(final RemoteInputChannel inputChannel);
}
NetworkClientHanlder对应的实现类为CreditBasedPartitionRequestClientHandler,CreditBasedPartitionRequestClientHandler负责接收服务端通过Nettychannel发送的数据,解析数据后交给对应的RemoteInputChannle进行处理:
class CreditBasedPartitionRequestClientHandler extends ChannelInboundHandlerAdapter implements NetworkClientHandler {
@Override
public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
//从netty channel中接收到数据
try {
//解析消息
decodeMsg(msg);
} catch (Throwable t) {
notifyAllChannelsOfErrorAndClose(t);
}
}
private void decodeMsg(Object msg) throws Throwable {
final Class> msgClazz = msg.getClass();
// ---- Buffer --------------------------------------------------------
if (msgClazz == NettyMessage.BufferResponse.class) {
//正常的数据
NettyMessage.BufferResponse bufferOrEvent = (NettyMessage.BufferResponse) msg;
//根据 ID 定位到对应的 RemoteInputChannel
RemoteInputChannel inputChannel = inputChannels.get(bufferOrEvent.receiverId);
if (inputChannel == null) {
//如果没有对应的 RemoteInputChannel
bufferOrEvent.releaseBuffer();
//取消对给定 receiverId 的订阅
cancelRequestFor(bufferOrEvent.receiverId);
return;
}
//解析消息,是buffer还是event
decodeBufferOrEvent(inputChannel, bufferOrEvent);
} else if (msgClazz == NettyMessage.ErrorResponse.class) {
// ---- Error ---------------------------------------------------------
......
} else {
throw new IllegalStateException("Received unknown message from producer: " + msg.getClass());
}
}
private void decodeBufferOrEvent(RemoteInputChannel inputChannel, NettyMessage.BufferResponse bufferOrEvent) throws Throwable {
try {
ByteBuf nettyBuffer = bufferOrEvent.getNettyBuffer();
final int receivedSize = nettyBuffer.readableBytes();
if (bufferOrEvent.isBuffer()) {
// ---- Buffer ------------------------------------------------
// Early return for empty buffers. Otherwise Netty's readBytes() throws an
// IndexOutOfBoundsException.
if (receivedSize == 0) {
inputChannel.onEmptyBuffer(bufferOrEvent.sequenceNumber, bufferOrEvent.backlog);
return;
}
//从对应的 RemoteInputChannel 中请求一个 Buffer
Buffer buffer = inputChannel.requestBuffer();
if (buffer != null) {
//将接收的数据写入buffer
nettyBuffer.readBytes(buffer.asByteBuf(), receivedSize);
//通知对应的channel,backlog是生产者那边堆积的buffer数量
inputChannel.onBuffer(buffer, bufferOrEvent.sequenceNumber, bufferOrEvent.backlog);
} else if (inputChannel.isReleased()) {
cancelRequestFor(bufferOrEvent.receiverId);
} else {
throw new IllegalStateException("No buffer available in credit-based input channel.");
}
} else {
// ---- Event -------------------------------------------------
// TODO We can just keep the serialized data in the Netty buffer and release it later at the reader
byte[] byteArray = new byte[receivedSize];
nettyBuffer.readBytes(byteArray);
MemorySegment memSeg = MemorySegmentFactory.wrap(byteArray);
//是一个事件,不需要从 RemoteInputChannel 中申请 buffer
Buffer buffer = new NetworkBuffer(memSeg, FreeingBufferRecycler.INSTANCE, false, receivedSize);
//通知对应的channel,backlog是生产者那边堆积的buffer数量
inputChannel.onBuffer(buffer, bufferOrEvent.sequenceNumber, bufferOrEvent.backlog);
}
} finally {
bufferOrEvent.releaseBuffer();
}
}
}
CreditBasedPartitionRequestClientHandler从网络中读取数据后交给RemoteInputChannel,RemoteInputChannel会将接收到的加入队列中,并根据生产端的堆积申请floatingbuffer:
public class RemoteInputChannel extends InputChannel implements BufferRecycler, BufferListener {
//接收到远程 ResultSubpartition 发送的 Buffer
public void onBuffer(Buffer buffer, int sequenceNumber, int backlog) throws IOException {
boolean recycleBuffer = true;
try {
final boolean wasEmpty;
synchronized (receivedBuffers) {
if (isReleased.get()) {
return;
}
//序号需要匹配
if (expectedSequenceNumber != sequenceNumber) {
onError(new BufferReorderingException(expectedSequenceNumber, sequenceNumber));
return;
}
//加入 receivedBuffers 队列中
wasEmpty = receivedBuffers.isEmpty();
receivedBuffers.add(buffer);
recycleBuffer = false;
}
++expectedSequenceNumber;
if (wasEmpty) {
//通知 InputGate,当前 channel 有新数据
notifyChannelNonEmpty();
}
if (backlog >= 0) {
//根据客户端的积压申请float buffer
onSenderBacklog(backlog);
}
} finally {
if (recycleBuffer) {
buffer.recycleBuffer();
}
}
}
//backlog 是发送端的堆积 的 buffer 数量,
//如果 bufferQueue 中 buffer 的数量不足,就去须从 LocalBufferPool 中请求 floating buffer
//在请求了新的 buffer 后,通知生产者有 credit 可用
void onSenderBacklog(int backlog) throws IOException {
int numRequestedBuffers = 0;
synchronized (bufferQueue) {
// Similar to notifyBufferAvailable(), make sure that we never add a buffer
// after releaseAllResources() released all buffers (see above for details).
if (isReleased.get()) {
return;
}
//需要的 buffer 数量是 backlog + initialCredit, backlog 是生产者当前的积压
numRequiredBuffers = backlog + initialCredit;
while (bufferQueue.getAvailableBufferSize() < numRequiredBuffers && !isWaitingForFloatingBuffers) {
//不停地请求新的 floating buffer
Buffer buffer = inputGate.getBufferPool().requestBuffer();
if (buffer != null) {
//从 buffer poll 中请求到 buffer
bufferQueue.addFloatingBuffer(buffer);
numRequestedBuffers++;
} else if (inputGate.getBufferProvider().addBufferListener(this)) {
// buffer pool 没有 buffer 了,加一个监听,当 LocalBufferPool 中有新的 buffer 时会回调 notifyBufferAvailable
// If the channel has not got enough buffers, register it as listener to wait for more floating buffers.
isWaitingForFloatingBuffers = true;
break;
}
}
}
if (numRequestedBuffers > 0 && unannouncedCredit.getAndAdd(numRequestedBuffers) == 0) {
//请求了新的floating buffer,要更新 credit
notifyCreditAvailable();
}
}
private void notifyCreditAvailable() {
checkState(partitionRequestClient != null, "Tried to send task event to producer before requesting a queue.");
//通知当前 channel 有新的 credit
partitionRequestClient.notifyCreditAvailable(this);
}
//LocalBufferPool 通知有 buffer 可用
@Override
public NotificationResult notifyBufferAvailable(Buffer buffer) {
NotificationResult notificationResult = NotificationResult.BUFFER_NOT_USED;
try {
synchronized (bufferQueue) {
checkState(isWaitingForFloatingBuffers,
"This channel should be waiting for floating buffers.");
// Important: make sure that we never add a buffer after releaseAllResources()
// released all buffers. Following scenarios exist:
// 1) releaseAllResources() already released buffers inside bufferQueue
// -> then isReleased is set correctly
// 2) releaseAllResources() did not yet release buffers from bufferQueue
// -> we may or may not have set isReleased yet but will always wait for the
// lock on bufferQueue to release buffers
if (isReleased.get() || bufferQueue.getAvailableBufferSize() >= numRequiredBuffers) {
isWaitingForFloatingBuffers = false;
return notificationResult;
}
//增加floating buffer
bufferQueue.addFloatingBuffer(buffer);
if (bufferQueue.getAvailableBufferSize() == numRequiredBuffers) {
//bufferQueue中有足够多的 buffer 了
isWaitingForFloatingBuffers = false;
notificationResult = NotificationResult.BUFFER_USED_NO_NEED_MORE;
} else {
//bufferQueue 中 buffer 仍然不足
notificationResult = NotificationResult.BUFFER_USED_NEED_MORE;
}
}
if (unannouncedCredit.getAndAdd(1) == 0) {
notifyCreditAvailable();
}
} catch (Throwable t) {
setError(t);
}
return notificationResult;
}
}
一旦RemoteInputChannel申请到新的buffer,就需要通知生产者更新credit,这需要发送一条AddCredit消息:
class PartitionRequestClient {
//交给 NetworkClientHandler 处理
public void notifyCreditAvailable(RemoteInputChannel inputChannel) {
clientHandler.notifyCreditAvailable(inputChannel);
}
}
class CreditBasedPartitionRequestClientHandler extends ChannelInboundHandlerAdapter implements NetworkClientHandler {
//有新的credit
@Override
public void notifyCreditAvailable(final RemoteInputChannel inputChannel) {
//触发一次自定义事件
ctx.executor().execute(() -> ctx.pipeline().fireUserEventTriggered(inputChannel));
}
@Override
public void channelWritabilityChanged(ChannelHandlerContext ctx) throws Exception {
writeAndFlushNextMessageIfPossible(ctx.channel());
}
@Override
public void userEventTriggered(ChannelHandlerContext ctx, Object msg) throws Exception {
if (msg instanceof RemoteInputChannel) {
//有新的credit会触发
boolean triggerWrite = inputChannelsWithCredit.isEmpty();
//加入到队列中
inputChannelsWithCredit.add((RemoteInputChannel) msg);
if (triggerWrite) {
writeAndFlushNextMessageIfPossible(ctx.channel());
}
} else {
ctx.fireUserEventTriggered(msg);
}
}
private void writeAndFlushNextMessageIfPossible(Channel channel) {
if (channelError.get() != null || !channel.isWritable()) {
return;
}
//从队列中取出 RemoteInputChannel, 发送消息
while (true) {
RemoteInputChannel inputChannel = inputChannelsWithCredit.poll();
// The input channel may be null because of the write callbacks
// that are executed after each write.
if (inputChannel == null) {
return;
}
//It is no need to notify credit for the released channel.
if (!inputChannel.isReleased()) {
//发送 AddCredit 的消息
AddCredit msg = new AddCredit(
inputChannel.getPartitionId(),
inputChannel.getAndResetUnannouncedCredit(), //获取并重置新增的credit
inputChannel.getInputChannelId());
// Write and flush and wait until this is done before
// trying to continue with the next input channel.
channel.writeAndFlush(msg).addListener(writeListener);
return;
}
}
}
private class WriteAndFlushNextMessageIfPossibleListener implements ChannelFutureListener {
@Override
public void operationComplete(ChannelFuture future) throws Exception {
try {
if (future.isSuccess()) {
writeAndFlushNextMessageIfPossible(future.channel());
} else if (future.cause() != null) {
notifyAllChannelsOfErrorAndClose(future.cause());
} else {
notifyAllChannelsOfErrorAndClose(new IllegalStateException("Sending cancelled by user."));
}
} catch (Throwable t) {
notifyAllChannelsOfErrorAndClose(t);
}
}
}
}