flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-5937) Add documentation about the task lifecycle.
Date Tue, 28 Feb 2017 16:01:46 GMT

    [ https://issues.apache.org/jira/browse/FLINK-5937?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15888317#comment-15888317
] 

ASF GitHub Bot commented on FLINK-5937:
---------------------------------------

Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3429#discussion_r103486405
  
    --- Diff: docs/internals/task_lifecycle.md ---
    @@ -0,0 +1,149 @@
    +---
    +title:  "Task Lifecycle"
    +nav-title: Task Lifecycle
    +nav-parent_id: internals
    +nav-pos: 5
    +---
    +<!--
    +Licensed to the Apache Software Foundation (ASF) under one
    +or more contributor license agreements.  See the NOTICE file
    +distributed with this work for additional information
    +regarding copyright ownership.  The ASF licenses this file
    +to you under the Apache License, Version 2.0 (the
    +"License"); you may not use this file except in compliance
    +with the License.  You may obtain a copy of the License at
    +
    +  http://www.apache.org/licenses/LICENSE-2.0
    +
    +Unless required by applicable law or agreed to in writing,
    +software distributed under the License is distributed on an
    +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    +KIND, either express or implied.  See the License for the
    +specific language governing permissions and limitations
    +under the License.
    +-->
    +
    +A task in Flink is the basic unit of execution. This is the place where each parallel
instance of your operators gets executed. 
    +
    +This document goes through the different phases in the lifecycle of the `StreamTask`,
which is the base for all different task sub-types in Flink's streaming engine. 
    +
    +* This will be replaced by the TOC
    +{:toc}
    +
    +## Operator Lifecycle in a nutshell
    +
    +As the task is the entity that executes your operators, its lifecycle is tightly integrated
with that of an operator. Given this, it is worth spending a few lines 
    +simply mentioning the basic methods representing the lifecycle of an operator before
diving into those of the `StreamTask` itself. The list is presented below in 
    +the order that they are called. These are:
    +
    +        // initialization
    +	    initializeState()
    +	    setup()
    +	    open()
    +	
    +	    // processing
    +	    processElement()
    +	    processWatermark()
    +	
    +	    // termination
    +	    close()
    +	    dispose()
    +
    +        // checkpointing
    +        snapshotState()
    +    
    +In a nutshell, the `initializeState()` gives an operator its initial state. After obtaining
it, the `setup()` is called to initialize some operator specific 
    +machinery, such as its `RuntimContext` and its metric collection data-structures, and
the `open()` executes any operator-specific initialization, such as
    +opening the user-defined function in the case of the `AbstractUdfStreamOperator`. 
    +
    +Now that everything is set, the operator is ready to process fresh incoming data. This
is done by invoking the `processElement()` and `processWatermark()` methods which
    +contain the logic for processing elements and watermark respectively.
    +
    +Finally, in the case of normal, fault-free termination of the operator (*e.g.* if the
stream is finite and its end is reached), the `close()` method is called to 
    +perform any final bookkeeping action required by the operator's logic, and the `dispose()`
is called after that to free any resources held by the operator. 
    +
    +In the case of termination due to a failure or due to manual cancellation, the execution
jumps directly to the `dispose()`, and skips any intermediate phases between
    +the phase the operator was in when the failure happened and the `dispose` one.
    +
    +**Checkpoints:** The `snapshotState()` method of the operator is called asynchronously
to the rest of the methods described above, whenever a checkpoint barrier is received. 
    +Its responsibility is to store the current state of the operator to the specified [state
backend]({{ site.baseurl }}/ops/state_backends.html) from where it is going to be 
    +retrieved when the job resumes execution after a failure. For a bried description of
Flink's checkpointing mechanism please keep on reading, and for a more detailed discussion

    +on the principles around checkpointing in Flink please read the corresponding documentation:
[Data Streaming Fault Tolerance]({{ site.baseurl }}/internals/stream_checkpointing.html).
    +
    +## Task Lifecycle
    +
    +Given the above brief introduction on the operator's main phases, this section describes
in more detail how a task calls them during its execution on a cluster. The sequence 
    +of the phases described here is mainly included in the `invoke()` method of the `StreamTask`.
The remainder of this document is split into two subsections, one describing the 
    +phases during a regular, fault-free execution of a task (see [Normal Execution](#normal-execution)),
and (a shorter) one describing the different sequence followed in case 
    +the task is cancelled (see [Interrupted Execution](#interrupted-execution)), 
    +either manually, or due some other reason, *e.g.* an exception thrown during execution.
    +
    +### Normal Execution
    +
    +The steps a task goes through when executed until completion without being interrupted
are illustrated below:
    +
    +	setInitialState()
    +	invoke()
    +		Create basic utils (config, etc) and load the chain of operators
    +		setup-operators()
    + 		task specific init()
    + 		initialize-operator-states()
    + 		open-operators()
    + 		run()
    + 		close-operators()
    + 		dispose-operators()
    + 		task specific cleanup()
    + 		common cleanup
    +
    +As shown above, after recovering the task configuration and initializing some important
runtime parameters, the very first step for the task is to retrieve its initial, 
    +task-wide state. This is done in the `setInitialState()`, and it is particularly important
in two cases:
    +
    +1. when the task is recovering from a failure and restarts from the last successful checkpoint,
and
    +2. when resuming from a [savepoint]({{ site.baseurl }}/setup/savepoints.html). 
    +
    +If it is the first time the task is executed, the initial task state is empty. 
    +
    +After recovering any initial state, the task goes into its `invoke()` method. There,
it first initializes the operators involved in the local computation by calling 
    +the `setup()` method of each one of them and then performs its task-specific initialization
by calling the local `init()` method. By task-specific, we mean that 
    +depending on the type of the task (`SourceTask`, `OneInputStreamTask` or `TwoInputStreamTask`,
etc), this step may differ, but in any case, here is where the necessary 
    +task-wide resources are acquired. As an example, the `OneInputStreamTask` which represents
a task that expects to have a single input stream, initializes the connection(s) 
    +to the location(s) of the different partitions of the input stream that are relevant
to the local task.
    +
    +Having acquired the necessary resources, it is time for the different operators and user-defined
functions to acquire their individual state from the task-wide state 
    +retrieved above. This is done in the `initializeState()` method, which calls the `initializeState()`
of each individual operator. This method should be overriden by 
    +every stateful operator and should contain the state initialization logic, both for the
first time a job is executed, but also for the case when the task recovers from 
    +a failure or using a savepoint.
    +
    +Now that all operators in the task have been initialized, the `open()` method of each
individual operator is called by the `openAllOperators()` method of the `StreamTask`. 
    +This is the place that implements all the operational initialization, such as register
any retrieved timers with the timer service. A single task may be executing multiple 
    +operators with one consuming the output of its predecessor. In this case, the `open()`
method is called from the last operator, *i.e.* the one whose output is also the output 
    +of the task itself, to the first. This is done so that when the first operator starts
processing the task's input, all downstream operators are ready to receive its output.
    +
    +Now the task can resume execution, and operators can start processing fresh input data.
This is the place where the task-specific `run()` mehtod is called. This method will 
    +run until either there is no more input data (finite stream), or the task is cancelled
(manually or not). Here is where the operator specific `processElement()` and 
    +`processWatermark()` methods are called.
    +
    +In the case of running till completion, *i.e.* there is no more input data to process,
after exiting from the `run()` method, the task enters its shutdown process. 
    +Initially, the timer service stops registering any new timers (e.g. from fired timers
that are being executed), clears all the not yet started timers, and awaits the 
    +completion of currently executing ones. Then the `closeAllOperators()` tries to gracefully
close the operators involved in the computation by calling the `close()` method 
    +of each one of them. Then any buffered output data is flushed so that they can be processed
by the downstream tasks, and finally the task tries to clear all the resources 
    +held by the operators by calling the `dispose()` method of each one of them. When opening
the different operators, we mentioned that the order is from the last to the first. 
    +Closing happens the other way round, from first to last.
    +
    +Finally, when all operators have been closed and all their resources freed, the task
shuts down its timer service, performs its task-specific cleanup, *e.g.* clean all its 
    +internal buffers, and then performs its generic task clean up which consists of closing
all its output channels and cleaning any output buffers.
    +
    +**Checkpoints:** Previously we saw that during `initializeState()`, and in case of recovering
from a failure, the task and all its operators and functions retrieve the 
    +state that was persisted to stable storage during the last successful checkpoint before
the failure. Checkpoints in Flink are performed periodically based on a user-specified 
    +interval, and are performed by a different thread than that of the main task thread.
This is the reason the are not included in the main phases of the task lifecycle. In a 
    +nutshell, special elements called `CheckpointBarriers` are injected periodically by the
source tasks of a job in the stream of input data, and travel with the actual data from 
    +source to sink. A source task injects these barriers after it is in running mode, and
assuming that the CheckpointCoordinator is also running. Whenever a task receives such a 
    --- End diff --
    
    CheckpointCoordinator -> `CheckpointCoordinator`?


> Add documentation about the task lifecycle.
> -------------------------------------------
>
>                 Key: FLINK-5937
>                 URL: https://issues.apache.org/jira/browse/FLINK-5937
>             Project: Flink
>          Issue Type: Bug
>          Components: Documentation
>    Affects Versions: 1.3.0
>            Reporter: Kostas Kloudas
>            Assignee: Kostas Kloudas
>             Fix For: 1.3.0
>
>




--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Mime
View raw message