This is an automated email from the ASF dual-hosted git repository.
leiyanfei pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git
The following commit(s) were added to refs/heads/master by this push:
new 1d8cbe9065e [FLINK-34483][docs] Improve the documentation of
'state.checkpoints.dir' and 'state.checkpoint-storage' (#24401)
1d8cbe9065e is described below
commit 1d8cbe9065e2ef34a748513b2469ecfe3626639e
Author: Yanfei Lei <[email protected]>
AuthorDate: Fri Mar 8 17:28:11 2024 +0800
[FLINK-34483][docs] Improve the documentation of 'state.checkpoints.dir'
and 'state.checkpoint-storage' (#24401)
---
.../generated/checkpointing_configuration.html | 4 ++--
.../generated/common_state_backends_section.html | 4 ++--
.../flink/configuration/CheckpointingOptions.java | 27 ++++++++++++++++++++--
3 files changed, 29 insertions(+), 6 deletions(-)
diff --git a/docs/layouts/shortcodes/generated/checkpointing_configuration.html
b/docs/layouts/shortcodes/generated/checkpointing_configuration.html
index 03979d7d6c9..2c40c7e1804 100644
--- a/docs/layouts/shortcodes/generated/checkpointing_configuration.html
+++ b/docs/layouts/shortcodes/generated/checkpointing_configuration.html
@@ -24,7 +24,7 @@
<td><h5>state.checkpoint-storage</h5></td>
<td style="word-wrap: break-word;">(none)</td>
<td>String</td>
- <td>The checkpoint storage implementation to be used to checkpoint
state.<br />The implementation can be specified either via their shortcut
name, or via the class name of a <code
class="highlighter-rouge">CheckpointStorageFactory</code>. If a factory is
specified it is instantiated via its zero argument constructor and its <code
class="highlighter-rouge">CheckpointStorageFactory#createFromConfig(ReadableConfig,
ClassLoader)</code> method is called.<br />Recognized shortcut [...]
+ <td>The checkpoint storage implementation to be used to checkpoint
state.<br />The implementation can be specified either via their shortcut
name, or via the class name of a <code
class="highlighter-rouge">CheckpointStorageFactory</code>. If a factory is
specified it is instantiated via its zero argument constructor and its <code
class="highlighter-rouge">CheckpointStorageFactory#createFromConfig(ReadableConfig,
ClassLoader)</code> method is called.<br />Recognized shortcut [...]
</tr>
<tr>
<td><h5>state.checkpoint.cleaner.parallel-mode</h5></td>
@@ -42,7 +42,7 @@
<td><h5>state.checkpoints.dir</h5></td>
<td style="word-wrap: break-word;">(none)</td>
<td>String</td>
- <td>The default directory used for storing the data files and meta
data of checkpoints in a Flink supported filesystem. The storage path must be
accessible from all participating processes/nodes(i.e. all TaskManagers and
JobManagers).</td>
+ <td>The default directory used for storing the data files and meta
data of checkpoints in a Flink supported filesystem. The storage path must be
accessible from all participating processes/nodes(i.e. all TaskManagers and
JobManagers). If the 'state.checkpoint-storage' is set to 'jobmanager', only
the meta data of checkpoints will be stored in this directory.</td>
</tr>
<tr>
<td><h5>state.checkpoints.num-retained</h5></td>
diff --git
a/docs/layouts/shortcodes/generated/common_state_backends_section.html
b/docs/layouts/shortcodes/generated/common_state_backends_section.html
index 157cb75c81f..7d35697849c 100644
--- a/docs/layouts/shortcodes/generated/common_state_backends_section.html
+++ b/docs/layouts/shortcodes/generated/common_state_backends_section.html
@@ -18,13 +18,13 @@
<td><h5>state.checkpoint-storage</h5></td>
<td style="word-wrap: break-word;">(none)</td>
<td>String</td>
- <td>The checkpoint storage implementation to be used to checkpoint
state.<br />The implementation can be specified either via their shortcut
name, or via the class name of a <code
class="highlighter-rouge">CheckpointStorageFactory</code>. If a factory is
specified it is instantiated via its zero argument constructor and its <code
class="highlighter-rouge">CheckpointStorageFactory#createFromConfig(ReadableConfig,
ClassLoader)</code> method is called.<br />Recognized shortcut [...]
+ <td>The checkpoint storage implementation to be used to checkpoint
state.<br />The implementation can be specified either via their shortcut
name, or via the class name of a <code
class="highlighter-rouge">CheckpointStorageFactory</code>. If a factory is
specified it is instantiated via its zero argument constructor and its <code
class="highlighter-rouge">CheckpointStorageFactory#createFromConfig(ReadableConfig,
ClassLoader)</code> method is called.<br />Recognized shortcut [...]
</tr>
<tr>
<td><h5>state.checkpoints.dir</h5></td>
<td style="word-wrap: break-word;">(none)</td>
<td>String</td>
- <td>The default directory used for storing the data files and meta
data of checkpoints in a Flink supported filesystem. The storage path must be
accessible from all participating processes/nodes(i.e. all TaskManagers and
JobManagers).</td>
+ <td>The default directory used for storing the data files and meta
data of checkpoints in a Flink supported filesystem. The storage path must be
accessible from all participating processes/nodes(i.e. all TaskManagers and
JobManagers). If the 'state.checkpoint-storage' is set to 'jobmanager', only
the meta data of checkpoints will be stored in this directory.</td>
</tr>
<tr>
<td><h5>state.savepoints.dir</h5></td>
diff --git
a/flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java
b/flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java
index 6b7d9268443..8cbec13c2cb 100644
---
a/flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java
+++
b/flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java
@@ -76,6 +76,17 @@ public class CheckpointingOptions {
* CheckpointStorageFactory#createFromConfig(ReadableConfig, ClassLoader)}
method is called.
*
* <p>Recognized shortcut names are 'jobmanager' and 'filesystem'.
+ *
+ * <p>{@link #CHECKPOINT_STORAGE} and {@link #CHECKPOINTS_DIRECTORY} are
usually combined to
+ * configure the checkpoint location. By default, the checkpoint meta data
and actual program
+ * state will be stored in the JobManager's memory directly.
+ * <li>When {@link #CHECKPOINT_STORAGE} is set to 'jobmanager', if {@link
+ * #CHECKPOINTS_DIRECTORY} is configured, the meta data of checkpoints
will be persisted to
+ * the path specified by {@link #CHECKPOINTS_DIRECTORY}. Otherwise,
the meta data will be
+ * stored in the JobManager's memory.
+ * <li>When {@link #CHECKPOINT_STORAGE} is set to 'filesystem', a valid
path must be configured
+ * to {@link #CHECKPOINTS_DIRECTORY}, and the checkpoint meta data and
actual program state
+ * will both be persisted to the path.
*/
@Documentation.Section(value =
Documentation.Sections.COMMON_STATE_BACKENDS, position = 2)
public static final ConfigOption<String> CHECKPOINT_STORAGE =
@@ -99,6 +110,15 @@ public class CheckpointingOptions {
.linebreak()
.text(
"Recognized shortcut names are
'jobmanager' and 'filesystem'.")
+ .linebreak()
+ .text(
+ "'state.checkpoint-storage' and
'state.checkpoints.dir' are usually combined to configure the checkpoint
location."
+ + " By default, the
checkpoint meta data and actual program state will be stored in the
JobManager's memory directly."
+ + " When
'state.checkpoint-storage' is set to 'jobmanager', if 'state.checkpoints.dir'
is configured,"
+ + " the meta data of
checkpoints will be persisted to the path specified by 'state.checkpoints.dir'."
+ + " Otherwise, the meta
data will be stored in the JobManager's memory."
+ + " When
'state.checkpoint-storage' is set to 'filesystem', a valid path must be
configured to 'state.checkpoints.dir',"
+ + " and the checkpoint
meta data and actual program state will both be persisted to the path.")
.build());
/** The maximum number of completed checkpoints to retain. */
@@ -215,7 +235,8 @@ public class CheckpointingOptions {
/**
* The default directory used for storing the data files and meta data of
checkpoints in a Flink
* supported filesystem. The storage path must be accessible from all
participating
- * processes/nodes(i.e. all TaskManagers and JobManagers).
+ * processes/nodes(i.e. all TaskManagers and JobManagers). If {@link
#CHECKPOINT_STORAGE} is set
+ * to 'jobmanager', only the meta data of checkpoints will be stored in
this directory.
*/
@Documentation.Section(value =
Documentation.Sections.COMMON_STATE_BACKENDS, position = 2)
public static final ConfigOption<String> CHECKPOINTS_DIRECTORY =
@@ -226,7 +247,9 @@ public class CheckpointingOptions {
.withDescription(
"The default directory used for storing the data
files and meta data of checkpoints "
+ "in a Flink supported filesystem. The
storage path must be accessible from all participating processes/nodes"
- + "(i.e. all TaskManagers and
JobManagers).");
+ + "(i.e. all TaskManagers and
JobManagers). If the '"
+ + CHECKPOINT_STORAGE.key()
+ + "' is set to 'jobmanager', only the meta
data of checkpoints will be stored in this directory.");
/**
* Whether to create sub-directories named by job id to store the data
files and meta data of