Thank you Zhijiang! The second question about config is just because I find a method in InputProcessorUtil. I guess AT_LEAST_ONCE mode is a simpler way to handle checkpont barrier?
private static CheckpointBarrierHandler createCheckpointBarrierHandler( StreamConfig config, InputGate[] inputGates, SubtaskCheckpointCoordinator checkpointCoordinator, String taskName, AbstractInvokable toNotifyOnCheckpoint) { switch (config.getCheckpointMode()) { case EXACTLY_ONCE: if (config.isUnalignedCheckpointsEnabled()) { return new AlternatingCheckpointBarrierHandler( new CheckpointBarrierAligner(taskName, toNotifyOnCheckpoint, inputGates), new CheckpointBarrierUnaligner(checkpointCoordinator, taskName, toNotifyOnCheckpoint, inputGates), toNotifyOnCheckpoint); } return new CheckpointBarrierAligner(taskName, toNotifyOnCheckpoint, inputGates); case AT_LEAST_ONCE: int numInputChannels = Arrays.stream(inputGates).mapToInt(InputGate::getNumberOfInputChannels).sum(); return new CheckpointBarrierTracker(numInputChannels, toNotifyOnCheckpoint); default: throw new UnsupportedOperationException("Unrecognized Checkpointing Mode: " + config.getCheckpointMode()); } } ________________________________ 发件人: Zhijiang <wangzhijiang...@aliyun.com> 发送时间: 2020年6月22日 10:41 收件人: Lu Weizheng <luweizhen...@hotmail.com>; user@flink.apache.org <user@flink.apache.org> 主题: Re: Unaligned Checkpoint and Exactly Once Hi Weizheng, The unaligned checkpoint (UC) only supports exactly-once mode in Flink 1.11 except savepoint mode. The savepoint is probably used in job rescaling scenario and we plan to support it in future release version. Of course UC can satisfy exactly-once semantic as promised. Regarding the config issue, i am not sure I get your point here. The first config is for describing whether the current setting mode (actually only exactly-once) enables UC or not, and the second config is for setting the different mode (exactly-once or at least-once). I guess you refer to merge them by using the first config form. But somehow they seem two different dimensions for config the checkpoint. One is for the semantic of data processing guarantee. And the other is for how we realize two different mechanisms to guarantee one (exactly-once) of the semantics. Best, Zhijiang ------------------------------------------------------------------ From:Lu Weizheng <luweizhen...@hotmail.com> Send Time:2020年6月22日(星期一) 07:20 To:user@flink.apache.org <user@flink.apache.org> Subject:Unaligned Checkpoint and Exactly Once Hi there, The new feature in Flink 1.11 will provide us the Unaligned Checkpoint which means a operator subtask does not need to wait all the Checkpoint barrier and will not block some channels. As the Checkpoint barrier is the key mechanism for Exactly Once guarantee, I am not sure Unaligned Checkpoint could still achieve Exactly Once guarantee or only AT Least Once? FLIP-76<https://cwiki.apache.org/confluence/display/FLINK/FLIP-76%3A+Unaligned+Checkpoints> : Unaligned checkpoints will initially be an optional feature. After collecting experience and implementing all necessary extensions, unaligned checkpoint will probably be enabled by default for exactly once. What's more, in the following two configs, Config 1 env.getCheckpointConfig().enableUnalignedCheckpoints(); Config 2 checkpointCfg.setCheckpointingMode(CheckpointingMode.AT_LEAST_ONCE); Does Config 2 use a even simpler way for Checkpoint than Unaligned Checkpoint? Hope for replies! Weizheng