diff --git a/docs/content.zh/docs/ops/state/state_backends.md b/docs/content.zh/docs/ops/state/state_backends.md
index 685fe236cd7ab..27161874a4216 100644
--- a/docs/content.zh/docs/ops/state/state_backends.md
+++ b/docs/content.zh/docs/ops/state/state_backends.md
@@ -46,6 +46,7 @@ Flink 内置了以下这些开箱即用的 state backends :
- *HashMapStateBackend*
- *EmbeddedRocksDBStateBackend*
+ - *ForStStateBackend*
如果不设置,默认使用 HashMapStateBackend。
@@ -97,6 +98,44 @@ EmbeddedRocksDBStateBackend 是目前唯一支持增量 CheckPoint 的 State Bac
每个 slot 中的 RocksDB instance 的内存大小是有限制的,详情请见 [这里]({{< ref "docs/ops/state/large_state_tuning" >}})。
+
+
+### The ForStStateBackend
+
+The *ForStStateBackend* is a state backend that is based on [ForSt project](https://github.com/ververica/ForSt),
+which is also a LSM-tree structured key-value store and built on top of the RocksDB.
+It is designed for disaggregated state management, for more details, see [here]({{< ref "docs/ops/state/disaggregated_state" >}}).
+Most importantly, it can hold its sst files on remote file systems that Flink supports, such as HDFS, S3, etc.
+This allows Flink to scale the state size beyond the local disk capacity of the TaskManager.
+Moreover, by putting the sst files on remote file systems, it can also provide a more lightweight
+way to perform checkpoint and recovery.
+
+The ForStStateBackend is still in the experimental stage and is not fully available for production.
+It always performs asynchronous incremental snapshots.
+
+The ForStStateBackend is encouraged for:
+
+- Jobs with very large state, long windows, large key/value states. Local disk may not be enough to
+store the state.
+- All high-availability setups.
+- Asynchronous state access is preferred. Since the ForStStateBackend is the only one supporting
+asynchronous state access.
+- Jobs that require lightweight checkpoint and recovery, such as cloud-native applications.
+
+Limitations of the ForStStateBackend (for now):
+
+- Same as EmbeddedRocksDBStateBackend, the maximum supported size per key and per value is 2^31 bytes each.
+- Does not support canonical savepoint, full snapshot, changelog and file-merging checkpoints.
+Always perform incremental snapshots.
+
+Compared with EmbeddedRocksDBStateBackend, ForStStateBackend stores data on remote file system, thus
+the amount of state that you can keep is unlimited. The local disk of TaskManager is only used to
+store cache of file, to provide better performance. Note that when most of the active state is on
+remote file system, the performance of state access may be affected by the network latency. Flink
+introduces asynchronous state access to mitigate this issue. If you are using the asynchronous state
+methods in State API V2, you can benefit from the asynchronous state access. To get familiar with the
+State API V2, please refer to the [State API V2 documentation]({{< ref "docs/dev/datastream/fault-tolerance/state_v2" >}}).
+
## 选择合适的 State Backend
diff --git a/docs/content/docs/ops/state/state_backends.md b/docs/content/docs/ops/state/state_backends.md
index 6f04e495882f4..3db1e0bdd4579 100644
--- a/docs/content/docs/ops/state/state_backends.md
+++ b/docs/content/docs/ops/state/state_backends.md
@@ -45,6 +45,7 @@ Out of the box, Flink bundles these state backends:
- *HashMapStateBackend*
- *EmbeddedRocksDBStateBackend*
+ - *ForStStateBackend*
If nothing else is configured, the system will use the HashMapStateBackend.
@@ -95,7 +96,7 @@ Certain RocksDB native metrics are available but disabled by default, you can fi
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.
-## The ForStStateBackend
+### The ForStStateBackend
The *ForStStateBackend* is a state backend that is based on [ForSt project](https://github.com/ververica/ForSt),
which is also a LSM-tree structured key-value store and built on top of the RocksDB.
@@ -299,7 +300,7 @@ See [configuration docs]({{< ref "docs/deployment/config" >}}#rocksdb-native-met
Enabling RocksDB's native metrics may have a negative performance impact on your application.
{{< /hint >}}
-### Advanced RocksDB Memory Turning
+### Advanced RocksDB Memory Tuning
{{< hint info >}}
Flink offers sophisticated default [memory management for RocksDB](#memory-management) that should work for most use-cases. The below mechanisms should mainly be used for *expert tuning* or *trouble shooting*.