By Ren Qingsheng
The preceding figure shows a typical example.
First, some of the records are read from Kafka through the Source. Then, these records are sent to the operators in Flink for computing. After that, the results are written to Elasticsearch through the Sink. The Source and the Sink serve as the interfaces at both ends of the Flink job.
As shown on the left side of the figure, source interfaces used in Flink before version 1.10 were the SourceFunction API for processing streaming data and the InputFormat API for processing batch data. In Flink, after version 1.10, the community introduced a new Source API to refactor the entire Source. Let's understand why this is done.
To address the inconsistency between batch and stream implementations.
As the ecosystem continues to grow, some problems are exposed in the old APIs. One of the most obvious problems is the inconsistency between batch and stream implementations.
To address the complex implementation of the simple interface.
The previous APIs seem to be relatively simple. However, all the logics and operations for the implementation of APIs are very complicated, which is not user-friendly enough for developers.
Therefore, to solve these problems, a new Source API is designed in FLIP-27, which has the following features:
The split enumerator runs on the job master, and the source reader runs on the task executor. Therefore, the enumerator plays a leader and coordinator role, while the reader is an executor.
Their checkpoint storages are separate, but with some communication in between. For example, an enumerator needs to assign tasks to a reader and also notifies the reader that no splits need to be processed subsequently. Due to the different operation environments, there will inevitably be some network communications between them. Therefore, the following definition of communication stack is introduced.
The communication stack provides developers with some events for implementation.
First, the top layer is a source event for developers to define some customization operations. For example, if a source reader must stop reading data under certain conditions, the split enumerator can inform the source reader through the source event.
The next layer is the operator coordinator, which communicates with the operator that performs tasks through operator events. Some operator events are predefined, such as adding splits and notifying the leader that there are no new splits. These events that work on all sources are abstracted at the operator event layer.
Address lookup determines the operator to which a message is sent. After the entire Flink job begins to be executed, there will be directed acyclic graphs one after another. Different operators may run on different task managers. It is down to Address lookup to find the corresponding tasks and operators.
Due to the network communication, there is a Remote Procedure Call (RPC) Gateway between the job master and the task executor. The network transmission between all events is achieved by means of RPC through this gateway.
The community has provided SourceReaderBase to simplify the implementation of source readers and reduce the work of developers. Users can directly inherit the SourceReaderBase class for development, which significantly simplifies the development work. Let's take a look at the SourceReaderBase. The source reader seems to have many components, as shown in the following figure, but it can be split into two parts for better understanding.
With the elementQueue in the middle as a barrier, the part marked in blue on the left is the component dealing with external systems. The part marked in orange on the right deals with the Flink engine side.
First, the left part contains one or more split readers. A fetcher drives each reader, and a fetcher manager manages multiple fetchers. The implementation here varies. For example, multiple partitions can be consumed by only one SplitReader on one thread. Or multiple threads can be enabled with each running one feature and one reader. Each reader consumes data in one partition in parallel. It's up to how users choose the implementation.
For performance, SplitReader extracts a batch of data from the external system each time and puts it into elementQueue. As shown in the figure, the blue box contains the batch of data extracted each time, and the orange box includes each piece of data in this batch.
On the right side of elementQueue are RecordEmitter and SourceOutput. RecordEmitter sends each record to the downstream SourceOutput for output. Each time RecordEmitter pulls a batch of data from the intermediate elementQueue and sends data to the downstream piece by piece. Note that the implementation must be non-blocking since RecordEmitter is driven by the main thread, which currently adopts a lock-free mailbox mode that divides the work to be executed into mails, and a mail is retrieved from the mailbox for work each time.
Each time RecordEmitter sends data to the downstream, it informs the downstream if there is any data to be processed subsequently. Meanwhile, the processing progress of the current split is recorded in SplitStates, including its current state and how it goes.
When SplitEnumerator finds a new split in the external system, it needs to call the addSplits method through RPC to add the new split to the reader. On the SplitFetchermanager side, the new split is allocated according to the threading model that the user has previously selected.
If there is only one thread, then the thread is assigned a new task, and the reader is assigned to read the new split. If there are multiple threads, a new thread is created, and a reader is also created to process the split separately. Similarly, the processing progress needs to be recorded in SplitStates.
Now, let's see how checkpoints are handled in the new Source API.
On the left is the coordinator, namely the split enumerator. As shown in the figure, it still has a split (Split_5) unassigned. On the arrows in the middle are some splits under transmission. Dotted lines indicate the checkpoint barrier. It can be seen that Split_2 is already ahead of the checkpoint, and Split_4 is behind the checkpoint.
The reader at the bottom is requesting a new split from SplitEnumerator. The three readers have been assigned some splits and processed them, and positions have been generated. Let's see what the enumerator and the reader need to store at the checkpoints.
The steps mentioned above deal with external systems, which means that the Flink engine itself is rarely involved. So users no longer need to worry about problems concerning checkpoint locks, multithreading, and so on. Instead, they can concentrate more on the development and the interaction with external systems. Therefore, the new Source API greatly simplifies the development through these abstractions.
Note that Flink features exactly once semantics without data repetition or data loss. For that, Flink has done much work, among which the two-phase commit at the Sink is of vital importance.
In the pre-commit phase, since the distributed system features "one coordinator with multiple executors" mode, the coordinator needs to request "commit" first. It needs to send a commit request message to all executors to start the entire two-phase commit.
When the executor receives the message, it will make some preparations for commit. After this, all executors will inform the coordinator that everything is ready for the next step. When the coordinator receives the request of "continue" from executors, the pre-commit phase ends. Next is the second phase, the commit phase.
The committer sends a message to the executors when it decides to commit. The executors will process the data prepared in order to execute the commit. After this, the executors will reply to the coordinator whether the commit is executed normally.
Once the coordinator decides to enter the commit phase, all executors must execute the order to the letter. Therefore, if there is something wrong with a coordinator at this phase, the commit still needs to be executed after the coordinator is recovered. Once the commit is determined, executors must carry out the commit all the way down.
If an executor fails to commit in the pre-commit phase, it may report an error, such as network disconnection, to the coordinator. Or, when the time is up and the coordinator does not receive the response request from executor three, the coordinator may trigger the rollback in the second phase. It will tell all executors that this commit fails and they need to roll back to the previous state. After that, the executors will roll back to undo the previous operation.
Take the Sink of this file system as an example.
After receiving the checkpoint barrier, the Sink of the file system performs the pre-commit operation (writing the current data to a temporary file on the hard disk). When the pre-commit phase is over, all operators will return the "ready for commit" message to the coordinator.
The commit is executed in the second phase. The JobManager sends a commit command to all operators. After receiving this command, the Sink performs the commit operation.
Again, take the file system as an example. As mentioned just now, the data is written to a temporary file in the pre-commit phase. Then, in the commit phase, the temporary file will be renamed according to the predefined name specification, thus implementing the commit operation.
Note that the temporary file is not useless. It plays a role in situations such as rollback. The two-phase commit mechanism is subtly used to ensure the exact-once semantics.
1) Writer: Continuously write data from the upstream to a certain intermediate state during the write or pre-commit phase.
2) Committable: The "intermediate state" mentioned above can perform the commit operation.
3) Committer: Commit the Committable.
4) Global Committer: An optional component depending on the external system, for example, Iceberg.
151 posts | 43 followers
FollowApache Flink Community China - August 19, 2021
Apache Flink Community China - April 23, 2020
Apache Flink Community China - August 4, 2021
Apache Flink Community China - June 2, 2022
Apache Flink Community China - May 18, 2022
Apache Flink Community China - April 19, 2022
151 posts | 43 followers
FollowAlibaba Cloud Function Compute is a fully-managed event-driven compute service. It allows you to focus on writing and uploading code without the need to manage infrastructure such as servers.
Learn MoreAlibaba Cloud provides big data consulting services to help enterprises leverage advanced data technology.
Learn MoreAlibaba Cloud experts provide retailers with a lightweight and customized big data consulting service to help you assess your big data maturity and plan your big data journey.
Learn MoreHigh Performance Computing (HPC) and AI technology helps scientific research institutions to perform viral gene sequencing, conduct new drug research and development, and shorten the research and development cycle.
Learn MoreMore Posts by Apache Flink Community