Skip to content

Commit

Permalink
[FLINK-15999][doc] Remove distinction between managed and raw state f…
Browse files Browse the repository at this point in the history
…rom state.md

Users should not be using raw state, except in rare edge cases. We can
just refer to managed keyed state ad keyed state and managed operator
state as operator state from now on.
  • Loading branch information
aljoscha committed Feb 21, 2020
1 parent 1c1acf7 commit e1d09e4
Showing 1 changed file with 6 additions and 26 deletions.
32 changes: 6 additions & 26 deletions docs/dev/stream/state/state.md
Original file line number Diff line number Diff line change
Expand Up @@ -59,29 +59,9 @@ of topic partitions and offsets as its Operator State.
The Operator State interfaces support redistributing state among
parallel operator instances when the parallelism is changed. There can be different schemes for doing this redistribution.

## Raw and Managed State
## Using Keyed State

*Keyed State* and *Operator State* exist in two forms: *managed* and *raw*.

*Managed State* is represented in data structures controlled by the Flink runtime, such as internal hash tables, or RocksDB.
Examples are "ValueState", "ListState", etc. Flink's runtime encodes
the states and writes them into the checkpoints.

*Raw State* is state that operators keep in their own data structures. When checkpointed, they only write a sequence of bytes into
the checkpoint. Flink knows nothing about the state's data structures and sees only the raw bytes.

All datastream functions can use managed state, but the raw state interfaces can only be used when implementing operators.
Using managed state (rather than raw state) is recommended, since with
managed state Flink is able to automatically redistribute state when the parallelism is
changed, and also do better memory management.

<span class="label label-danger">Attention</span> If your managed state needs custom serialization logic, please see
the [corresponding guide](custom_serialization.html) in order to ensure future compatibility. Flink's default serializers
don't need special treatment.

## Using Managed Keyed State

The managed keyed state interface provides access to different types of state that are all scoped to
The keyed state interfaces provides access to different types of state that are all scoped to
the key of the current input element. This means that this type of state can only be used
on a `KeyedStream`, which can be created via `stream.keyBy(…)`.

Expand Down Expand Up @@ -255,7 +235,7 @@ object ExampleCountWindowAverage extends App {
.print()
// the printed output will be (1,4) and (1,5)

env.execute("ExampleManagedState")
env.execute("ExampleKeyedState")
}
{% endhighlight %}
</div>
Expand Down Expand Up @@ -545,9 +525,9 @@ val counts: DataStream[(String, Int)] = stream
})
{% endhighlight %}

## Using Managed Operator State
## Using Operator State

To use managed operator state, a stateful function can implement either the more general `CheckpointedFunction`
To use operator state, a stateful function can implement either the more general `CheckpointedFunction`
interface, or the `ListCheckpointed<T extends Serializable>` interface.

#### CheckpointedFunction
Expand All @@ -566,7 +546,7 @@ is called every time the user-defined function is initialized, be that when the
or be that when the function is actually recovering from an earlier checkpoint. Given this, `initializeState()` is not
only the place where different types of state are initialized, but also where state recovery logic is included.

Currently, list-style managed operator state is supported. The state
Currently, list-style operator state is supported. The state
is expected to be a `List` of *serializable* objects, independent from each other,
thus eligible for redistribution upon rescaling. In other words, these objects are the finest granularity at which
non-keyed state can be redistributed. Depending on the state accessing method,
Expand Down

0 comments on commit e1d09e4

Please sign in to comment.