前言:
本文章所使用的flink版本为:2.11.0
1、Flink的数据流中的数据类型
flink将数据流中的数据分为了两大类型,一种叫Buffer,一种叫Event。两者被封装到一个叫BufferOrEvent的类里面,如下图所示:
public class BufferOrEvent {
private final Buffer buffer;
private final AbstractEvent event;
...
}
那么哪些数据属于Buffer,哪些数据属于Event呢?我们看看AbstractEvent抽象类的解释:
/**
* This type of event can be used to exchange notification messages between
* different {@link TaskExecutor} objects at runtime using the communication
* channels.
*
* 翻译:在任务运行期间,通过channels,event(事件)类型被用来在不同的taskexecutor
* 之间传递、交换信息
*/
public abstract class AbstractEvent implements IOReadableWritable {}
在我们日常开发中,最常见的Event数据有:CheckpointBarrier,CancelCheckpointMarker.
而最常见的Buffer数据有:WaterMark和自定义的数据
下面我们就来看看每个任务是如何接收数据并做处理的。
2、任务的数据接收
以该类作为分析入口: org.apache.flink.streaming.runtime.io.StreamTaskNetworkInput
emitNext函数:
@Override
public InputStatus emitNext(DataOutput<T> output) throws Exception {
//作者注: 无限循环,直到获取到checkpointbarrier或者watermark或者实际的数据为止
while (true) {
/* 作者注: currentRecordDeserializer 的赋值是在processBufferOrEvent方法中,
* processBufferOrEvent是在有Buffer数据时才进行调用。
**/
if (currentRecordDeserializer != null) {
DeserializationResult result = currentRecordDeserializer.getNextRecord(deserializationDelegate);
if (result.isBufferConsumed()) {
currentRecordDeserializer.getCurrentBuffer().recycleBuffer();
currentRecordDeserializer = null;
}
if (result.isFullRecord()) {
//作者注:处理Buffer数据
processElement(deserializationDelegate.getInstance(), output);
return InputStatus.MORE_AVAILABLE;
}
}
/*作者注:从pollNext方法中获取每一个channel的数据。pollNext中会对event数据
* (CheckPointBarrier)进行处理.
**/
Optional<BufferOrEvent> bufferOrEvent = checkpointedInputGate.pollNext();
if (bufferOrEvent.isPresent()) {
/* 作者注:如果是event数据,不做处理,因为event数据在调用pollNext方法中已经做
* 过处理
**/
if (bufferOrEvent.get().isEvent() && bufferOrEvent.get().getEvent() instanceof CheckpointBarrier) {
return InputStatus.MORE_AVAILABLE;
}
/* 作者注:如果不是event数据,调用processBufferOrEvent函数,将数据放到
* currentRecordDeserializer中
**/
processBufferOrEvent(bufferOrEvent.get());
} else {
if (checkpointedInputGate.isFinished()) {
checkState(checkpointedInputGate.getAvailableFuture().isDone(), "Finished BarrierHandler should be available");
return InputStatus.END_OF_INPUT;
}
return InputStatus.NOTHING_AVAILABLE;
}
}
}
processBufferOrEvent函数:
这个函数将获取到的buffer数据添加到currentRecordDeserializer中,然后用上面的while(true)循环去获取处理。
private void processBufferOrEvent(BufferOrEvent bufferOrEvent) throws IOException {
if (bufferOrEvent.isBuffer()) {
lastChannel = channelIndexes.get(bufferOrEvent.getChannelInfo());
checkState(lastChannel != StreamTaskInput.UNSPECIFIED);
currentRecordDeserializer = recordDeserializers[lastChannel];
checkState(currentRecordDeserializer != null,
"currentRecordDeserializer has already been released");
//作者注:把buffer数据塞入到currentRecordDeserializer
currentRecordDeserializer.setNextBuffer(bufferOrEvent.getBuffer());
}
else {
// Event received
final AbstractEvent event = bufferOrEvent.getEvent();
// TODO: with checkpointedInputGate.isFinished() we might not need to support any events on this level.
if (event.getClass() != EndOfPartitionEvent.class) {
throw new IOException("Unexpected event: " + event);
}
// release the record deserializer immediately,
// which is very valuable in case of bounded stream
releaseDeserializer(channelIndexes.get(bufferOrEvent.getChannelInfo()));
}
}
3、checkpointBarrier的处理过程
由2中所述,checkpointBarrier的处理过程是在 checkpointedInputGate.pollNext();我们看一下这个pollNext函数,如下:
@Override public Optional<BufferOrEvent> pollNext() throws Exception { while (true) { //作者注: 从输入网关中获取数据 Optional<BufferOrEvent> next = inputGate.pollNext(); if (!next.isPresent()) { return handleEmptyBuffer(); } BufferOrEvent bufferOrEvent = next.get(); checkState(!barrierHandler.isBlocked(bufferOrEvent.getChannelInfo())); //作者注:如果是buffer类型(waterMark 和 自定义数据)的数据,则不做处理,直接返回,交由上一个方法进行处理。 if (bufferOrEvent.isBuffer()) { return next; } // 作者注:如果是CheckpointBarrier类型,则取出进行处理。处理过程看processBarrier方法。 else if (bufferOrEvent.getEvent().getClass() == CheckpointBarrier.class) { CheckpointBarrier checkpointBarrier = (CheckpointBarrier) bufferOrEvent.getEvent(); // 作者注:barrierHandler 分为两种:CheckpointBarrierAligner和CheckpointBarrierTracker // CheckpointBarrierAligner对应exactly-once语境 barrierHandler.processBarrier(checkpointBarrier, bufferOrEvent.getChannelInfo()); return next; } else if (bufferOrEvent.getEvent().getClass() == CancelCheckpointMarker.class) { barrierHandler.processCancellationBarrier((CancelCheckpointMarker) bufferOrEvent.getEvent()); } else { if (bufferOrEvent.getEvent().getClass() == EndOfPartitionEvent.class) { barrierHandler.processEndOfPartition(); } return next; } } }
根据flink的语义不同,barrierHandler的处理方式也不一样,下面主要针对exactly-once和at-least-once两种语义分别说明;
exactly-once语义:exactly-once语义对应的barrierHandler是CheckpointBarrierAligner。Aligner是对齐的意思,为了保证exactly-once语义,checkpointbarrier优先到达的channel被阻塞,直到所有channel里的checkpointbarrier都到达,且快照完成,才释放阻塞。代码如下:
@Override
public void processBarrier(CheckpointBarrier receivedBarrier, InputChannelInfo channelInfo) throws Exception {
final long barrierId = receivedBarrier.getId();
// fast path for single channel cases
//作者注: 如果上游只有一个分区,直接进行处理即可
if (totalNumberOfInputChannels == 1) {
resumeConsumption(channelInfo);
if (barrierId > currentCheckpointId) {
// new checkpoint
currentCheckpointId = barrierId;
notifyCheckpoint(receivedBarrier, latestAlignmentDurationNanos);
}
return;
}
// -- general code path for multiple input channels --
// 判断接收的是否是第一个barrierid
if (isCheckpointPending()) {
// this is only true if some alignment is already progress and was not canceled
//作者注: 如果barrierid和当前的CheckpointId相等,则调用onBarrier阻塞当前channel
if (barrierId == currentCheckpointId) {
// regular case
onBarrier(channelInfo);
}
/*
* 作者注: 如果barrierId大于当前的CheckpointId,说明当前的CheckpointId还未完成,下一个barrierId已经到来。
* 1、正常情况下,这种现象应该不会发生,因为只要上一个CheckpointId还未处理完,channel是被阻塞的。
* 2、如果这种现象发生,则进行处理,上一个CheckpointId的数据不再处理,以新来的barrierId为基础重新进行对齐。
* 处理过程:通知终止上一个barrierid的快照 --> 释放所有的堵塞channel,继续消费数据 --> 进行新barrierid的对齐预处理。
**/
else if (barrierId > currentCheckpointId) {
// we did not complete the current checkpoint, another started before
LOG.warn("{}: Received checkpoint barrier for checkpoint {} before completing current checkpoint {}. " +
"Skipping current checkpoint.",
taskName,
barrierId,
currentCheckpointId);
// let the task know we are not completing this
// 作者注: 通知终止上一个barrierid的执行
notifyAbort(currentCheckpointId,
new CheckpointException(
"Barrier id: " + barrierId,
CheckpointFailureReason.CHECKPOINT_DECLINED_SUBSUMED));
// abort the current checkpoint
//作者注: 释放所有的堵塞channel,继续消费数据
releaseBlocksAndResetBarriers();
// begin a new checkpoint
//作者注: 进行新barrierid的对齐预处理。
beginNewAlignment(barrierId, channelInfo, receivedBarrier.getTimestamp());
}
else {
// ignore trailing barrier from an earlier checkpoint (obsolete now)
//作者注: 异常情况:如果barrierId小于当前的CheckpointId,直接释放当前channel继续消费数据
resumeConsumption(channelInfo);
}
}
// 作者注: 如果接收的是第一个新的barrierId,则进行新barrierid的对齐预处理。
else if (barrierId > currentCheckpointId) {
// first barrier of a new checkpoint
beginNewAlignment(barrierId, channelInfo, receivedBarrier.getTimestamp());
}
//作者注:异常情况 barrierId小于当前的CheckpointId,直接释放当前channel继续消费数据
else {
// either the current checkpoint was canceled (numBarriers == 0) or
// this barrier is from an old subsumed checkpoint
resumeConsumption(channelInfo);
}
// check if we have all barriers - since canceled checkpoints always have zero barriers
// this can only happen on a non canceled checkpoint
//作者注: 如果所有channel的barrierid都接收到,则释放阻塞的channel继续消费数据,并且通知jobManager已完成barrierid
if (numBarriersReceived + numClosedChannels == totalNumberOfInputChannels) {
// actually trigger checkpoint
if (LOG.isDebugEnabled()) {
LOG.debug("{}: Received all barriers, triggering checkpoint {} at {}.",
taskName,
receivedBarrier.getId(),
receivedBarrier.getTimestamp());
}
// 作者注:释放阻塞的channel继续消费数据
releaseBlocksAndResetBarriers();
// 作者注:通知JobManager当前barrierid已完成
notifyCheckpoint(receivedBarrier, latestAlignmentDurationNanos);
}
}
at-least-once语义处理:对应的处理类是:org.apache.flink.streaming.runtime.io.CheckpointBarrierTracker。相比exactly-once语义,at-least-once语义处理无需进行channel的阻塞,执行代码如下:
public void processBarrier(CheckpointBarrier receivedBarrier, InputChannelInfo channelInfo) throws Exception {
final long barrierId = receivedBarrier.getId();
// fast path for single channel trackers
// 作者注: 如果上游只有一个分区,直接通知进行cp快照
if (totalNumberOfInputChannels == 1) {
notifyCheckpoint(receivedBarrier, 0);
return;
}
// general path for multiple input channels
if (LOG.isDebugEnabled()) {
LOG.debug("Received barrier for checkpoint {} from channel {}", barrierId, channelInfo);
}
// find the checkpoint barrier in the queue of pending barriers
CheckpointBarrierCount barrierCount = null;
int pos = 0;
// 作者注:从pendingCheckpoints中获取和当前barrierId相同的CheckPointBarrier
for (CheckpointBarrierCount next : pendingCheckpoints) {
if (next.checkpointId == barrierId) {
barrierCount = next;
break;
}
pos++;
}
if (barrierCount != null) {
// add one to the count to that barrier and check for completion
//作者注:将当前的获取到的barrierId的量加1
int numBarriersNew = barrierCount.incrementBarrierCount();
//作者注:如果当前barrierId所有上游的barrier都已经获取到,则进行两个处理:
// 1、小于当前barrierId的barrier不在处理,从pendingCheckpoints弹出(pollFirst)
// 2、想JobManager通知当前barrierID已全部获取。
if (numBarriersNew == totalNumberOfInputChannels) {
// checkpoint can be triggered (or is aborted and all barriers have been seen)
// first, remove this checkpoint and all all prior pending
// checkpoints (which are now subsumed)
for (int i = 0; i <= pos; i++) {
pendingCheckpoints.pollFirst();
}
// notify the listener
if (!barrierCount.isAborted()) {
if (LOG.isDebugEnabled()) {
LOG.debug("Received all barriers for checkpoint {}", barrierId);
}
notifyCheckpoint(receivedBarrier, 0);
}
}
}
//作者注:如果当前barrierId是第一次获取到,则加入到pendingCheckpoints中去。
else {
// first barrier for that checkpoint ID
// add it only if it is newer than the latest checkpoint.
// if it is not newer than the latest checkpoint ID, then there cannot be a
// successful checkpoint for that ID anyways
if (barrierId > latestPendingCheckpointID) {
markCheckpointStart(receivedBarrier.getTimestamp());
latestPendingCheckpointID = barrierId;
pendingCheckpoints.addLast(new CheckpointBarrierCount(barrierId));
// make sure we do not track too many checkpoints
// 为了保证pendingCheckpoints不至于过大,超过阈值,则将最小的barrierId去掉
if (pendingCheckpoints.size() > MAX_CHECKPOINTS_TO_TRACK) {
pendingCheckpoints.pollFirst();
}
}
}
}
以上就是checkpointbarrier的处理过程。
4、watermark的处理过程
由2中代码知道,Buffer数据的处理是在 processElement(deserializationDelegate.getInstance(), output)函数中完成的,我们看一下这个函数:
private void processElement(StreamElement recordOrMark, DataOutput<T> output) throws Exception {
//作者注: 自定义数据的处理
if (recordOrMark.isRecord()){
output.emitRecord(recordOrMark.asRecord());
//作者注: WaterMark的处理
} else if (recordOrMark.isWatermark()) {
statusWatermarkValve.inputWatermark(recordOrMark.asWatermark(), lastChannel);
} else if (recordOrMark.isLatencyMarker()) {
output.emitLatencyMarker(recordOrMark.asLatencyMarker());
} else if (recordOrMark.isStreamStatus()) {
statusWatermarkValve.inputStreamStatus(recordOrMark.asStreamStatus(), lastChannel);
} else {
throw new UnsupportedOperationException("Unknown type of StreamElement");
}
}
1、watermark的处理:进入该方法进行追踪,statusWatermarkValve.inputWatermark(recordOrMark.asWatermark(), lastChannel);
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();
// if the input watermark's value is less than the last received watermark for its input channel, ignore it also.
// 作者注: 如果当前的watermark 大于 该channel保存的watermark,则进行替换
if (watermarkMillis > channelStatuses[channelIndex].watermark) {
channelStatuses[channelIndex].watermark = watermarkMillis;
// previously unaligned input channels are now aligned if its watermark has caught up
// 作者注: lastOutputWatermark保存的是上一次所有channel的watermark都对齐时的最小的watermark。
// 如果当前watermark大于lastOutputWatermark,说明当前channel重新进行了对齐,则当前isWatermarkAligned置为true
if (!channelStatuses[channelIndex].isWatermarkAligned && watermarkMillis >= lastOutputWatermark) {
channelStatuses[channelIndex].isWatermarkAligned = true;
}
// now, attempt to find a new min watermark across all aligned channels
//作者注:寻找所有channel最小的watermark,并进行处理。
findAndOutputNewMinWatermarkAcrossAlignedChannels();
}
}
}
findAndOutputNewMinWatermarkAcrossAlignedChannels方法解析:
private void findAndOutputNewMinWatermarkAcrossAlignedChannels() throws Exception {
long newMinWatermark = Long.MAX_VALUE;
boolean hasAlignedChannels = false;
// determine new overall watermark by considering only watermark-aligned channels across all channels
// 作者注: 遍历所有的channel,找出最小的watermark
for (InputChannelStatus channelStatus : channelStatuses) {
if (channelStatus.isWatermarkAligned) {
hasAlignedChannels = true;
newMinWatermark = Math.min(channelStatus.watermark, newMinWatermark);
}
}
// we acknowledge and output the new overall watermark if it really is aggregated
// from some remaining aligned channel, and is also larger than the last output watermark
//作者注:如果所有的channel都已经完成了watermark的对齐,且最小的watermark比lastOutputWatermark大,
// 则进行 lastOutputWatermark的替换,且进行watermark的处理。
if (hasAlignedChannels && newMinWatermark > lastOutputWatermark) {
lastOutputWatermark = newMinWatermark;
output.emitWatermark(new Watermark(lastOutputWatermark));
}
}
解释:如何理解watermark的对齐?
正常情况下,watermark都是周期性生成的(比如说10秒生成一次),当当前算子(通常是窗口的算子)接收到上游所有channel同一周期生成的watermark时,才算是watermark完成一次对齐,这个和接收到同一个barrierId的checkpointBarrier才算完成一次barrier的对齐是一个意思。
5、自定义数据的处理过程
自定义数据处理过程如下图所示:output.emitRecord(recordOrMark.asRecord());这个函数最终会把数据传到自定义的函数中进行计算。
private void processElement(StreamElement recordOrMark, DataOutput<T> output) throws Exception {
//作者注: 自定义数据的处理
if (recordOrMark.isRecord()){
output.emitRecord(recordOrMark.asRecord());
//作者注: WaterMark的处理
} else if (recordOrMark.isWatermark()) {
statusWatermarkValve.inputWatermark(recordOrMark.asWatermark(), lastChannel);
} else if (recordOrMark.isLatencyMarker()) {
output.emitLatencyMarker(recordOrMark.asLatencyMarker());
} else if (recordOrMark.isStreamStatus()) {
statusWatermarkValve.inputStreamStatus(recordOrMark.asStreamStatus(), lastChannel);
} else {
throw new UnsupportedOperationException("Unknown type of StreamElement");
}
}
以上,是数据从某个task接收之后的处理过程。