NOTE: This document is a design draft and does not reflect the state of a stable release, but the work in progress on the current master, and future considerations.
This document is intended to give an overview of how operator state is handled and represented in streaming programs.
State access patterns in streaming programs
To design a properly functioning, scalable state representation we need to understand the different state access and handling patterns that applications need in streaming programs. We can distinguish three main state types: local state, partitioned state, out-of-core state.
Local (non-partitioned) state
Local or non-partitioned state is the simplest form of operator state which represent the current state of a specific operator instance in a parallel streaming operator. Local states stored at different operator instances do not interact with each other. For instance if we have a mapper with parallelism of 10 that means each parallel instance holds it’s own local state. An important thing to note here is that state updates reflected in a particular local state will depend only on the input of that specific operator instance. As there is no strict control over the input which each operator instance processes (only via stream partitioning), the operator itself needs to deal with this non-deterministic nature of the inputs and this also limits expressivity. Typical usage of non-partitioned state includes source operators (storing offsets), oany global aggregation/summary or analysis operators where the local results will be merged into a global result afterwards. Scaling out non-partitioned state can in most cases be done by just starting a new operator instance with a blank state, or a user supplied splitting function can be used to split the state of an existing instance. For reducing job parallelism the user should generally provide a function that can merge 2 local states to maintain correct results. |
Alternatively, the user can also use the Checkpointed interface for implementing local state functionality. This interface gives more fine-grained control over the checkpoint/restore process:
Partitioned state
Partitioned state provides a representation for operator states that are tied to partitions (keys) of the operator input. An independent state is maintained for each partition and operator states for different keys don't interact. Partitioning is done by some user defined key which is extracted from each input and inputs with the same key share the state. Big advantage of partitioned state is both expressivity and trivial scalability. In case of partitioned state, the operator instances need not to be aware of the partitioning of the inputs as they can only access the state for the current input and the system guarantees proper partitioning by the selected key. This makes the implementation of per-key operators very simple. Operations using partitioned state can also benefit from the partial ordering guarantees that the flink runtime provides, to implement deterministic behaviour. Furthermore partitioned state can easily be scaled automatically by moving some partitions to new operators and changing the input partitioning. Typical usage includes any per key stateful operation, such as group aggregates/summaries, or analysis over several distinct groups in the stream for instance pattern detection. |
Partitioned state is only accessible from operators applied on KeyedDataStreams (the state partitioning will be done by the key in the stream):
Out-of-core state