You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
Copy file name to clipboardExpand all lines: docs/content.zh/docs/deployment/config.md
+28-2
Original file line number
Diff line number
Diff line change
@@ -160,7 +160,7 @@ These values are configured as memory sizes, for example *1536m* or *2g*.
160
160
161
161
You can configure checkpointing directly in code within your Flink job or application. Putting these values here in the configuration defines them as defaults in case the application does not configure anything.
162
162
163
-
-`state.backend.type`: The state backend to use. This defines the data structure mechanism for taking snapshots. Common values are `hashmap`or `rocksdb`.
163
+
-`state.backend.type`: The state backend to use. This defines the data structure mechanism for taking snapshots. Common values are `hashmap`, `rocksdb`or `forst`.
164
164
-`execution.checkpointing.dir`: The directory to write checkpoints to. This takes a path URI like *s3://mybucket/flink-app/checkpoints* or *hdfs://namenode:port/flink/checkpoints*.
165
165
-`execution.checkpointing.savepoint-dir`: The default directory for savepoints. Takes a path URI, similar to `execution.checkpointing.dir`.
166
166
-`execution.checkpointing.interval`: The base interval setting. To enable checkpointing, you need to set this value larger than 0.
@@ -352,6 +352,12 @@ These are the options commonly needed to configure the RocksDB state backend. Se
352
352
353
353
{{< generated/state_backend_rocksdb_section >}}
354
354
355
+
### ForSt State Backend
356
+
357
+
These are the options commonly needed to configure the ForSt state backend. See the [Advanced ForSt Backend Section](#advanced-forst-state-backends-options) for options necessary for advanced low level configurations and trouble-shooting.
358
+
359
+
{{< generated/state_backend_forst_section >}}
360
+
355
361
----
356
362
----
357
363
@@ -374,6 +380,16 @@ Enabling RocksDB's native metrics may cause degraded performance and should be s
These options give fine-grained control over the behavior and resources of ColumnFamilies.
490
512
With the introduction of `state.backend.rocksdb.memory.managed` and `state.backend.rocksdb.memory.fixed-per-slot` (Apache Flink 1.10), it should be only necessary to use the options here for advanced performance tuning. These options here can also be specified in the application program via `RocksDBStateBackend.setRocksDBOptions(RocksDBOptionsFactory)`.
Copy file name to clipboardExpand all lines: docs/content.zh/docs/ops/metrics.md
+6-1
Original file line number
Diff line number
Diff line change
@@ -1541,7 +1541,12 @@ Note that for failed checkpoints, metrics are updated on a best efforts basis an
1541
1541
### RocksDB
1542
1542
Certain RocksDB native metrics are available but disabled by default, you can find full documentation [here]({{< ref "docs/deployment/config" >}}#rocksdb-native-metrics)
1543
1543
1544
-
### ForStDB
1544
+
### ForSt
1545
+
1546
+
Certain ForSt native metrics are available but disabled by default, you can find full documentation [here]({{< ref "docs/deployment/config" >}}#forst-native-metrics)
Copy file name to clipboardExpand all lines: docs/content/docs/deployment/config.md
+28-2
Original file line number
Diff line number
Diff line change
@@ -160,7 +160,7 @@ These values are configured as memory sizes, for example *1536m* or *2g*.
160
160
161
161
You can configure checkpointing directly in code within your Flink job or application. Putting these values here in the configuration defines them as defaults in case the application does not configure anything.
162
162
163
-
-`state.backend.type`: The state backend to use. This defines the data structure mechanism for taking snapshots. Common values are `hashmap`or `rocksdb`.
163
+
-`state.backend.type`: The state backend to use. This defines the data structure mechanism for taking snapshots. Common values are `hashmap`, `rocksdb`or `forst`.
164
164
-`execution.checkpointing.dir`: The directory to write checkpoints to. This takes a path URI like *s3://mybucket/flink-app/checkpoints* or *hdfs://namenode:port/flink/checkpoints*.
165
165
-`execution.checkpointing.savepoint-dir`: The default directory for savepoints. Takes a path URI, similar to `execution.checkpointing.dir`.
166
166
-`execution.checkpointing.interval`: The base interval setting. To enable checkpointing, you need to set this value larger than 0.
@@ -354,6 +354,12 @@ These are the options commonly needed to configure the RocksDB state backend. Se
354
354
355
355
{{< generated/state_backend_rocksdb_section >}}
356
356
357
+
### ForSt State Backend
358
+
359
+
These are the options commonly needed to configure the ForSt state backend. See the [Advanced ForSt Backend Section](#advanced-forst-state-backends-options) for options necessary for advanced low level configurations and trouble-shooting.
360
+
361
+
{{< generated/state_backend_forst_section >}}
362
+
357
363
----
358
364
----
359
365
@@ -376,6 +382,16 @@ Enabling RocksDB's native metrics may cause degraded performance and should be s
These options give fine-grained control over the behavior and resources of ColumnFamilies.
492
514
With the introduction of `state.backend.rocksdb.memory.managed` and `state.backend.rocksdb.memory.fixed-per-slot` (Apache Flink 1.10), it should be only necessary to use the options here for advanced performance tuning. These options here can also be specified in the application program via `RocksDBStateBackend.setRocksDBOptions(RocksDBOptionsFactory)`.
Copy file name to clipboardExpand all lines: docs/content/docs/ops/metrics.md
+6-1
Original file line number
Diff line number
Diff line change
@@ -1531,7 +1531,12 @@ Note that for failed checkpoints, metrics are updated on a best efforts basis an
1531
1531
### RocksDB
1532
1532
Certain RocksDB native metrics are available but disabled by default, you can find full documentation [here]({{< ref "docs/deployment/config" >}}#rocksdb-native-metrics)
1533
1533
1534
-
### ForStDB
1534
+
### ForSt
1535
+
1536
+
Certain ForSt native metrics are available but disabled by default, you can find full documentation [here]({{< ref "docs/deployment/config" >}}#forst-native-metrics)
Copy file name to clipboardExpand all lines: docs/content/docs/ops/state/state_backends.md
+53-4
Original file line number
Diff line number
Diff line change
@@ -95,12 +95,50 @@ Certain RocksDB native metrics are available but disabled by default, you can fi
95
95
96
96
The total memory amount of RocksDB instance(s) per slot can also be bounded, please refer to documentation [here]({{< ref "docs/ops/state/large_state_tuning" >}}#bounding-rocksdb-memory-usage) for details.
97
97
98
+
## The ForStStateBackend
99
+
100
+
The *ForStStateBackend* is a state backend that is based on [ForSt project](https://github.com/ververica/ForSt),
101
+
which is also a LSM-tree structured key-value store and built on top of the RocksDB.
102
+
It is designed to provide a more efficient way to store and access state in Flink applications.
103
+
Most importantly, it can hold its sst files on remote file systems that Flink supports, such as HDFS, S3, etc.
104
+
This allows Flink to scale the state size beyond the local disk capacity of the TaskManager.
105
+
Moreover, by putting the sst files on remote file systems, it can also provide a more lightweight
106
+
way to perform checkpoint and recovery.
107
+
108
+
The ForStStateBackend is still in the experimental stage and is not fully available for production.
109
+
It always performs asynchronous incremental snapshots.
110
+
111
+
The ForStStateBackend is encouraged for:
112
+
113
+
- Jobs with very large state, long windows, large key/value states. Local disk may not be enough to
114
+
store the state.
115
+
- All high-availability setups.
116
+
- Asynchronous state access is preferred. Since the ForStStateBackend is the only one supporting
117
+
asynchronous state access.
118
+
- Jobs that require lightweight checkpoint and recovery, such as cloud-native applications.
119
+
120
+
Limitations of the ForStStateBackend (for now):
121
+
122
+
- Same as EmbeddedRocksDBStateBackend, the maximum supported size per key and per value is 2^31 bytes each.
123
+
- Does not support canonical savepoint, full snapshot, changelog and file-merging checkpoints.
124
+
Always perform incremental snapshots.
125
+
126
+
Compared with EmbeddedRocksDBStateBackend, ForStStateBackend stores data on remote file system, thus
127
+
the amount of state that you can keep is unlimited. The local disk of TaskManager is only used to
128
+
store cache of file, to provide better performance. Note that when most of the active state is on
129
+
remote file system, the performance of state access may be affected by the network latency. Flink
130
+
introduces asynchronous state access to mitigate this issue. If you are using the asynchronous state
131
+
methods in State API V2, you can benefit from the asynchronous state access. To get familiar with the
132
+
State API V2, please refer to the [State API V2 documentation]({{< ref "docs/dev/datastream/fault-tolerance/state_v2" >}}).
133
+
98
134
## Choose The Right State Backend
99
135
100
136
When deciding between `HashMapStateBackend` and `RocksDB`, it is a choice between performance and scalability.
101
137
`HashMapStateBackend` is very fast as each state access and update operates on objects on the Java heap; however, state size is limited by available memory within the cluster.
102
-
On the other hand, `RocksDB` can scale based on available disk space and is the only state backend to support incremental snapshots.
138
+
On the other hand, `RocksDB` can scale based on available disk space.
103
139
However, each state access and update requires (de-)serialization and potentially reading from disk which leads to average performance that is an order of magnitude slower than the memory state backends.
140
+
If you are handling very large state even exceeding the available disk space,
141
+
or you prefer a fast rescale under cloud-native setup, you should consider using `ForStStateBackend`.
104
142
105
143
{{< hint info >}}
106
144
In Flink 1.13 we unified the binary format of Flink's savepoints. That means you can take a savepoint and then restore from it using a different state backend.
@@ -150,18 +188,29 @@ If you want to use the `EmbeddedRocksDBStateBackend` in your IDE or configure it
150
188
</dependency>
151
189
```
152
190
191
+
Same for `ForStStateBackend`:
192
+
```xml
193
+
<dependency>
194
+
<groupId>org.apache.flink</groupId>
195
+
<artifactId>flink-statebackend-forst</artifactId>
196
+
<version>{{< version >}}</version>
197
+
<scope>provided</scope>
198
+
</dependency>
199
+
```
200
+
153
201
{{< hint info >}}
154
-
Since RocksDB is part of the default Flink distribution, you do not need this dependency if you are not using any RocksDB code in your job and configure the state backend via `state.backend.type` and further [checkpointing]({{< ref "docs/deployment/config" >}}#checkpointing) and [RocksDB-specific]({{< ref "docs/deployment/config" >}}#rocksdb-state-backend) parameters in your [Flink configuration file]({{< ref "docs/deployment/config#flink-configuration-file" >}}).
202
+
Since RocksDB and ForSt is part of the default Flink distribution, you do not need this dependency if you are not using any RocksDB code in your job and configure the state backend via `state.backend.type` and further [checkpointing]({{< ref "docs/deployment/config" >}}#checkpointing) and [RocksDB-specific]({{< ref "docs/deployment/config" >}}#rocksdb-state-backend) or [ForSt-specific]({{< ref "docs/deployment/config" >}}#forst-state-backend) parameters in your [Flink configuration file]({{< ref "docs/deployment/config#flink-configuration-file" >}}).
155
203
{{< /hint >}}
156
204
157
205
158
206
### Setting Default State Backend
159
207
160
208
A default state backend can be configured in the [Flink configuration file]({{< ref "docs/deployment/config#flink-configuration-file" >}}), using the configuration key `state.backend.type`.
161
209
162
-
Possible values for the config entry are *hashmap* (HashMapStateBackend), *rocksdb* (EmbeddedRocksDBStateBackend), or the fully qualified class
210
+
Possible values for the config entry are *hashmap* (HashMapStateBackend), *rocksdb* (EmbeddedRocksDBStateBackend), *forst* (ForStStateBackend) or the fully qualified class
163
211
name of the class that implements the state backend factory {{< gh_link file="flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackendFactory.java" name="StateBackendFactory" >}},
164
-
such as `org.apache.flink.state.rocksdb.EmbeddedRocksDBStateBackendFactory` for EmbeddedRocksDBStateBackend.
212
+
such as `org.apache.flink.state.rocksdb.EmbeddedRocksDBStateBackendFactory` for EmbeddedRocksDBStateBackend
213
+
and `org.apache.flink.state.forst.ForStStateBackendFactory` for ForStStateBackend.
165
214
166
215
The `execution.checkpointing.dir` option defines the directory to which all backends write checkpoint data and meta data files.
167
216
You can find more details about the checkpoint directory structure [here]({{< ref "docs/ops/state/checkpoints" >}}#directory-structure).
<td>Whether to let the task thread be the coordinator thread responsible for distributing requests. If set to 'true', the task thread will be responsible for distributing requests, otherwise, a dedicated coordinator thread will be used. The default value is 'false'.</td>
<td>Whether to let write requests be executed within the coordinator thread. If set to 'true', write requests will be executed within the coordinator thread, otherwise, a dedicated write thread will be used. The default value is 'true'.</td>
22
+
</tr>
23
+
<tr>
24
+
<td><h5>state.backend.forst.local-dir</h5></td>
25
+
<tdstyle="word-wrap: break-word;">(none)</td>
26
+
<td>String</td>
27
+
<td>The local directory (on the TaskManager) where ForSt puts some metadata files. By default, it will be <WORKING_DIR>/tmp. See <codeclass="highlighter-rouge">process.taskmanager.working-dir</code> for more details.</td>
<td>The fixed total amount of memory per slot, shared among all ForSt instances.This option overrides the 'state.backend.forst.memory.managed' option.</td>
<td>The fixed total amount of memory per Task Manager, shared among all ForSt instances. This is a cluster-level option. This option only takes effect if 'state.backend.forst.memory.managed' is set to false and 'state.backend.forst.memory.fixed-per-slot' is not configured. If so, then each ForSt column family state has its own memory caches (as controlled by the column family options). The relevant options for the shared resources (e.g. write-buffer-ratio) can be set on the same level (config.yaml). Note that this feature breaks resource isolation between the slots.</td>
<td>If set true, the ForSt state backend will automatically configure itself to use the managed memory budget of the task slot, and divide the memory over write buffers, indexes, block caches, etc.</td>
<td>The options factory class for users to add customized options in DBOptions and ColumnFamilyOptions for ForSt. If set, the ForSt state backend will load the class and apply configs to DBOptions and ColumnFamilyOptions after loading ones from 'ForStConfigurableOptions' and pre-defined options.</td>
0 commit comments