背景
本文紧接上篇,将介绍在Task现场启动之后,数据是如何从网络中读取转换处理的。本文主要从源码角度一步步分析StreamTask中数据流转过程,其中涉及到上下游netty的交互、数据的反压和checkpoint机制会在接下来的篇章详细讲解。
StreamTask整体数据流
StreamTask有多种实现,本文主要以OneInputStreamTask为例进行介绍。
public abstract class StreamTask<OUT, OP extends StreamOperator<OUT>>
extends AbstractInvokable
implements AsyncExceptionHandler {
//mailboxProcessor的默认行为
protected void processInput(MailboxDefaultAction.Controller controller) throws Exception {
//通过StreamInputProcessor读取数据
InputStatus status = inputProcessor.processInput();
if (status == InputStatus.MORE_AVAILABLE && recordWriter.isAvailable()) {
return;
}
if (status == InputStatus.END_OF_INPUT) {
controller.allActionsCompleted();
return;
}
CompletableFuture<?> jointFuture = getInputOutputJointFuture(status);
MailboxDefaultAction.Suspension suspendedDefaultAction = controller.suspendDefaultAction();
jointFuture.thenRun(suspendedDefaultAction::resume);
}
}
Executor在启动Task线程之后,会反射调用StreamTask的invoke方法,然后mailboxProcessor会循环处理Mail和数据(上文以及详细介绍过了)。其中数据通过StreamInputProcessor进行读取和处理,处理流程大概如下图,内部主要由StreamTaskInput、DataOutput和OperatorChain三个组件。
StreamTaskInput负责从内部的InputChannel(对netty的channel的封装)读取和缓存数据。
DataOutput负责汇总上游StreamTaskInput读取的数据,然后输入给OperatorChain的headOperator。
OperatorChain封装了所有的算子逻辑以及RecordWriter(对数据分区缓存,等待下游拉取)
StreamInputProcessor
该类的主要方法就是processInput(),通过StreamTaskInput从网络中读取并缓存数据,然后发生给DataOutput。
public final class StreamOneInputProcessor<IN> implements StreamInputProcessor {
private final StreamTaskInput<IN> input;
private final DataOutput<IN> output;
private final OperatorChain<?, ?> operatorChain;
@Override
public InputStatus processInput() throws Exception {
InputStatus status = input.emitNext(output);
if (status == InputStatus.END_OF_INPUT) {
synchronized (lock) {
operatorChain.endHeadOperatorInput(1);
}
}
return status;
}
}
StreamTaskInput
StreamTaskInput通过emitNext从checkpointedInputGate(具有barrier对齐功能的InputGate)中读取数据,并生成新的水位线并对齐barrier。
public final class StreamTaskNetworkInput<T> implements StreamTaskInput<T> {
@Override
public InputStatus emitNext(DataOutput<T> output) throws Exception {
while (true) {
// get the stream element from the deserializer
if (currentRecordDeserializer != null) {
DeserializationResult result = currentRecordDeserializer.getNextRecord(deserializationDelegate);
if (result.isBufferConsumed()) {
currentRecordDeserializer.getCurrentBuffer().recycleBuffer();
currentRecordDeserializer = null;
}
if (result.isFullRecord()) {
//数据读取完整之后,发送给DataOutput,
//并生产新的水位线(内部有channels数组,保存所有channel的水位线,并将最小的水位线发生到下游)
processElement(deserializationDelegate.getInstance(), output);
return InputStatus.MORE_AVAILABLE;
}
}
//从checkpointedInputGate中读取数据,同时会处理barrier对齐,并出去checkpoint操作
Optional<BufferOrEvent> bufferOrEvent = checkpointedInputGate.pollNext();
if (bufferOrEvent.isPresent()) {
//InputGate中读取的数据会存储在currentRecordDeserializer中
processBufferOrEvent(bufferOrEvent.get());
} else {
if (checkpointedInputGate.isFinished()) {
checkState(checkpointedInputGate.getAvailableFuture().isDone(), "Finished BarrierHandler should be available");
if (!checkpointedInputGate.isEmpty()) {
throw new IllegalStateException("Trailing data in checkpoint barrier handler.");
}
return InputStatus.END_OF_INPUT;
}
return InputStatus.NOTHING_AVAILABLE;
}
}
}
}
StreamTaskInput中有一个StatusWatermarkValve 对象维护各个channel的水位线,并在读取数据后向下游发送新的水位线
public class StatusWatermarkValve {
public void inputWatermark(Watermark watermark, int channelIndex) throws Exception {
// ignore the input watermark if its input channel, or all input channels are idle (i.e. overall the valve is idle).
if (lastOutputStreamStatus.isActive() && channelStatuses[channelIndex].streamStatus.isActive()) {
long watermarkMillis = watermark.getTimestamp();
// 判断当前channel的水位线是否需要更新
if (watermarkMillis > channelStatuses[channelIndex].watermark) {
channelStatuses[channelIndex].watermark = watermarkMillis;
// previously unaligned input channels are now aligned if its watermark has caught up
if (!channelStatuses[channelIndex].isWatermarkAligned && watermarkMillis >= lastOutputWatermark) {
channelStatuses[channelIndex].isWatermarkAligned = true;
}
// 从所有channels选择最小的水位线发生到下游
findAndOutputNewMinWatermarkAcrossAlignedChannels();
}
}
}
private void findAndOutputNewMinWatermarkAcrossAlignedChannels() throws Exception {
long newMinWatermark = Long.MAX_VALUE;
boolean hasAlignedChannels = false;
// 所有channels中最小的水位线
for (InputChannelStatus channelStatus : channelStatuses) {
if (channelStatus.isWatermarkAligned) {
hasAlignedChannels = true;
newMinWatermark = Math.min(channelStatus.watermark, newMinWatermark);
}
}
// 将水位线发生给DataOutPut
if (hasAlignedChannels && newMinWatermark > lastOutputWatermark) {
lastOutputWatermark = newMinWatermark;
output.emitWatermark(new Watermark(lastOutputWatermark));
}
}
}
StreamTaskInput中的checkpointedInputGate是对InputGate封装后的具有barrier对齐并执行checkpoint的对象。主要功能是从InputGate中读取并缓存数据,并执行checkpoint操作。barrier对齐和checkpoint操作通过内部的barrierHandler对象执行。
public class CheckpointedInputGate implements PullingAsyncDataInput<BufferOrEvent> {
/** InputGate从网络中读取并缓存的数据. */
private final InputGate inputGate;
@Override
public Optional<BufferOrEvent> pollNext() throws Exception {
while (true) {
// process buffered BufferOrEvents before grabbing new ones
Optional<BufferOrEvent> next;
if (bufferStorage.isEmpty()) {
//从InputGate的缓存队列中读取数据
next = inputGate.pollNext();
}
else {
// TODO: FLINK-12536 for non credit-based flow control, getNext method is blocking
next = bufferStorage.pollNext();
if (!next.isPresent()) {
return pollNext();
}
}
if (!next.isPresent()) {
return handleEmptyBuffer();
}
BufferOrEvent bufferOrEvent = next.get();
if (barrierHandler.isBlocked(offsetChannelIndex(bufferOrEvent.getChannelIndex()))) {
// 如果该channel是阻塞状态(既barrier以及到达),则缓存bufferOrEvent
bufferStorage.add(bufferOrEvent);
if (bufferStorage.isFull()) {
//如果缓存区已满,则此次checkpoint失败,放弃此次checkpoint
barrierHandler.checkpointSizeLimitExceeded(bufferStorage.getMaxBufferedBytes());
bufferStorage.rollOver();
}
}
else if (bufferOrEvent.isBuffer()) {
//如果是数据,则不处理 直接返回
return next;
}
else if (bufferOrEvent.getEvent().getClass() == CheckpointBarrier.class) {
CheckpointBarrier checkpointBarrier = (CheckpointBarrier) bufferOrEvent.getEvent();
if (!endOfInputGate) {
// barrierHandler执行barrier对齐,并处理checkpoint
if (barrierHandler.processBarrier(checkpointBarrier, offsetChannelIndex(bufferOrEvent.getChannelIndex()), bufferStorage.getPendingBytes())) {
bufferStorage.rollOver();
}
}
}
else if (bufferOrEvent.getEvent().getClass() == CancelCheckpointMarker.class) {
if (barrierHandler.processCancellationBarrier((CancelCheckpointMarker) bufferOrEvent.getEvent())) {
bufferStorage.rollOver();
}
}
else {
if (bufferOrEvent.getEvent().getClass() == EndOfPartitionEvent.class) {
if (barrierHandler.processEndOfPartition()) {
bufferStorage.rollOver();
}
}
return next;
}
}
}
接下来我们详细看下,CheckpointBarrierHandler 是如何进行barrier对齐并触发checkpoint的。
其实和生成watermark的方式类似,CheckpointBarrierHandler 使用一个channels数组,记录那些channel的barrier已经到达(需要缓存channel中的数据)。当barrier对齐后回调StreamTask的triggerCheckpointOnBarrier方法执行checkpoint操作。
public class CheckpointBarrierAligner extends CheckpointBarrierHandler {
@Override
public boolean processBarrier(CheckpointBarrier receivedBarrier, int channelIndex, long bufferedBytes) throws Exception {
final long barrierId = receivedBarrier.getId();
// 如果只有一个channel,则不如要对对齐barrier,直接触发checkpoint
if (totalNumberOfInputChannels == 1) {
if (barrierId > currentCheckpointId) {
// 生产新的checkpointId
currentCheckpointId = barrierId;
//该方法会回调StreamTask中的triggerCheckpointOnBarrier方法。
notifyCheckpoint(receivedBarrier, bufferedBytes, latestAlignmentDurationNanos);
}
return false;
}
boolean checkpointAborted = false;
//多个channels
if (numBarriersReceived > 0) {
// 执行的是当前正在对齐barrier
if (barrierId == currentCheckpointId) {
// 进行barrier对齐
onBarrier(channelIndex);
}
else if (barrierId > currentCheckpointId) {
// 当前checkpoint还未完成,下一个checkpoint就开始了,放弃当前ck
LOG.warn("{}: Received checkpoint barrier for checkpoint {} before completing current checkpoint {}. " +
"Skipping current checkpoint.",
taskName,
barrierId,
currentCheckpointId);
notifyAbort(currentCheckpointId,
new CheckpointException(
"Barrier id: " + barrierId,
CheckpointFailureReason.CHECKPOINT_DECLINED_SUBSUMED));
// 重置blockChannels数组
releaseBlocksAndResetBarriers();
checkpointAborted = true;
// 发起新的checkpoint
beginNewAlignment(barrierId, channelIndex);
}
else {
// 忽略之前迟到的barrier
return false;
}
}
else if (barrierId > currentCheckpointId) {
// 新的ck的barrier,开启新的barrier对齐
beginNewAlignment(barrierId, channelIndex);
}
else {
return false;
}
// barrier全部对齐,触发StreamTask中的checkpoint操作
if (numBarriersReceived + numClosedChannels == totalNumberOfInputChannels) {
// actually trigger checkpoint
if (LOG.isDebugEnabled()) {
LOG.debug("{}: Received all barriers, triggering checkpoint {} at {}.",
taskName,
receivedBarrier.getId(),
receivedBarrier.getTimestamp());
}
// 重置blockChannels数组
releaseBlocksAndResetBarriers();
notifyCheckpoint(receivedBarrier, bufferedBytes, latestAlignmentDurationNanos);
return true;
}
return checkpointAborted;
}
//barrier对齐的方法,blockedChannels保存所有barrier到达的需要阻塞的channel。
protected void onBarrier(int channelIndex) throws IOException {
if (!blockedChannels[channelIndex]) {
blockedChannels[channelIndex] = true;
numBarriersReceived++;
if (LOG.isDebugEnabled()) {
LOG.debug("{}: Received barrier from channel {}.", taskName, channelIndex);
}
}
else {
throw new IOException("Stream corrupt: Repeated barrier for same checkpoint on input " + channelIndex);
}
}
checkpointedInputGate是通过InputGate的pollNext方法读取缓存队列中的数据,接下来我们详细分析一下。
通过分析源码我们发现InputGate将所有有数据缓存的channel保存在队列中,然后读取channel队列中channel的数据。
public class SingleInputGate extends InputGate {
private final Map<IntermediateResultPartitionID, InputChannel> inputChannels;
//保存有数据的channel
private final ArrayDeque<InputChannel> inputChannelsWithData = new ArrayDeque<>();
@Override
public Optional<BufferOrEvent> pollNext() throws IOException, InterruptedException {
return getNextBufferOrEvent(false);
}
private Optional<BufferOrEvent> getNextBufferOrEvent(boolean blocking) throws IOException, InterruptedException {
//...
//从inputChannelsWithData 队列中获取数据
Optional<InputWithData<InputChannel, BufferAndAvailability>> next = waitAndGetNextData(blocking);
if (!next.isPresent()) {
return Optional.empty();
}
InputWithData<InputChannel, BufferAndAvailability> inputWithData = next.get();
return Optional.of(transformToBufferOrEvent(
inputWithData.data.buffer(),
inputWithData.moreAvailable,
inputWithData.input));
}
private Optional<InputWithData<InputChannel, BufferAndAvailability>> waitAndGetNextData(boolean blocking)
throws IOException, InterruptedException {
while (true) {
//从inputChannelsWithData 队列中获取一个有数据的channel
Optional<InputChannel> inputChannel = getChannel(blocking);
if (!inputChannel.isPresent()) {
return Optional.empty();
}
Optional<BufferAndAvailability> result = inputChannel.get().getNextBuffer();
//...
}
}
接下来我们详细分析一下InputChannel是如何从网络中读取数据的。
InputChannel读取网络数据流程
InputChannel读取的网络数据缓存在receivedBuffers队列中。
public class RemoteInputChannel extends InputChannel implements BufferRecycler, BufferListener {
//缓存buffer数据
private final ArrayDeque<Buffer> receivedBuffers = new ArrayDeque<>();
//上游分区的连接(封装了netty的channel,用于上游分区的server进行通信)
private volatile PartitionRequestClient partitionRequestClient;
@Override
Optional<BufferAndAvailability> getNextBuffer() throws IOException {
checkState(!isReleased.get(), "Queried for a buffer after channel has been closed.");
checkState(partitionRequestClient != null, "Queried for a buffer before requesting a queue.");
checkError();
final Buffer next;
final boolean moreAvailable;
synchronized (receivedBuffers) {
//从receivedBuffers队列中读取数据
next = receivedBuffers.poll();
moreAvailable = !receivedBuffers.isEmpty();
}
numBytesIn.inc(next.getSize());
numBuffersIn.inc();
return Optional.of(new BufferAndAvailability(next, moreAvailable, getSenderBacklog()));
}
}
InputChannel和上游subpartition交互的详细流程将在下一章flink中的网络数据传输流程讲解
RecordWriterOutput
OperatorChain处理完成的数据会通过RecordWriterOutput组件发生到网络中,供下游的Task使用。RecordWriterOutput通过RecordWriter将数据分区写到ResultPartition的ResultSubPartition中,ResultSubPartition将数据缓存起来等待下游Task消费(PipelinedSubpartitionView消费BufferConsumer)。
StreamTask中的每个输出边都有有一个RecordWriterOutput组件写出数据。
RecordWriter写出数据
下图展示了RecordWriter是如何将数据序列化之后,写到subPartition的缓存队列中,等待下游消费的过程。
- RecordWriter通过内部的RecordSerializer将StreamRecord序列化为ByteBuffer类型(为了方便快速的操作byte[])
- RecordWriter向ResultPartition申请BufferBuilder(同时会生成一个BufferConsumer加入到subResultPartition的队列中),用于暂时缓存并构造BufferConsumer(BufferBuilder和BufferConsumer内部使用的是相同的一个MemorySegment)。ResultPartition向LocalBufferPool申请MemorySegment(内部使用byte[]缓存数据),用于存储Buffer数据。
- RecordWriter将RecordSerializer中序列化后的ByteBuffer类型数据,不断写入到BufferBuilder中。当BufferBuilder满了之后,会申请新的BufferBuilder。
- BufferBuilder满了之后,并通知viewRead读取BufferConsumer。
- 当BufferConsumer中的数据被推送到网络中后,回收MemorySegment的内存。
Tips:BufferBuilder和BufferConsumer的设计是为了实现buffer数据的读写分离。线程安全的角度来讲,BufferBuilder是线程安全的,BufferConsumer是线程不安全的。Buffer数据只能通过一个BufferBuilder进行构造,但是可以通过多个线程消费BufferConsumer中的Buffer数据,如此读写分离实现了高效的线程访问。
RecordWriter内部主要有ResultPartitionWriter (提供了ResultPartition的接口,将数据写入ResultPartition)和RecordSerializer(将数据序列化为byte[],封装为NIO的ByteBuffer类型)两个个组件
public class ChannelSelectorRecordWriter<T extends IOReadableWritable> implements AvailabilityProvider {
protected final ResultPartitionWriter targetPartition;
protected final RecordSerializer<T> serializer;
//每个channel对应一个BufferBuilder,当该channel的BufferBuilder满了之后,
//再重新向ResultPartition的LocalBufferPool申请
private final BufferBuilder[] bufferBuilders;
private final ChannelSelector<T> channelSelector;
@Override
//1
public void emit(T record) throws IOException, InterruptedException {
//将record发生到具体的subResultPartition中
emit(record, channelSelector.selectChannel(record));
}
//2
protected void emit(T record, int targetChannel) throws IOException, InterruptedException {
checkErroneous();
//RecordSerializer将record的数据序列化后,保存在内部的ByteBuffer类型变量中
serializer.serializeRecord(record);
// 将RecordSerializer中的ByteBuffer缓存数据写入到ResultPartition的BufferBuilder中
//当BufferBuilder存满之后,构造BufferConsumer,并刷到对应的SubResultPartition的BufferConsumer队列中
if (copyFromSerializerToTargetChannel(targetChannel)) {
serializer.prune();
}
}
//3
protected boolean copyFromSerializerToTargetChannel(int targetChannel) throws IOException, InterruptedException {
serializer.reset();
boolean pruneTriggered = false;
//先获取本地的BufferBuilder,如没有再向ResultPartition申请新的BufferBuilder
BufferBuilder bufferBuilder = getBufferBuilder(targetChannel);
//将RecordSerializer内部以ByteBuffer类型保存的数据,复制到BufferBuilder内部的memorySegment对象的byte[]中
SerializationResult result = serializer.copyToBufferBuilder(bufferBuilder);
while (result.isFullBuffer()) {
finishBufferBuilder(bufferBuilder);
if (result.isFullRecord()) {
pruneTriggered = true;
emptyCurrentBufferBuilder(targetChannel);
break;
}
//当BufferBuilder满了之后,构造新的BufferBuilder,并创建BufferConsumer(内部和BufferBuilder维护同一个memorySegment对象)添加到
//subResultPartition的BufferConsumer队列中等待PipelinedSubpartitionView消费
bufferBuilder = requestNewBufferBuilder(targetChannel);
//继续将RecordSerializer内部剩余的数据拷贝到BufferBuilder中
result = serializer.copyToBufferBuilder(bufferBuilder);
}
checkState(!serializer.hasSerializedData(), "All data should be written at once");
if (flushAlways) {
//通知PipelinedSubpartitionView当前BufferConsumer可以消费
flushTargetPartition(targetChannel);
}
return pruneTriggered;
}