Page properties |
---|
Document the state by adding a label to the FLIP page with one of "discussion", "accepted", "released", "rejected". |
...
|
...
JIRA:
Released: -
|
Motivation
Apache Flink's durability story is a mystery to many users. One of the most common recurring questions from users comes from not understanding the relationship between state, state backends, and snapshots. Some of this confusion can be abated with learning material. Still, the question is so pervasive that we believe Flink's user APIs should be better communicate with what different components are responsible.
...
Code Block | ||
---|---|---|
| ||
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setStateBackend(new HashMapStateBackend()) EmbeddedRocksDBStateBackend rocksDB = new EmbeddedRocksDBStateBackend(); rocksDB.setOptionsFactory(/** blah **/); env.setStateBackend(rocksDB); env.setDefaultSavepointDirectory("s3://savepoint"); env.getCheckpointConfig().setCheckpointStorage(new JobManagerCheckpointStorage()); env.getCheckpointConfig().setCheckpointStorage(new FileSystemCheckpointStorage("s3://checkpoints")); // shortcut for env.getCheckpointConfig().setCheckpointStorage(new FileSystemCheckpointStorage("s3://checkpoints")); env.getCheckpointConfig().setCheckpointStorage("s3://checkpoints"); |
The default state backend will be HashMapStateBackend and default checkpoint storage will be JobManagerCheckpointStorage. This is equivalent to the semantics of MemoryStateBackend which is the default today.
...
Old | New |
---|---|
MemoryStateBackend() | HashMapStateBackend() + JobManagerCheckpointStorage() |
FsStateBackend() | HashMapStateBackend() + FileSystemCheckpointStorage() |
RocksDBStateBackend(new MemoryStateBackend()) | EmbeddedRocksDBStateBackend() + JobManagerCheckpointStorage() |
RocksDBStateBackend(new FsStateBackend()) | EmbeddedRocksDBStateBackend() + FileSystemCheckpointStorage() |
MemoryStateBackend("file://path") | HashMapStateBackend() + JobManagerCheckpointStorage("file://path") |
All flink-conf configurations will be duplicated for the new state backend or snapshot storage instance, whichever is appropriate. Again, no functionality will be added or dropped in this change. Existing flink-conf keys will also be specified as deprecated keys on the new state backends to ease migration.
...