diff --git a/docs/layouts/shortcodes/generated/forst_configuration.html b/docs/layouts/shortcodes/generated/forst_configuration.html
index c0ab2ef24a221..704d549635b90 100644
--- a/docs/layouts/shortcodes/generated/forst_configuration.html
+++ b/docs/layouts/shortcodes/generated/forst_configuration.html
@@ -16,15 +16,15 @@
state.backend.forst.cache.reserve-size |
- -1 |
- Long |
- The reserved size of cache, when set to a positive number. Meaning that the cache will reserve the specified size of disk space. This option and the 'state.backend.forst.cache.size-based-limit' option can be set simultaneously, the smaller cache limit will be used as the upper limit. |
+ 0 bytes |
+ MemorySize |
+ The reserved size of cache, when set to a positive number. Meaning that the cache will reserve the specified size of disk space. This option and the 'state.backend.forst.cache.size-based-limit' option can be set simultaneously, the smaller cache limit will be used as the upper limit. The default value is '0 bytes', meaning the cache will not reserve any disk space. |
state.backend.forst.cache.size-based-limit |
- -1 |
- Long |
- The size-based capacity limit of cache, a non-positive number indicates that there is no limit. The default value is '-1', which means that the cache size is not limited by size. |
+ 0 bytes |
+ MemorySize |
+ The size-based capacity limit of cache.The default value is '0 bytes', which means that the cache size is not limited by size. |
state.backend.forst.executor.inline-coordinator |
diff --git a/docs/layouts/shortcodes/generated/state_backend_forst_section.html b/docs/layouts/shortcodes/generated/state_backend_forst_section.html
index 38505dbd2156d..f784985b60a47 100644
--- a/docs/layouts/shortcodes/generated/state_backend_forst_section.html
+++ b/docs/layouts/shortcodes/generated/state_backend_forst_section.html
@@ -16,15 +16,15 @@
state.backend.forst.cache.reserve-size |
- -1 |
- Long |
- The reserved size of cache, when set to a positive number. Meaning that the cache will reserve the specified size of disk space. This option and the 'state.backend.forst.cache.size-based-limit' option can be set simultaneously, the smaller cache limit will be used as the upper limit. |
+ 0 bytes |
+ MemorySize |
+ The reserved size of cache, when set to a positive number. Meaning that the cache will reserve the specified size of disk space. This option and the 'state.backend.forst.cache.size-based-limit' option can be set simultaneously, the smaller cache limit will be used as the upper limit. The default value is '0 bytes', meaning the cache will not reserve any disk space. |
state.backend.forst.cache.size-based-limit |
- -1 |
- Long |
- The size-based capacity limit of cache, a non-positive number indicates that there is no limit. The default value is '-1', which means that the cache size is not limited by size. |
+ 0 bytes |
+ MemorySize |
+ The size-based capacity limit of cache.The default value is '0 bytes', which means that the cache size is not limited by size. |
state.backend.forst.executor.read-io-parallelism |
diff --git a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStOptions.java b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStOptions.java
index 5eb8c1cebda6f..cdf5ee6db0383 100644
--- a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStOptions.java
+++ b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStOptions.java
@@ -27,6 +27,7 @@
import org.apache.flink.configuration.description.Description;
import org.apache.flink.configuration.description.TextElement;
+import static org.apache.flink.configuration.description.TextElement.text;
import static org.apache.flink.state.forst.ForStStateBackend.CHECKPOINT_DIR_AS_PRIMARY_SHORTCUT;
import static org.apache.flink.state.forst.ForStStateBackend.LOCAL_DIR_AS_PRIMARY_SHORTCUT;
import static org.apache.flink.state.forst.ForStStateBackend.PriorityQueueStateType.ForStDB;
@@ -77,25 +78,34 @@ public class ForStOptions {
LOCAL_DIRECTORIES.key()));
@Documentation.Section(Documentation.Sections.STATE_BACKEND_FORST)
- public static final ConfigOption CACHE_SIZE_BASE_LIMIT =
+ public static final ConfigOption CACHE_SIZE_BASE_LIMIT =
ConfigOptions.key("state.backend.forst.cache.size-based-limit")
- .longType()
- .defaultValue(-1L)
+ .memoryType()
+ .defaultValue(MemorySize.ZERO)
.withDescription(
- "The size-based capacity limit of cache, a non-positive number indicates that there is no limit. "
- + "The default value is '-1', which means that the cache size is not limited by size.");
+ Description.builder()
+ .text(
+ "The size-based capacity limit of cache."
+ + "The default value is '%s', which means that the cache size is not limited by size.",
+ text(MemorySize.ZERO.toString()))
+ .build());
@Documentation.Section(Documentation.Sections.STATE_BACKEND_FORST)
- public static final ConfigOption CACHE_RESERVED_SIZE =
+ public static final ConfigOption CACHE_RESERVED_SIZE =
ConfigOptions.key("state.backend.forst.cache.reserve-size")
- .longType()
- .defaultValue(-1L)
- .withDescription(
- "The reserved size of cache, when set to a positive number. Meaning that "
- + "the cache will reserve the specified size of disk space. "
- + "This option and the '"
- + CACHE_SIZE_BASE_LIMIT.key()
- + "' option can be set simultaneously, the smaller cache limit will be used as the upper limit.");
+ .memoryType()
+ .defaultValue(MemorySize.ZERO)
+ .withDescription(
+ Description.builder()
+ .text(
+ "The reserved size of cache, when set to a positive number. Meaning that "
+ + "the cache will reserve the specified size of disk space. "
+ + "This option and the '%s' option can be set simultaneously, the "
+ + "smaller cache limit will be used as the upper limit. "
+ + "The default value is '%s', meaning the cache will not reserve any disk space.",
+ text(CACHE_SIZE_BASE_LIMIT.key()),
+ text(MemorySize.ZERO.toString()))
+ .build());
/** The options factory class for ForSt to create DBOptions and ColumnFamilyOptions. */
@Documentation.Section(Documentation.Sections.EXPERT_FORST)
diff --git a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStResourceContainer.java b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStResourceContainer.java
index 4b6bf545bd175..e4e9d9c3bb654 100644
--- a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStResourceContainer.java
+++ b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStResourceContainer.java
@@ -189,8 +189,8 @@ public ForStResourceContainer(
this.enableStatistics = enableStatistics;
this.handlesToClose = new ArrayList<>();
this.cacheBasePath = configuration.getOptional(CACHE_DIRECTORY).map(Path::new).orElse(null);
- this.cacheCapacity = configuration.get(CACHE_SIZE_BASE_LIMIT);
- this.cacheReservedSize = configuration.get(CACHE_RESERVED_SIZE);
+ this.cacheCapacity = configuration.get(CACHE_SIZE_BASE_LIMIT).getBytes();
+ this.cacheReservedSize = configuration.get(CACHE_RESERVED_SIZE).getBytes();
this.metricGroup = metricGroup;
}