An Overview of End-to-End Exactly-Once Processing in Apache Flink (with Apache Kafka, too!)
01 Mar 2018 Piotr Nowojski (@PiotrNowojski) & Mike Winters (@wints)
This post is an adaptation of Piotr Nowojski’s presentation from Flink Forward Berlin 2017. You can find the slides and a recording of the presentation on the Flink Forward Berlin website.
Apache Flink 1.4.0, released in December 2017, introduced a significant milestone for stream processing with Flink: a new feature called TwoPhaseCommitSinkFunction (relevant Jira here) that extracts the common logic of the two-phase commit protocol and makes it possible to build end-to-end exactly-once applications with Flink and a selection of data sources and sinks, including Apache Kafka versions 0.11 and beyond. It provides a layer of abstraction and requires a user to implement only a handful of methods to achieve end-to-end exactly-once semantics.
If that’s all you need to hear, let us point you to the relevant place in the Flink documentation, where you can read about how to put TwoPhaseCommitSinkFunction to use.
But if you’d like to learn more, in this post, we’ll share an in-depth overview of the new feature and what is happening behind the scenes in Flink.
Throughout the rest of this post, we’ll:
- Describe the role of Flink’s checkpoints for guaranteeing exactly-once results within a Flink application.
- Show how Flink interacts with data sources and data sinks via the two-phase commit protocol to deliver end-to-end exactly-once guarantees.
- Walk through a simple example on how to use
TwoPhaseCommitSinkFunctionto implement an exactly-once file sink.
Exactly-once Semantics Within an Apache Flink Application
When we say “exactly-once semantics”, what we mean is that each incoming event affects the final results exactly once. Even in case of a machine or software failure, there’s no duplicate data and no data that goes unprocessed.
Flink has long provided exactly-once semantics within a Flink application. Over the past few years, we’ve written in depth about Flink’s checkpointing, which is at the core of Flink’s ability to provide exactly-once semantics. The Flink documentation also provides a thorough overview of the feature.
Before we continue, here’s a quick summary of the checkpointing algorithm because understanding checkpoints is necessary for understanding this broader topic.
A checkpoint in Flink is a consistent snapshot of:
- The current state of an application
- The position in an input stream
Flink generates checkpoints on a regular, configurable interval and then writes the checkpoint to a persistent storage system, such as S3 or HDFS. Writing the checkpoint data to the persistent storage happens asynchronously, which means that a Flink application continues to process data during the checkpointing process.
In the event of a machine or software failure and upon restart, a Flink application resumes processing from the most recent successfully-completed checkpoint; Flink restores application state and rolls back to the correct position in the input stream from a checkpoint before processing starts again. This means that Flink computes results as though the failure never occurred.
Before Flink 1.4.0, exactly-once semantics were limited to the scope of a Flink application only and did not extend to most of the external systems to which Flink sends data after processing.
But Flink applications operate in conjunction with a wide range of data sinks, and developers should be able to maintain exactly-once semantics beyond the context of one component.
To provide end-to-end exactly-once semantics–that is, semantics that also apply to the external systems that Flink writes to in addition to the state of the Flink application–these external systems must provide a means to commit or roll back writes that coordinate with Flink’s checkpoints.
One common approach for coordinating commits and rollbacks in a distributed system is the two-phase commit protocol. In the next section, we’ll go behind the scenes and discuss how Flink’s TwoPhaseCommitSinkFunction utilizes the two-phase commit protocol to provide end-to-end exactly-once semantics.
End-to-end Exactly Once Applications with Apache Flink
We’ll walk through the two-phase commit protocol and how it enables end-to-end exactly-once semantics in a sample Flink application that reads from and writes to Kafka. Kafka is a popular messaging system to use along with Flink, and Kafka recently added support for transactions with its 0.11 release. This means that Flink now has the necessary mechanism to provide end-to-end exactly-once semantics in applications when receiving data from and writing data to Kafka.
Flink’s support for end-to-end exactly-once semantics is not limited to Kafka and you can use it with any source / sink that provides the necessary coordination mechanism. For example, Pravega, an open-source streaming storage system from Dell/EMC, also supports end-to-end exactly-once semantics with Flink via the TwoPhaseCommitSinkFunction.

In the sample Flink application that we’ll discuss today, we have:
- A data source that reads from Kafka (in Flink, a KafkaConsumer)
- A windowed aggregation
- A data sink that writes data back to Kafka (in Flink, a KafkaProducer)
For the data sink to provide exactly-once guarantees, it must write all data to Kafka within the scope of a transaction. A commit bundles all writes between two checkpoints.
This ensures that writes are rolled back in case of a failure.
However, in a distributed system with multiple, concurrently-running sink tasks, a simple commit or rollback is not sufficient, because all of the components must “agree” together on committing or rolling back to ensure a consistent result. Flink uses the two-phase commit protocol and its pre-commit phase to address this challenge.
The starting of a checkpoint represents the “pre-commit” phase of our two-phase commit protocol. When a checkpoint starts, the Flink JobManager injects a checkpoint barrier (which separates the records in the data stream into the set that goes into the current checkpoint vs. the set that goes into the next checkpoint) into the data stream.
The barrier is passed from operator to operator. For every operator, it triggers the operator’s state backend to take a snapshot of its state.

The data source stores its Kafka offsets, and after completing this, it passes the checkpoint barrier to the next operator.
This approach works if an operator has internal state only. Internal state is everything that is stored and managed by Flink’s state backends - for example, the windowed sums in the second operator. When a process has only internal state, there is no need to perform any additional action during pre-commit aside from updating the data in the state backends before it is checkpointed. Flink takes care of correctly committing those writes in case of checkpoint success or aborting them in case of failure.

However, when a process has external state, this state must be handled a bit differently. External state usually comes in the form of writes to an external system such as Kafka. In that case, to provide exactly-once guarantees, the external system must provide support for transactions that integrates with a two-phase commit protocol.
We know that the data sink in our example has such external state because it’s writing data to Kafka. In this case, in the pre-commit phase, the data sink must pre-commit its external transaction in addition to writing its state to the state backend.

The pre-commit phase finishes when the checkpoint barrier passes through all of the operators and the triggered snapshot callbacks complete. At this point the checkpoint completed successfully and consists of the state of the entire application, including pre-committed external state. In case of a failure, we would re-initialize the application from this checkpoint.
The next step is to notify all operators that the checkpoint has succeeded. This is the commit phase of the two-phase commit protocol and the JobManager issues checkpoint-completed callbacks for every operator in the application. The data source and window operator have no external state, and so in the commit phase, these operators don’t have to take any action. The data sink does have external state, though, and commits the transaction with the external writes.

So let’s put all of these different pieces together:
- Once all of the operators complete their pre-commit, they issue a commit.
- If at least one pre-commit fails, all others are aborted, and we roll back to the previous successfully-completed checkpoint.
- After a successful pre-commit, the commit must be guaranteed to eventually succeed – both our operators and our external system need to make this guarantee. If a commit fails (for example, due to an intermittent network issue), the entire Flink application fails, restarts according to the user’s restart strategy, and there is another commit attempt. This process is critical because if the commit does not eventually succeed, data loss occurs.
Therefore, we can be sure that all operators agree on the final outcome of the checkpoint: all operators agree that the data is either committed or that the commit is aborted and rolled back.
Implementing the Two-Phase Commit Operator in Flink
All the logic required to put a two-phase commit protocol together can be a little bit complicated and that’s why Flink extracts the common logic of the two-phase commit protocol into the abstract TwoPhaseCommitSinkFunction class.
Let’s discuss how to extend a TwoPhaseCommitSinkFunction on a simple file-based example. We need to implement only four methods and present their implementations for an exactly-once file sink:
beginTransaction -to begin the transaction, we create a temporary file in a temporary directory on our destination file system. Subsequently, we can write data to this file as we process it.preCommit -on pre-commit, we flush the file, close it, and never write to it again. We’ll also start a new transaction for any subsequent writes that belong to the next checkpoint.commit -on commit, we atomically move the pre-committed file to the actual destination directory. Please note that this increases the latency in the visibility of the output data.abort -on abort, we delete the temporary file.
As we know, if there’s any failure, Flink restores the state of the application to the latest successful checkpoint. One potential catch is in a rare case when the failure occurs after a successful pre-commit but before notification of that fact (a commit) reaches our operator. In that case, Flink restores our operator to the state that has already been pre-committed but not yet committed.
We must save enough information about pre-committed transactions in checkpointed state to be able to either abort or commit transactions after a restart. In our example, this would be the path to the temporary file and target directory.
The TwoPhaseCommitSinkFunction takes this scenario into account, and it always issues a preemptive commit when restoring state from a checkpoint. It is our responsibility to implement a commit in an idempotent way. Generally, this shouldn’t be an issue. In our example, we can recognize such a situation: the temporary file is not in the temporary directory, but has already been moved to the target directory.
There are a handful of other edge cases that TwoPhaseCommitSinkFunction takes into account, too. Learn more in the Flink documentation.
Wrapping Up
If you’ve made it this far, thanks for staying with us through a detailed post. Here are some key points that we covered:
- Flink’s checkpointing system serves as Flink’s basis for supporting a two-phase commit protocol and providing end-to-end exactly-once semantics.
- An advantage of this approach is that Flink does not materialize data in transit the way that some other systems do–there’s no need to write every stage of the computation to disk as is the case is most batch processing.
- Flink’s new
TwoPhaseCommitSinkFunctionextracts the common logic of the two-phase commit protocol and makes it possible to build end-to-end exactly-once applications with Flink and external systems that support transactions - Starting with Flink 1.4.0, both the Pravega and Kafka 0.11 producers provide exactly-once semantics; Kafka introduced transactions for the first time in Kafka 0.11, which is what made the Kafka exactly-once producer possible in Flink.
- The Kafka 0.11 producer is implemented on top of the
TwoPhaseCommitSinkFunction, and it offers very low overhead compared to the at-least-once Kafka producer.
We’re very excited about what this new feature enables, and we look forward to being able to support additional producers with the TwoPhaseCommitSinkFunction in the future.
This post first appeared on the data Artisans blogand was contributed to Apache Flink and the Flink blog by the original authors Piotr Nowojski and Mike Winters.
An Overview of End-to-End Exactly-Once Processing in Apache Flink (with Apache Kafka, too!)的更多相关文章
- Kafka设计解析(二十二)Flink + Kafka 0.11端到端精确一次处理语义的实现
转载自 huxihx,原文链接 [译]Flink + Kafka 0.11端到端精确一次处理语义的实现 本文是翻译作品,作者是Piotr Nowojski和Michael Winters.前者是该方案 ...
- <译>Flink编程指南
Flink 的流数据 API 编程指南 Flink 的流数据处理程序是常规的程序 ,通过再流数据上,实现了各种转换 (比如 过滤, 更新中间状态, 定义窗口, 聚合).流数据可以来之多种数据源 (比如 ...
- 入门大数据---Flink学习总括
第一节 初识 Flink 在数据激增的时代,催生出了一批计算框架.最早期比较流行的有MapReduce,然后有Spark,直到现在越来越多的公司采用Flink处理.Flink相对前两个框架真正做到了高 ...
- 大数据开发-Flink-数据流DataStream和DataSet
Flink主要用来处理数据流,所以从抽象上来看就是对数据流的处理,正如前面大数据开发-Flink-体系结构 && 运行架构提到写Flink程序实际上就是在写DataSource.Tra ...
- Apache Sqoop - Overview——Sqoop 概述
Apache Sqoop - Overview Apache Sqoop 概述 使用Hadoop来分析和处理数据需要将数据加载到集群中并且将它和企业生产数据库中的其他数据进行结合处理.从生产系统加载大 ...
- Apache Sqoop - Overview Apache Sqoop 概述
使用Hadoop来分析和处理数据需要将数据加载到集群中并且将它和企业生产数据库中的其他数据进行结合处理.从生产系统加载大块数据到Hadoop中或者从大型集群的map reduce应用中获得数据是个挑战 ...
- Java资源大全中文版(Awesome最新版)
Awesome系列的Java资源整理.awesome-java 就是akullpp发起维护的Java资源列表,内容包括:构建工具.数据库.框架.模板.安全.代码分析.日志.第三方库.书籍.Java 站 ...
- Linux 集群
html,body { } .CodeMirror { height: auto } .CodeMirror-scroll { } .CodeMirror-lines { padding: 4px 0 ...
- Gradle笔记系列(一)
1.Gradle概述 Gradle是一个基于Apache Ant和Apache Maven概念的项目自动化构建工具.它使用一种基于Groovy的特定领域语言(DSL)来声明项目设置,抛弃了基于XML的 ...
随机推荐
- C# 8中的范围类型(Range Type)
C# 8.0中加入了一个新的范围类型(Range Type). 这里我们首先展示一些代码,并一步一步为代码添加一些不同的东西, 为大家展示一下范围类型的功能和用法. 我们最原始的代码如下: stati ...
- 机器学习之决策树一-ID3原理与代码实现
决策树之系列一ID3原理与代码实现 本文系作者原创,转载请注明出处:https://www.cnblogs.com/further-further-further/p/9429257.html 应用实 ...
- [十五]javaIO之SequenceInputStream
功能简介 SequenceInputStream 合并流 顾名思义,就是可以吧两个流合并起来 他并没有很复杂,单纯的很,仅仅实现了InputStream 他拥有两个构造方法把两个InputS ...
- [十四]基础类型之StringBuffer 与 StringBuilder对比
StringBuilder 和 StringBuffer是高度类似的两个类 StringBuilder是StringBuffer的版本改写,下面从几个方面简单的对比下他们的区别 类继承关系 上文中,我 ...
- javaWeb项目中的路径格式 请求url地址 客户端路径 服务端路径 url-pattern 路径 获取资源路径 地址 url
javaweb项目中有很多场景的路径客户端的POST/GET请求,服务器的请求转发,资源获取需要设置路径等这些路径表达的含义都有不同,所以想要更好的书写规范有用的路径代码 需要对路径有一个清晰地认知 ...
- 痞子衡嵌入式:ARM Cortex-M文件那些事(3)- 工程文件(.ewp)
大家好,我是痞子衡,是正经搞技术的痞子.今天痞子衡给大家讲的是嵌入式开发里的project文件. 前面两节课里,痞子衡分别给大家介绍了嵌入式开发中的两种典型input文件:源文件(.c/.h/.s). ...
- vb.net 分割byte数组的方法SplitBytes
以下代码随手写的 并没有大量测试 效率也有待提升 如果需要C#的请自行转换 Function SplitBytes(Data As Byte(), Delimiter As Byte()) As Li ...
- DataTable转换成List集合,传递到HTML页面
public string GetPwd(string str) { var dt= bll.Gets(str); List<string> list = new List<stri ...
- [MySQL] 索引中的b树索引
1.索引如果没有特别指明类型,一般是说b树索引,b树索引使用b树数据结构存储数据,实际上很多存储引擎使用的是b+树,每一个叶子节点都包含指向下一个叶子节点的指针,从而方便叶子节点的范围遍历 2.底层的 ...
- C# 操作Excel图形——绘制、读取、隐藏、删除图形
简介 本篇文章将介绍C# 如何处理Excel图形相关的问题,包括以下内容要点: 1.绘制图形 1.1 绘制图形并添加文本到图形 1.2 添加图片到图形 1.3 设置图形阴影效果 1.4 设置图形透明度 ...