[FLINK-8475][config][docs] Integrate JM options

This closes #5392.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/a0e961df
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/a0e961df
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/a0e961df

Branch: refs/heads/master
Commit: a0e961dff70627e02e559c526ba8fbb0943c56b4
Parents: 10ad8c3
Author: zentol <ches...@apache.org>
Authored: Mon Jan 22 17:32:38 2018 +0100
Committer: zentol <ches...@apache.org>
Committed: Mon Feb 12 18:48:58 2018 +0100

----------------------------------------------------------------------
 .../generated/job_manager_configuration.html    | 76 ++++++++++++++++++++
 docs/ops/config.md                              | 12 ++--
 .../flink/configuration/JobManagerOptions.java  | 32 +++++++--
 3 files changed, 107 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/a0e961df/docs/_includes/generated/job_manager_configuration.html
----------------------------------------------------------------------
diff --git a/docs/_includes/generated/job_manager_configuration.html 
b/docs/_includes/generated/job_manager_configuration.html
new file mode 100644
index 0000000..13b9faa
--- /dev/null
+++ b/docs/_includes/generated/job_manager_configuration.html
@@ -0,0 +1,76 @@
+<table class="table table-bordered">
+    <thead>
+        <tr>
+            <th class="text-left" style="width: 20%">Key</th>
+            <th class="text-left" style="width: 15%">Default</th>
+            <th class="text-left" style="width: 65%">Description</th>
+        </tr>
+    </thead>
+    <tbody>
+        <tr>
+            <td><h5>jobmanager.archive.fs.dir</h5></td>
+            <td>(none)</td>
+            <td></td>
+        </tr>
+        <tr>
+            <td><h5>jobmanager.execution.attempts-history-size</h5></td>
+            <td>16</td>
+            <td>The maximum number of prior execution attempts kept in 
history.</td>
+        </tr>
+        <tr>
+            <td><h5>jobmanager.execution.failover-strategy</h5></td>
+            <td>"full"</td>
+            <td>The maximum number of prior execution attempts kept in 
history.</td>
+        </tr>
+        <tr>
+            <td><h5>jobmanager.heap.mb</h5></td>
+            <td>1024</td>
+            <td>JVM heap size (in megabytes) for the JobManager.</td>
+        </tr>
+        <tr>
+            <td><h5>jobmanager.resourcemanager.reconnect-interval</h5></td>
+            <td>2000</td>
+            <td>This option specifies the interval in order to trigger a 
resource manager reconnection if the connection to the resource manager has 
been lost. This option is only intended for internal use.</td>
+        </tr>
+        <tr>
+            <td><h5>jobmanager.rpc.address</h5></td>
+            <td>(none)</td>
+            <td>The config parameter defining the network address to connect 
to for communication with the job manager. This value is only interpreted in 
setups where a single JobManager with static name or address exists (simple 
standalone setups, or container setups with dynamic service name resolution). 
It is not used in many high-availability setups, when a leader-election service 
(like ZooKeeper) is used to elect and discover the JobManager leader from 
potentially multiple standby JobManagers.</td>
+        </tr>
+        <tr>
+            <td><h5>jobmanager.rpc.port</h5></td>
+            <td>6123</td>
+            <td>The config parameter defining the network port to connect to 
for communication with the job manager. Like jobmanager.rpc.address, this value 
is only interpreted in setups where a single JobManager with static 
name/address and port exists (simple standalone setups, or container setups 
with dynamic service name resolution). This config option is not used in many 
high-availability setups, when a leader-election service (like ZooKeeper) is 
used to elect and discover the JobManager leader from potentially multiple 
standby JobManagers.</td>
+        </tr>
+        <tr>
+            <td><h5>jobstore.cache-size</h5></td>
+            <td>52428800</td>
+            <td>The job store cache size in bytes which is used to keep 
completed jobs in memory.</td>
+        </tr>
+        <tr>
+            <td><h5>jobstore.expiration-time</h5></td>
+            <td>3600</td>
+            <td>The time in seconds after which a completed job expires and is 
purged from the job store.</td>
+        </tr>
+        <tr>
+            <td><h5>slot.allocation.resourcemanager.timeout</h5></td>
+            <td>300000</td>
+            <td>The timeout in milliseconds for allocation a slot from 
Resource Manager.</td>
+        </tr>
+        <tr>
+            <td><h5>slot.idle.timeout</h5></td>
+            <td>300000</td>
+            <td>The timeout in milliseconds for a idle slot in Slot Pool.</td>
+        </tr>
+        <tr>
+            <td><h5>slot.request.resourcemanager.timeout</h5></td>
+            <td>10000</td>
+            <td>The timeout in milliseconds for sending a request to Resource 
Manager.</td>
+        </tr>
+        <tr>
+            <td><h5>slot.request.timeout</h5></td>
+            <td>600000</td>
+            <td>The timeout in milliseconds for requesting a slot from Slot 
Pool.</td>
+        </tr>
+    </tbody>
+</table>

http://git-wip-us.apache.org/repos/asf/flink/blob/a0e961df/docs/ops/config.md
----------------------------------------------------------------------
diff --git a/docs/ops/config.md b/docs/ops/config.md
index b02ab95..b94f212 100644
--- a/docs/ops/config.md
+++ b/docs/ops/config.md
@@ -262,13 +262,14 @@ These parameters configure the default HDFS used by 
Flink. Setups that do not sp
 
 - `fs.hdfs.hdfssite`: The absolute path of Hadoop's own configuration file 
"hdfs-site.xml" (DEFAULT: null).
 
-### JobManager &amp; TaskManager
+### JobManager
 
-The following parameters configure Flink's JobManager and TaskManagers.
+{% include generated/job_manager_configuration.html %}
 
-- `jobmanager.rpc.address`: The external address of the JobManager, which is 
the master/coordinator of the distributed system (DEFAULT: **localhost**). 
**Note:** The address (host name or IP) should be accessible by all nodes 
including the client.
+### TaskManager
+
+The following parameters configure Flink's TaskManagers.
 
-- `jobmanager.rpc.port`: The port number of the JobManager (DEFAULT: **6123**).
 
 - `taskmanager.hostname`: The hostname of the network interface that the 
TaskManager binds to. By default, the TaskManager searches for network 
interfaces that can connect to the JobManager and other TaskManagers. This 
option can be used to define a hostname if that strategy fails for some reason. 
Because different TaskManagers need different values for this option, it 
usually is specified in an additional non-shared TaskManager-specific config 
file.
 
@@ -278,8 +279,6 @@ The following parameters configure Flink's JobManager and 
TaskManagers.
 
 - `taskmanager.data.ssl.enabled`: Enable SSL support for the taskmanager data 
transport. This is applicable only when the global ssl flag 
security.ssl.enabled is set to true (DEFAULT: **true**)
 
-- `jobmanager.heap.mb`: JVM heap size (in megabytes) for the JobManager 
(DEFAULT: **256**).
-
 - `taskmanager.heap.mb`: JVM heap size (in megabytes) for the TaskManagers, 
which are the parallel workers of the system. In contrast to Hadoop, Flink runs 
operators (e.g., join, aggregate) and user-defined functions (e.g., Map, 
Reduce, CoGroup) inside the TaskManager (including sorting/hashing/caching), so 
this value should be as large as possible (DEFAULT: **512**). On YARN setups, 
this value is automatically configured to the size of the TaskManager's YARN 
container, minus a certain tolerance value.
 
 - `taskmanager.numberOfTaskSlots`: The number of parallel operator or user 
function instances that a single TaskManager can run (DEFAULT: **1**). If this 
value is larger than 1, a single TaskManager takes multiple instances of a 
function or operator. That way, the TaskManager can utilize multiple CPU cores, 
but at the same time, the available memory is divided between the different 
operator or function instances. This value is typically proportional to the 
number of physical CPU cores that the TaskManager's machine has (e.g., equal to 
the number of cores, or half the number of cores).
@@ -322,7 +321,6 @@ The following parameters configure Flink's JobManager and 
TaskManagers.
 
 - `taskmanager.exit-on-fatal-akka-error`: Whether the TaskManager shall be 
terminated in case of a fatal Akka error (quarantining event). (DEFAULT: 
**false**)
 
-- `jobmanager.tdd.offload.minsize`: Maximum size of the 
`TaskDeploymentDescriptor`'s serialized task and job information to still 
transmit them via RPC. Larger blobs may be offloaded to the BLOB server. 
(DEFAULT: **1 KiB**).
 
 ### Distributed Coordination (via Akka)
 

http://git-wip-us.apache.org/repos/asf/flink/blob/a0e961df/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java
----------------------------------------------------------------------
diff --git 
a/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java
 
b/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java
index a9f3673..a6df986 100644
--- 
a/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java
+++ 
b/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java
@@ -40,7 +40,14 @@ public class JobManagerOptions {
         */
        public static final ConfigOption<String> ADDRESS =
                key("jobmanager.rpc.address")
-               .noDefaultValue();
+               .noDefaultValue()
+               .withDescription("The config parameter defining the network 
address to connect to" +
+                       " for communication with the job manager." +
+                       " This value is only interpreted in setups where a 
single JobManager with static" +
+                       " name or address exists (simple standalone setups, or 
container setups with dynamic" +
+                       " service name resolution). It is not used in many 
high-availability setups, when a" +
+                       " leader-election service (like ZooKeeper) is used to 
elect and discover the JobManager" +
+                       " leader from potentially multiple standby 
JobManagers.");
 
        /**
         * The config parameter defining the network port to connect to
@@ -55,14 +62,23 @@ public class JobManagerOptions {
         */
        public static final ConfigOption<Integer> PORT =
                key("jobmanager.rpc.port")
-               .defaultValue(6123);
+               .defaultValue(6123)
+               .withDescription("The config parameter defining the network 
port to connect to" +
+                       " for communication with the job manager." +
+                       " Like " + ADDRESS.key() + ", this value is only 
interpreted in setups where" +
+                       " a single JobManager with static name/address and port 
exists (simple standalone setups," +
+                       " or container setups with dynamic service name 
resolution)." +
+                       " This config option is not used in many 
high-availability setups, when a" +
+                       " leader-election service (like ZooKeeper) is used to 
elect and discover the JobManager" +
+                       " leader from potentially multiple standby 
JobManagers.");
 
        /**
         * JVM heap size (in megabytes) for the JobManager.
         */
        public static final ConfigOption<Integer> JOB_MANAGER_HEAP_MEMORY =
                key("jobmanager.heap.mb")
-               .defaultValue(1024);
+               .defaultValue(1024)
+               .withDescription("JVM heap size (in megabytes) for the 
JobManager.");
 
        /**
         * The maximum number of prior execution attempts kept in history.
@@ -70,14 +86,16 @@ public class JobManagerOptions {
        public static final ConfigOption<Integer> MAX_ATTEMPTS_HISTORY_SIZE =
                key("jobmanager.execution.attempts-history-size")
                        .defaultValue(16)
-                       
.withDeprecatedKeys("job-manager.max-attempts-history-size");
+                       
.withDeprecatedKeys("job-manager.max-attempts-history-size")
+                       .withDescription("The maximum number of prior execution 
attempts kept in history.");
 
        /**
         * The maximum number of prior execution attempts kept in history.
         */
        public static final ConfigOption<String> EXECUTION_FAILOVER_STRATEGY =
                key("jobmanager.execution.failover-strategy")
-                       .defaultValue("full");
+                       .defaultValue("full")
+                       .withDescription("The maximum number of prior execution 
attempts kept in history.");
 
        /**
         * This option specifies the interval in order to trigger a resource 
manager reconnection if the connection
@@ -87,7 +105,9 @@ public class JobManagerOptions {
         */
        public static final ConfigOption<Long> 
RESOURCE_MANAGER_RECONNECT_INTERVAL =
                key("jobmanager.resourcemanager.reconnect-interval")
-               .defaultValue(2000L);
+               .defaultValue(2000L)
+               .withDescription("This option specifies the interval in order 
to trigger a resource manager reconnection if the connection" +
+                       " to the resource manager has been lost. This option is 
only intended for internal use.");
 
        /**
         * The location where the JobManager stores the archives of completed 
jobs.

Reply via email to