Skip to content

Commit

Permalink
[hotfix][state/forst] Make file cache options recognize size type (#2…
Browse files Browse the repository at this point in the history
  • Loading branch information
Zakelly authored Feb 13, 2025
1 parent 7881dc3 commit 0898eb5
Show file tree
Hide file tree
Showing 4 changed files with 38 additions and 28 deletions.
12 changes: 6 additions & 6 deletions docs/layouts/shortcodes/generated/forst_configuration.html
Original file line number Diff line number Diff line change
Expand Up @@ -16,15 +16,15 @@
</tr>
<tr>
<td><h5>state.backend.forst.cache.reserve-size</h5></td>
<td style="word-wrap: break-word;">-1</td>
<td>Long</td>
<td>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.</td>
<td style="word-wrap: break-word;">0 bytes</td>
<td>MemorySize</td>
<td>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.</td>
</tr>
<tr>
<td><h5>state.backend.forst.cache.size-based-limit</h5></td>
<td style="word-wrap: break-word;">-1</td>
<td>Long</td>
<td>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.</td>
<td style="word-wrap: break-word;">0 bytes</td>
<td>MemorySize</td>
<td>The size-based capacity limit of cache.The default value is '0 bytes', which means that the cache size is not limited by size.</td>
</tr>
<tr>
<td><h5>state.backend.forst.executor.inline-coordinator</h5></td>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,15 +16,15 @@
</tr>
<tr>
<td><h5>state.backend.forst.cache.reserve-size</h5></td>
<td style="word-wrap: break-word;">-1</td>
<td>Long</td>
<td>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.</td>
<td style="word-wrap: break-word;">0 bytes</td>
<td>MemorySize</td>
<td>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.</td>
</tr>
<tr>
<td><h5>state.backend.forst.cache.size-based-limit</h5></td>
<td style="word-wrap: break-word;">-1</td>
<td>Long</td>
<td>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.</td>
<td style="word-wrap: break-word;">0 bytes</td>
<td>MemorySize</td>
<td>The size-based capacity limit of cache.The default value is '0 bytes', which means that the cache size is not limited by size.</td>
</tr>
<tr>
<td><h5>state.backend.forst.executor.read-io-parallelism</h5></td>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -77,25 +78,34 @@ public class ForStOptions {
LOCAL_DIRECTORIES.key()));

@Documentation.Section(Documentation.Sections.STATE_BACKEND_FORST)
public static final ConfigOption<Long> CACHE_SIZE_BASE_LIMIT =
public static final ConfigOption<MemorySize> 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<Long> CACHE_RESERVED_SIZE =
public static final ConfigOption<MemorySize> 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)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
}

Expand Down

0 comments on commit 0898eb5

Please sign in to comment.