Author|Gao Yun (Yun Qian)
Click to enter Flink Chinese Learning Network
first part
Introduction
Flink can support distributed processing of both finite and infinite datasets. In the last few versions, Flink has gradually implemented the DataStream API and Table/SQL API that integrates streams and batches. Most users have both stream processing and batch processing requirements. The integrated development interface of stream and batch can help these users reduce the complexity of development, operation and maintenance, and ensure the consistency of the two types of job processing results. For example, Alibaba Double Ten A scene [1] .
Figure 1. Stream execution mode versus batch execution mode. Taking the Count operator as an example, in streaming mode, the arriving data is disordered, and the operator will read and write the state corresponding to the element and perform incremental calculations. In batch mode, the operator will sort the data first, and the data with the same key will be processed uniformly.
Under the interface of stream-batch integration, Flink provides two different execution modes, namely stream execution mode and batch execution mode. In the streaming execution mode, Flink processes the arriving data based on intermediate state increments, and it can support the processing of both limited data and unlimited data. The batch execution mode is based on executing all the tasks in the job in topological order, and avoids random access to the state by sorting the data in advance, so it can only be used for limited data processing, but in general, it can be achieved. good performance. Although in many scenarios users directly use batch mode to process limited data sets, there are also many scenarios where users still rely on stream processing mode to process limited data sets. For example, the user may want to use SQL's Retraction feature or the user may rely on the approximately time-ordered nature of data in streaming mode (eg Kappa+ schema [2] ). In addition, many users need to execute jobs that include both infinite data streams and finite-dimensional tables, which must also be executed in a streaming mode.
In streaming execution mode, Checkpointing [3] is the core mechanism to guarantee Exactly-once semantics. By periodically saving the state of a job, Flink can resume from the latest savepoint and continue execution when an error occurs. However, in previous versions, Flink did not support checkpointing after some tasks were executed. For jobs that include both infinite and finite data input, this issue will cause the job to not be able to continue Checkpointing after processing of the finite data input is complete, resulting in a recalculation from a long time ago when an error occurs.
In addition, the inability to perform Checkpoint operations on jobs after some tasks have ended will also affect jobs that use two-phase commit sinks to ensure end-to -end consistency [4] . In order to ensure end-to-end consistency, sinks with two-phase commit usually first write data to a temporary file or enable the transaction of an external system, and then commit the data written before the checkpoint after the checkpoint is successfully completed, thus avoiding replay after an error occurs This part of the data causes data duplication. However, if the job contains limited data sources, the job will not be able to continue submitting data after this part of the source node tasks ends. Especially in the case where all data sources are limited data sources, the job cannot submit this part of the data from the last Checkpoint to the end of the job execution. In the previous implementation, Flink directly ignored this part of the data at the end of the job, which caused great trouble for users, and many users were asking this question on the mailing list.
Therefore, in order to improve the streaming execution mode's support for limited data streams, Flink needs to:
- The Checkpoint operation continues after the support task is over.
- Modify the process of job end to ensure that all data can be submitted normally.
Below we will first briefly describe the changes made to these two goals. In the second part, we will also share a more detailed implementation.
Supports Checkpoints with end tasks
In general, the core idea of supporting the Checkpoint operation including the end task is to mark the completed operator, so that the execution of this part of the operator can be skipped after restarting. As shown in Figure 2, in Flink, Checkpoint is composed of the states of all operators. If all the concurrent execution of an operator has completed, then we can mark the operator as "execution complete" and skip it after restarting. For other operators, the state of the operator is composed of all concurrent states that are currently running. When restarted, the state of the operator will be re-divided among all concurrency.
Figure 2. Extended Checkpoint format
In order to complete the above Checkpoint operation when the task is completed, we have modified the process of the Checkpoint operation. When performing Checkpoint before, the Checkpoint controller in the JobManager will first notify all source nodes to save the current state, and then the source node will notify subsequent operators through the Barrier event. Since the source node may have completed execution now, the Checkpoint controller needs to instead notify those tasks that themselves have not finished execution, but that all predecessor tasks have completed execution. Finally, if all tasks of an operator have either become "completed" when starting Checkpoint, or have processed all data when saving the current state, the operator will be marked as "executed complete".
Except that we limit the modification of the job topology during job upgrade when there is indeed a task execution completed during Checkpoint, the above modification is transparent to the user. Specifically, we do not allow users to add new operators before an operator that is marked as "executed complete", because this will cause a "executed complete" operator to have a predecessor that has not been "executed complete", and This violates the semantics of operators ending in topological order in Flink.
Correct the flow of job end
Based on the above-mentioned ability to checkpoint jobs that contain end tasks, we can now solve the problem that operators with two-phase submission cannot submit the last part of data in streaming mode. In general, there are two possible ways for a Flink job to end:
- All data sources are limited, in which case the job ends after processing all input data and submitting all output to the external system.
- User explicitly performs stop-with-savepoint [--drain] action. In this case the job will end after creating a Savepoint. If --drain is specified, the job will end permanently, in which case the commit of temporary data from all external systems needs to be done. On the other hand, if this parameter is not specified, the job is expected to be restarted based on the savepoint in the future. In this case, it is not necessary to complete the submission of all temporary data, as long as the state recorded in the savepoint and the state of the temporary data in the external system are maintained. You can agree.
Let's start by looking at all cases where data sources are limited. In order to achieve end-to-end consistency, the operator using two-phase commit will only commit the data before the checkpoint after the checkpoint is completed. However, in the previous implementation, for the data generated from the last periodic Checkpoint to the end of the job execution, the job did not have a suitable opportunity to submit, resulting in data loss. It should be noted that it is not advisable to submit this part of the data directly at the end of the job: if a restart occurs due to an error in other tasks after a task is submitted, the data from the last Checkpoint will be deleted. replay, resulting in duplication of data.
There is also a problem in the case where the user stops the job with stop-with-savepoint [--drain]. In the previous implementation, Flink would first block all tasks and then create a Savepoint. After the Savepoint is successful, all data source tasks will actively stop running, thus ending the entire job execution. Although it seems that we can submit all the data through this Savepoint, in the current implementation, there are some logics that are actually executed during the process of job stop running. If these logics generate new data, these data will eventually be executed. will be lost. For example, in the previous implementation the endInput() method was executed while the job was stopped, and some operators may send data in this method, such as AsyncWaitOperator for asynchronous operations.
Finally, although the execution of stop-with-savepoint does not need to submit all data when the drain parameter is not specified, we still hope that the process of job end in this case can be unified with the first two cases, so as to ensure the maintainability of the code.
In order to solve the problems in the existing implementation, we need to modify the process of job end to ensure that all data can be guaranteed to be submitted when needed. As shown in Figure 3, an immediate idea is that we can add a step to the task life cycle and let the task wait for the next Checkpoint to complete before ending. However, as described below, this approach still does not solve all problems.
Figure 3. Comparison of two methods for ensuring that data submission is completed before the end of the task. The first method directly inserts a step in the life cycle of the task, that is, waits for the end of the next Checkpoint, but in this way, different tasks cannot wait for the same Checkpoint / Savepoint. The second way decouples "complete execution logic" and "task end", allowing all tasks to complete data processing first, and then they have a chance to wait for the same Checkpoint / Savepoint.
For all data sources are limited, this straightforward approach can solve the problem of unsubmitted data, but it can lead to serious performance problems. As shown in Figure 4, if there are multiple cascading tasks, each of which contains sinks submitted in two phases, then each task needs to wait for the next Checkpoint to complete before ending, so the entire job needs to wait for 3 Checkpoints to complete. end, which will have a larger impact on the execution time of the job.
Figure 4. An example of a multi-level task and each task contains a two-phase commit operator
For the case of stop-with-savepoint [--drain], this direct idea cannot be implemented, because in this case, because different tasks have to wait for different Checkpoint/Savepoint, the final job cannot get a complete Savepoint.
Therefore, we cannot adopt this direct idea. The way we use is to decouple "job completes all execution logic" from "job end": we first let all tasks complete all execution logic, including calling "endInput()" these life cycle methods, and then all tasks You can wait for the next Checkpoint / Savepoint in parallel. Also, for the stop-with-savepoint [--drain] case, we similarly reverse the current implementation: all tasks complete all execution logic first, and then they can wait for the next savepoint to complete. It can be seen that through this method, we can use the same process to unify the end of all jobs.
Based on this idea, as shown in the right half of Figure 3, in order to decouple "job completes all execution logic" and "job ends", we introduce a new EndOfData event. For each task, after all execution logic is completed, it will first send an EndOfData event to all downstreams, so that downstreams can also clearly infer that they have completed all execution logic. Then all tasks can wait in parallel for the completion of the next Checkpoint or the specified Savepoint. At this time, these tasks can submit all data to the external system and end.
Finally, during the modification process, we also reorganized and renamed the two operator lifecycle algorithms "close()" and "dispose()". The semantics of the two methods are different because close() is actually only called when the job ends normally, while dispose() is called on both normal and abnormal exits. However, it is difficult for users to see the semantics from these two names. Therefore, we renamed these two methods to "finish()" and "close()":
- finish() marks that all operators have been executed and no new data will be generated. Therefore, it will only be called when the job has ended normally and has been fully executed (ie, all data sources have finished executing or the user has used stop-with-savepoint --drain).
- close() is called in all cases to release resources occupied by the task.
the second part
In the first part above, we have briefly covered the work done to support Checkpoints with end tasks and to optimize the job end process. In this part, we will introduce more implementation details, including the specific process of Checkpoint at the end of the task and the specific process of the end of the job.
Checkpoint implementation with end task
As mentioned in the first part, the core idea of supporting Checkpoint operations including end tasks is to mark operators that have been fully executed, and to skip the execution of these operators after restarting. To implement this idea, we need to modify the current Checkpoint process to create these markers and use them when restoring. This section describes the detailed implementation of this process.
In the previous implementation, checkpointing was only possible when all tasks were running. As shown in Figure 5, in these cases the Checkpoint coordinator will first notify all data source tasks, and the data source tasks will continue to notify subsequent tasks after the completion state is saved. Similarly, in the case where the execution of some tasks ends, we need to first find the new "source tasks" in the currently running part, that is, those tasks that are running but all predecessor tasks have been completed, and then notify These tasks are used to start Checkpoint. The Checkpoint coordinator calculates the current list of "source tasks" in the latest state atomic of task records in the JobManaer.
There may be a state competition in the process of notifying these source tasks: when the Checkpoint coordinator selects a task for notification, the task may just finish executing and report the end status, causing the notification to fail. In this case, we choose to terminate this Checkpoint.
Figure 5. Trigger mode of Checkpoint after some tasks are over
In order to record the end state of the operator in Checkpoint, we need to extend the format of Checkpoint. A Checkpoint is composed of the states of all stateful operators, and the state of each operator is composed of the states of all its concurrent instances. It should be pointed out here that the concept of task (Task) is not reflected in Checkpoint. A task is more of a physical execution window that drives the execution of all concurrent instances of operators it contains. However, during multiple executions of a job, since the user may modify the job topology, so that the division of tasks changes, the tasks may not be in one-to-one correspondence between the two executions. Therefore, the end-of-execution marker needs to be attached to the operator state in the Checkpoint.
As shown in Figure 2 of the first part, operators can be divided into three categories according to the current execution state of the operator during Checkpointing:
- Complete execution end: If all concurrent instances of an operator are executed, the operator can be considered to be completely executed, and the execution of the operator can be skipped after restarting. We need to mark these operators.
- Partial execution ends: If a partial instance of an operator completes execution, it needs to continue executing the remaining logic after the job restarts. Overall we can think that the state of the operator in this case is composed of the states of all concurrent instances that are still executing, and these states can represent logic that has not yet been executed.
- No completed instance: In this case the operator state is the same as the existing implementation.
When subsequent jobs are restarted from the checkpoint, we can skip the operators that are completely executed and continue to execute the other two types of operators.
However, for operators whose partial execution ends, the actual situation is more complicated. On restart, the remaining state of the partial execution end operator will be redistributed to all instances, a process similar to the concurrent modification of the operator. For all types of states, the states of Keyed State [5] and normal Oeprator State [6] can be distributed normally, but Broacast State [7] and Union Operator State [8] have problems:
- Broadcast State always broadcasts the state of the first concurrent instance to all new concurrent instances after a restart. However, if the first concurrent instance has finished executing, its state will be empty, which will cause the state of all concurrent instances to become empty and the operator will be executed from scratch, which is not expected.
- Union Operator State aggregates the states of all operators and distributes them to all new concurrent instances after restarting. Based on this behavior, many operators may choose one of the concurrent instances to store state shared by all concurrent instances. Similarly, if the selected concurrent instance has finished executing, then this part of the state is lost.
In the scenario where this actually modifies concurrency, these two problems do not occur, because there are no subtasks that have been executed in this case. In order to solve the above problems, for Broadcast State, we choose any subtask of the running state as the source of the broadcast state; for Union Operator State, we need to ensure that the state of all subtasks can be collected, so currently if we observe a use If the execution of the operator part of the Union Operator State ends, we cancel this Checkpoint, and then wait until all subtasks of the operator are executed, and then the Checkpoint can continue.
In principle, the user can make modifications to the topology between executions. However, considering the end of the task, there are certain restrictions on topology modification: the user cannot add new operators before a completely ended operator. Flink will detect when the job is restarted and report an error if there are such modifications.
Revised job end flow
As mentioned in the first part, based on the ability to continue checkpointing after some tasks are over, we can revise the existing job end process to ensure that the two-phase submission operator can always submit data normally. This section will describe the end process before and after modification in detail.
The original job end process
As mentioned earlier, the job end includes two cases: all data sources end or the user executes stop-with-savepoint --drain. Let's first look at the previous job end process.
End of all data sources
If all data sources are limited, the job will end after all data processing is complete and all data needs to be submitted. In this case, the data source task will first send a MAX_WATERMARK
( Long.MAX_VALUE
) and then start the end task. During the end process, the task will sequentially call endOfInput(), close(), and dispose() on all operators, and then send an EndOfPartitionEvent event downstream. Subsequent tasks will also start executing the end process after reading the EndOfPartitionEvent event in all input edges, and this process is repeated until all tasks end.
- Source operators emit MAX_WATERMARK
On received MAX_WATERMARK for non-source operators
a. Trigger all the event-time timers
b. Emit MAX_WATERMARK
Source tasks finished
a. endInput(inputId) for all the operators
b. close() for all the operators
c. dispose() for all the operators
d. Emit EndOfPartitionEvent
e. Task cleanup
On received EndOfPartitionEvent for non-source tasks
a. endInput(int inputId) for all the operators
b. close() for all the operators
c. dispose() for all the operators
d. Emit EndOfPartitionEvent
e. Task cleanup
User executes stop-with-savepoint --drain
Users can execute a stop-with-savepoint [--drain] operation on a finite or infinite dataflow job to end the job. In this case, the job will first trigger a synchronous savepoint operation and block all tasks until the savepoint completes. If the Savepoint completes successfully, all data source tasks will actively execute the end process, and the subsequent process is similar to the case where all data sources are limited.
- Trigger a savepoint
Sources received savepoint trigger RPC
a. If with –-drain
i. source operators emit MAX_WATERMARK
b. Source emits savepoint barrier
On received MAX_WATERMARK for non-source operators
a. Trigger all the event times
b. Emit MAX_WATERMARK
On received savepoint barrier for non-source operators
a. The task blocks till the savepoint succeed
Finish the source tasks actively
a. If with –-drain
ii. endInput(inputId) for all the operators
b. close() for all the operators
c. dispose() for all the operators
d. Emit EndOfPartitionEvent
e. Task cleanup
On received EndOfPartitionEvent for non-source tasks
a. If with –-drain
i. endInput(int inputId) for all the operators
b. close() for all the operators
c. dispose() for all the operators
d. Emit EndOfPartitionEvent e. Task cleanup
The command has an optional --drain parameter, if this parameter is not specified, subsequent jobs can resume execution from the Savepoint, otherwise the user expects the job to end permanently. Therefore, the job will send MAX_WATERMARK and call endInput() on all operators only if the user specifies this parameter.
Job end flow after correction
As mentioned in the first part, in the revised end process, we decoupled "task completion execution logic" and "task end" by adding a new EndOfData event. Each task will first send an EndOfData event downstream after completing all execution logic, so that downstream tasks can also complete all execution logic first, and then all tasks can wait in parallel for the next Checkpoint or the specified Savepoint to complete submitting all data.
This section will describe the revised execution flow in detail. Since we renamed the close() / dispose() methods finish() / close(), we will stick to this terminology in subsequent descriptions.
The revised execution flow is as follows:
Source tasks finished due to no more records or stop-with-savepoint.
a. if no more records or stop-with-savepoint –-drain
i. source operators emit MAX_WATERMARK
ii. endInput(inputId) for all the operators
iii. finish() for all the operators
iv. emit EndOfData[isDrain = true] event
b. else if stop-with-savepoint
i. emit EndOfData[isDrain = false] event
c. Wait for the next checkpoint / the savepoint after operator finished complete
d. close() for all the operators
e. Emit EndOfPartitionEvent
f. Task cleanup
On received MAX_WATERMARK for non-source operators
a. Trigger all the event times
b. Emit MAX_WATERMARK
On received EndOfData for non-source tasks
a.If isDrain
i. endInput(int inputId) for all the operators
ii. finish() for all the operators
b. Emit EndOfData[isDrain = the flag value of the received event]
On received EndOfPartitionEvent for non-source tasks
a. Wait for the next checkpoint / the savepoint after operator finished complete
b. close() for all the operators
c. Emit EndOfPartitionEvent
d. Task cleanup
Figure 6. An example of a job using the revised end process
An example is shown in Figure 6. Let's first look at the case where all data sources are finite.
If task C ends first after processing all the data, it will first send MAX_WATERMARK, then execute the corresponding end lifecycle method for all operators and send the EndOfData event. After this, it first waits for the next Checkpoint to complete and then sends the EndOfPartitionEvent event.
After receiving the EndOfData event, task D will first execute the end-of-life cycle method corresponding to the end of the operator. Since any Checkpoint that starts after the operator execution ends can submit the remaining data, and the Barrier event of the Checkpoint that task C relies on to submit data arrives after the EndOfData event, task D can actually be completed with the same Checkpoint as task C. data submission.
Task E is slightly different because it has two inputs, while task A may continue to run for a while. Therefore, task E must wait until it reads the EndOfData event from both inputs before it can start to end the operator execution, and it needs to rely on a different Checkpoint to complete the data submission.
On the other hand, when stop-with-savepoint [--drain] is used to end the job, the entire process is the same as when the data source is limited, except that instead of waiting for an arbitrary next Checkpoint, all tasks wait for a specified Savepoint to arrive. Complete data submission. In addition, in this case, since task C and task A must end at the same time, we can guarantee that task E can also wait for this specific Savepoint before ending.
in conclusion
By supporting Checkpoint jobs after some tasks end and revising the process of job end, we can support jobs that use limited data sources and unlimited data sources at the same time, and can ensure that the last part of data can be submitted normally when all data sources are limited . This part of the modification guarantees data consistency and end integrity, and supports error recovery for jobs with limited data sources. This mechanism was primarily implemented in 1.14, and is turned on by default in 1.15. If you run into any problems, you are welcome to start discussions or ask questions on the dev or user/user-zh mailing lists.
[1] https://www.ververica.com/blog/apache-flinks-stream-batch-unification-powers-alibabas-11.11-in-2020
[2] https://www.youtube.com/watch?t=666&v=4qSlsYogALo&feature=youtu.be
[4] https://flink.apache.org/features/2018/03/01/end-to-end-exactly-once-apache-flink.html
Click to enter Flink Chinese Learning Network
For more technical issues related to Flink, you can scan the code to join the community DingTalk exchange group to get the latest technical articles and community dynamics as soon as possible. Please pay attention to the public number~
Recommended activities
Alibaba Cloud's enterprise-level product based on Apache Flink - real-time computing Flink version is now open:
99 yuan to try out the Flink version of real-time computing (yearly and monthly, 10CU), and you will have the opportunity to get Flink's exclusive custom sweater; another package of 3 months and above will have a 15% discount!
Learn more about the event: https://www.aliyun.com/product/bigdata/en
**粗体** _斜体_ [链接](http://example.com) `代码` - 列表 > 引用
。你还可以使用@
来通知其他用户。