本文将从源码层面来分析在Flink中是如何保证sink数据到HDFS的exactly-once语义的。
Flink中sink数据到HDFS是由BucketingSink
来完成。BucketingSink
产生的HDFS文件路径格式如下,
/{base/path}/{bucket/path}/{part_prefix}-{parallel_task_index}-{count}{part_suffix}
其中,
{base/path}
,构造BucketingSink
时指定的base路径;{bucket/path}
,分桶路径,BucketingSink
可以对数据进行分桶(也可以理解为分区),可以根据系统时间进行分桶,也可以根据数据进行分桶,通过实现Bucketer
可以自定义分桶规则;{part_prefix}
与{part_suffix}
,可自定义的字符串,{part_prefix}
默认为part
,{part_suffix}
默认为空;{parallel_task_index}
,因为存在并行的sink task,因此写出去的文件需要用task index来区分;{count}
,一个sink task在一个分桶下不可能只允许写入一个文件,BucketingSink
会对文件进行滚动操作,有两种rolling策略,文件大小以及时间(与日志文件rolling策略类似)。通过BucketingSink#setBatchSize
及BucketingSink#setBatchRolloverInterval
可以分别设置该大小及时间。默认大小为384MB
,时间为Long.MAX_VALUE
。{count}
从0开始,根据文件rolling递增;为了实现exactly-once语义,BucketingSink
产生的文件有3种不同的状态:
{in_progress_prefix}{part_prefix}-{parallel_task_index}-{count}{part_suffix}{in_progress_suffix}
;in-progress
状态的文件关闭后进入pending
状态,文件重命名,等待Checkpoint。文件名格式为{pending_prefix}{part_prefix}-{parallel_task_index}-{count}{part_suffix}{pending_suffix}
;pending
状态的文件即可置为finished
状态,文件重命名,该状态的文件即为最终产生的可用文件,文件名格式之前已经描述过了;另外,in-progress
状态文件关闭,进入pending
状态,由两种机制触发,一个是上文已经提到过的rolling策略,另一个则是分桶inactive触发的文件关闭。BucketingSink
会定期扫描所有分桶,当某个分桶超过一定时间没有写入,则会关闭该分桶下in-progress
状态的文件,进入pending
状态。通过BucketingSink#setInactiveBucketCheckInterval
及BucketingSink#setInactiveBucketThreshold
即可分别设置扫描周期以及分桶inactive时间阈值,默认都是60秒。
接下来开始进入源码分析,主要是这几个方法:
initializeState
open
invoke
snapshotState
notifyCheckpointComplete
先说明下为什么是这几个方法。每个Flink程序都会被转换成JobGraph
,在运行时最终部署成一个个的StreamTask
,每个StreamTask
执行自己负责的OperatorChain
。而所有的SinkFunction
都是由StreamSink
这个operator来运行。
/**
* A {@link StreamOperator} for executing {@link SinkFunction SinkFunctions}.
*/
@Internal
public class StreamSink<IN> extends AbstractUdfStreamOperator<Object, SinkFunction<IN>>
implements OneInputStreamOperator<IN, Object> { ... }
StreamTask
的执行是通过其invoke
方法。invoke
方法做的事情大致如下,
* -- invoke()
* |
* +----> Create basic utils (config, etc) and load the chain of operators
* +----> operators.setup()
* +----> task specific init()
* +----> initialize-operator-states()
* +----> open-operators()
* +----> run()
* +----> close-operators()
* +----> dispose-operators()
* +----> common cleanup
* +----> task specific cleanup()
其中,initialize operator states,open operators以及run是这里主要关注的三个操作,对应的代码块如下,
synchronized (lock) {
// both the following operations are protected by the lock
// so that we avoid race conditions in the case that initializeState()
// registers a timer, that fires before the open() is called.
initializeState();
openAllOperators();
}
// final check to exit early before starting to run
if (canceled) {
throw new CancelTaskException();
}
// let the task do its work
isRunning = true;
run();
initializeState
方法会调用到BucketingSink#initializeState
;openAllOperators
方法会调用到BucketingSink#open
;run
方法会调用到BucketingSink#invoke
。
另外两个方法,snapshotState
以及notifyCheckpointComplete
是跟Checkpoint相关的,StreamTask
在执行Checkpoint时,会调用operator的snapshotState
方法,最终会调用到BucketingSink#snapshotState
。整个Job Checkpoint成功后会发送通知,BucketingSink#notifyCheckpointComplete
会被调用。
下面就来看下这些方法的具体实现。
initializeState
主要做了两件事:
initFileSystem
初始化Hadoop的FileSystem
;handleRestoredBucketState
从Checkpoint/Savepoint中恢复状态信息;我们往下先看看其他方法再回过头来看状态恢复的实现,即handleRestoredBucketState
方法。
open
方法也比较简单,主要是利用ProcessingTimeService
注册定时器,定时检查上文提到的inactive的分桶。
invoke
方法主要做了四件事,如下注释,
@Override
public void invoke(T value) throws Exception {
/******* 第一步,使用Bucketer获取当前数据所属的分桶 *******/
Path bucketPath = bucketer.getBucketPath(clock, new Path(basePath), value);
long currentProcessingTime = processingTimeService.getCurrentProcessingTime();
/******* 第二步,获取分桶状态,若分桶不存在,则生成分桶状态信息 *******/
BucketState<T> bucketState = state.getBucketState(bucketPath);
if (bucketState == null) {
bucketState = new BucketState<>(currentProcessingTime);
state.addBucketState(bucketPath, bucketState);
}
/******* 第三步,对分桶当前正在写入的文件执行rolling策略 *******/
if (shouldRoll(bucketState, currentProcessingTime)) {
openNewPartFile(bucketPath, bucketState);
}
/******* 第四步,数据写入文件 *******/
bucketState.writer.write(value);
bucketState.lastWrittenToTime = currentProcessingTime;
}
分桶状态(BucketState
)信息包括:
currentFile
,该分桶当前正在被写入,即in-progress
状态的文件;currentFileValidLength
,该文件的有效长度;creationTime
,该文件的创建时间;lastWrittenToTime
,该分桶最后一次写入的时间;partCounter
,上文提到的文件名称格式中的{count}
值;pendingFiles
,该分桶下处于pending
状态的文件;pendingFilesPerCheckpoint
,等待Checkpoint成功通知的文件;来看下文件新建跟写入的实现。由openNewPartFile
新建文件,该方法主要做两件事:
closeCurrentPartFile
方法,如果当前分桶有处于in-progress
状态的文件,则调用Writer#close
方法关闭该文件,并且将该文件重命名,置为pending
状态,并修改分桶状态的pendingFiles
信息;in-progress
状态的文件名,调用Writer#open
打开文件,修改分桶状态的currentFile
,creationTime
等信息;文件的写入则是调用Writer#write
来完成。可以看到这里分别使用了Writer
的open
,write
,close
方法来实现文件的创建,写入以及关闭。Writer
的默认实现为StringWriter
,下面来看下该实现。
open
,调用由BucketingSink
初始化的FileSystem
的create
方法来创建文件,得到FSDataOutputStream
;write
,调用FSDataOutputStream#write
方法写入数据;close
,首先根据配置(syncOnFlush
)调用FSDataOutputStream
的hsync
或者hflush
来flush数据(二者区别请参考API文档),然后调用FSDataOutputStream#close
方法关闭文件;snapshotState
主要是以下三步,
Writer#flush
将缓存的数据写出去,并记录文件长度,更新currentFileValidLength
信息;pendingFilesPerCheckpoint
信息,记录本次Checkpoint该分桶对应的pendingFiles
,并将pendingFiles
置空;OperatorStateStore
当中,以便后续进行持久化;notifyCheckpointComplete
主要做的事情就是根据snapshotState
中记录的pendingFilesPerCheckpoint
信息,将所有等待Checkpoint成功的pending
状态的文件重命名,置为最终的,也是可用的finished
状态。
现在可以回过头来看下,状态恢复是如何实现,从而保证了exactly-once语义的。状态恢复实现,上文我们已经提到,是在handleRestoredBucketState
方法。
private void handleRestoredBucketState(State<T> restoredState) {
Preconditions.checkNotNull(restoredState);
for (BucketState<T> bucketState : restoredState.bucketStates.values()) {
/******* Checkpoint成功时`pendingFiles`应该是空的 *******/
// we can clean all the pending files since they were renamed to
// final files after this checkpoint was successful
// (we re-start from the last **successful** checkpoint)
bucketState.pendingFiles.clear();
/******* 处理上一次Checkpoint成功时处于`in-progress`状态的文件 *******/
handlePendingInProgressFile(bucketState.currentFile, bucketState.currentFileValidLength);
// Now that we've restored the bucket to a valid state, reset the current file info
bucketState.currentFile = null;
bucketState.currentFileValidLength = -1;
bucketState.isWriterOpen = false;
/******* 处理分桶状态的`pendingFilesPerCheckpoint`信息 *******/
handlePendingFilesForPreviousCheckpoints(bucketState.pendingFilesPerCheckpoint);
bucketState.pendingFilesPerCheckpoint.clear();
}
}
可以看到主要是根据Checkpoint成功后持久化下来的分桶状态信息进行处理,in-progress
状态的文件以及pendingFilesPerCheckpoint
信息。下面来看下对应的handlePendingInProgressFile
以及handlePendingFilesForPreviousCheckpoints
这两个方法。
处于in-progress
状态的文件,在该次Checkpoint成功之后,故障发生之前(这里假设状态恢复是由于发生故障引起),有以下几种可能,
in-progress
状态,没有继续写入,文件有效长度仍然是Checkpoint时由snapshotState
方法记录下的currentFileValidLength
;in-progress
状态,发生了写入,文件有效长度大于记录下的currentFileValidLength
;pending
状态;finished
状态;无论处于以上哪种状态,现在需要做的,就是将文件的有效长度恢复到Checkpoint成功时记录的currentFileValidLength
。因为恢复后的in-progress
状态的文件不再继续写入(产生新文件来写入后续数据),因此先将该文件重命名置为finished
状态(已经是finished
状态则省略这一步),然后有两种做法,
truncate
,则直接将文件truncate到currentFileValidLength
这个有效长度即可;{valid_length_prefix}{part_prefix}-{parallel_task_index}-{count}{part_suffix}{valid_length_suffix}
文件,将currentFileValidLength
写入该文件,后续在读取数据文件时需要先读取这个记录了该文件有效长度的文件以确保数据的有效性,否则可能读取到重复的数据,这样就不能保证exactly-once语义了;这个方法主要是处理分桶状态的pendingFilesPerCheckpoint
信息,当Checkpoint成功(snapshotState
方法)持久化下来的pendingFilesPerCheckpoint
,保存的是等待Checkpoint成功通知的处于pending
状态的文件,这些文件在Checkpoint成功之后,故障发生之前,有下面两种可能:
pending
状态,因为从pengding
转为finished
状态是在Checkpoint成功通知到才会发生(也是就成功执行了notifyCheckpointComplete
方法)。针对这种情况,现在需要将这些文件置为finished
状态;finished
状态,无需额外操作;这里需要额外说明,Checkpoint是否成功,在发送通知,即调用notifyCheckpointComplete
之前就已经确定了。notifyCheckpointComplete
即使发生了异常也不会导致Checkpoint失败,参考CheckpointListener
的API文档,
This method is called as a notification once a distributed checkpoint has been completed. Note that any exception during this method will not cause the checkpoint to fail any more.
经过源码层面的分析可以看到,对exactly-once语义的保证,是通过引入中间状态(in-progress
及pending
)和最终可用状态(finished
)来实现的,是一种两阶段提交(2PC)的方案。当故障发生时,对处于中间状态的数据进行回滚或者提交(initializeState
方法)以保证数据的有效性。数据只有流转(Checkpoint成功)到最终状态才是可用的。
本文通过对源码的分析来了解BucketingSink
对exactly-once语义的保证。另外,这里记录下一个潜在的问题,就是数据写入的性能问题,当前的设计,每个分桶下只有一个正在写入,即in-progress
状态的文件,并且是在invoke
方法同步写入,在数据量大的情况下,数据写入的性能一定是不容乐观的。后面可能会有这方面相关的优化,例如异步化,stay tuned o(∩_∩)o