This is an automated email from the ASF dual-hosted git repository.

azagrebin pushed a commit to branch flip116
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 0c8911612f0acf57e0708d10a6daabaa1222911e
Author: Andrey Zagrebin <azagre...@apache.org>
AuthorDate: Wed Apr 15 17:03:05 2020 +0300

    [FLINK-16746][conf] Deprecate jobmanager.heap.size
---
 .../generated/all_jobmanager_section.html          |  6 ---
 .../_includes/generated/common_memory_section.html | 48 ++++++++++++++++++++++
 .../generated/job_manager_configuration.html       | 46 ++++++++++++++++++++-
 docs/ops/config.md                                 |  6 ++-
 docs/ops/config.zh.md                              |  6 ++-
 docs/ops/deployment/cluster_setup.md               |  4 +-
 docs/ops/deployment/cluster_setup.zh.md            |  4 +-
 docs/ops/deployment/kubernetes.md                  |  2 +-
 docs/ops/deployment/kubernetes.zh.md               |  2 +-
 docs/ops/deployment/mesos.md                       |  4 +-
 docs/ops/deployment/mesos.zh.md                    |  4 +-
 .../flink/configuration/JobManagerOptions.java     | 16 +++-----
 flink-end-to-end-tests/test-scripts/common.sh      |  2 +-
 flink-jepsen/src/jepsen/flink/db.clj               |  2 +-
 14 files changed, 117 insertions(+), 35 deletions(-)

diff --git a/docs/_includes/generated/all_jobmanager_section.html 
b/docs/_includes/generated/all_jobmanager_section.html
index 3369667..6ef552c 100644
--- a/docs/_includes/generated/all_jobmanager_section.html
+++ b/docs/_includes/generated/all_jobmanager_section.html
@@ -27,12 +27,6 @@
             <td>This option specifies how the job computation recovers from 
task failures. Accepted values are:<ul><li>'full': Restarts all tasks to 
recover the job.</li><li>'region': Restarts all tasks that could be affected by 
the task failure. More details can be found <a 
href="../dev/task_failure_recovery.html#restart-pipelined-region-failover-strategy">here</a>.</li></ul></td>
         </tr>
         <tr>
-            <td><h5>jobmanager.heap.size</h5></td>
-            <td style="word-wrap: break-word;">1 gb</td>
-            <td>MemorySize</td>
-            <td>JVM heap size for the JobManager.</td>
-        </tr>
-        <tr>
             <td><h5>jobmanager.rpc.address</h5></td>
             <td style="word-wrap: break-word;">(none)</td>
             <td>String</td>
diff --git a/docs/_includes/generated/common_memory_section.html 
b/docs/_includes/generated/common_memory_section.html
index 329916f..ce0878c 100644
--- a/docs/_includes/generated/common_memory_section.html
+++ b/docs/_includes/generated/common_memory_section.html
@@ -9,6 +9,54 @@
     </thead>
     <tbody>
         <tr>
+            <td><h5>jobmanager.memory.flink.size</h5></td>
+            <td style="word-wrap: break-word;">(none)</td>
+            <td>MemorySize</td>
+            <td>Total Flink Memory size for the JobManager. This includes all 
the memory that a JobManager consumes, except for JVM Metaspace and JVM 
Overhead. It consists of JVM Heap Memory and Off-heap Memory. See also 
'jobmanager.memory.process.size' for total process memory size 
configuration.</td>
+        </tr>
+        <tr>
+            <td><h5>jobmanager.memory.heap.size</h5></td>
+            <td style="word-wrap: break-word;">(none)</td>
+            <td>MemorySize</td>
+            <td>JVM Heap Memory size for JobManager. The minimum recommended 
JVM Heap size is 128.000mb (134217728 bytes).</td>
+        </tr>
+        <tr>
+            <td><h5>jobmanager.memory.jvm-metaspace.size</h5></td>
+            <td style="word-wrap: break-word;">256 mb</td>
+            <td>MemorySize</td>
+            <td>JVM Metaspace Size for the JobManager.</td>
+        </tr>
+        <tr>
+            <td><h5>jobmanager.memory.jvm-overhead.fraction</h5></td>
+            <td style="word-wrap: break-word;">0.1</td>
+            <td>Float</td>
+            <td>Fraction of Total Process Memory to be reserved for JVM 
Overhead. This is off-heap memory reserved for JVM overhead, such as thread 
stack space, compile cache, etc. This includes native memory but not direct 
memory, and will not be counted when Flink calculates JVM max direct memory 
size parameter. The size of JVM Overhead is derived to make up the configured 
fraction of the Total Process Memory. If the derived size is less or greater 
than the configured min or max size,  [...]
+        </tr>
+        <tr>
+            <td><h5>jobmanager.memory.jvm-overhead.max</h5></td>
+            <td style="word-wrap: break-word;">1 gb</td>
+            <td>MemorySize</td>
+            <td>Max JVM Overhead size for the JobManager. This is off-heap 
memory reserved for JVM overhead, such as thread stack space, compile cache, 
etc. This includes native memory but not direct memory, and will not be counted 
when Flink calculates JVM max direct memory size parameter. The size of JVM 
Overhead is derived to make up the configured fraction of the Total Process 
Memory. If the derived size is less or greater than the configured min or max 
size, the min or max size will [...]
+        </tr>
+        <tr>
+            <td><h5>jobmanager.memory.jvm-overhead.min</h5></td>
+            <td style="word-wrap: break-word;">192 mb</td>
+            <td>MemorySize</td>
+            <td>Min JVM Overhead size for the JobManager. This is off-heap 
memory reserved for JVM overhead, such as thread stack space, compile cache, 
etc. This includes native memory but not direct memory, and will not be counted 
when Flink calculates JVM max direct memory size parameter. The size of JVM 
Overhead is derived to make up the configured fraction of the Total Process 
Memory. If the derived size is less or greater than the configured min or max 
size, the min or max size will [...]
+        </tr>
+        <tr>
+            <td><h5>jobmanager.memory.off-heap.size</h5></td>
+            <td style="word-wrap: break-word;">128 mb</td>
+            <td>MemorySize</td>
+            <td>Off-heap Memory size for JobManager. The JVM direct memory 
limit of the Job Manager process (-XX:MaxDirectMemorySize) will be set to this 
value. This option covers all off-heap memory usage including direct and native 
memory allocation.</td>
+        </tr>
+        <tr>
+            <td><h5>jobmanager.memory.process.size</h5></td>
+            <td style="word-wrap: break-word;">(none)</td>
+            <td>MemorySize</td>
+            <td>Total Process Memory size for the JobManager. This includes 
all the memory that a JobManager JVM process consumes, consisting of Total 
Flink Memory, JVM Metaspace, and JVM Overhead. In containerized setups, this 
should be set to the container memory. See also 'jobmanager.memory.flink.size' 
for Total Flink Memory size configuration.</td>
+        </tr>
+        <tr>
             <td><h5>taskmanager.memory.flink.size</h5></td>
             <td style="word-wrap: break-word;">(none)</td>
             <td>MemorySize</td>
diff --git a/docs/_includes/generated/job_manager_configuration.html 
b/docs/_includes/generated/job_manager_configuration.html
index a5c6c83..52cfa13 100644
--- a/docs/_includes/generated/job_manager_configuration.html
+++ b/docs/_includes/generated/job_manager_configuration.html
@@ -33,10 +33,52 @@
             <td>This option specifies how the job computation recovers from 
task failures. Accepted values are:<ul><li>'full': Restarts all tasks to 
recover the job.</li><li>'region': Restarts all tasks that could be affected by 
the task failure. More details can be found <a 
href="../dev/task_failure_recovery.html#restart-pipelined-region-failover-strategy">here</a>.</li></ul></td>
         </tr>
         <tr>
-            <td><h5>jobmanager.heap.size</h5></td>
+            <td><h5>jobmanager.memory.flink.size</h5></td>
+            <td style="word-wrap: break-word;">(none)</td>
+            <td>MemorySize</td>
+            <td>Total Flink Memory size for the JobManager. This includes all 
the memory that a JobManager consumes, except for JVM Metaspace and JVM 
Overhead. It consists of JVM Heap Memory and Off-heap Memory. See also 
'jobmanager.memory.process.size' for total process memory size 
configuration.</td>
+        </tr>
+        <tr>
+            <td><h5>jobmanager.memory.heap.size</h5></td>
+            <td style="word-wrap: break-word;">(none)</td>
+            <td>MemorySize</td>
+            <td>JVM Heap Memory size for JobManager. The minimum recommended 
JVM Heap size is 128.000mb (134217728 bytes).</td>
+        </tr>
+        <tr>
+            <td><h5>jobmanager.memory.jvm-metaspace.size</h5></td>
+            <td style="word-wrap: break-word;">256 mb</td>
+            <td>MemorySize</td>
+            <td>JVM Metaspace Size for the JobManager.</td>
+        </tr>
+        <tr>
+            <td><h5>jobmanager.memory.jvm-overhead.fraction</h5></td>
+            <td style="word-wrap: break-word;">0.1</td>
+            <td>Float</td>
+            <td>Fraction of Total Process Memory to be reserved for JVM 
Overhead. This is off-heap memory reserved for JVM overhead, such as thread 
stack space, compile cache, etc. This includes native memory but not direct 
memory, and will not be counted when Flink calculates JVM max direct memory 
size parameter. The size of JVM Overhead is derived to make up the configured 
fraction of the Total Process Memory. If the derived size is less or greater 
than the configured min or max size,  [...]
+        </tr>
+        <tr>
+            <td><h5>jobmanager.memory.jvm-overhead.max</h5></td>
             <td style="word-wrap: break-word;">1 gb</td>
             <td>MemorySize</td>
-            <td>JVM heap size for the JobManager.</td>
+            <td>Max JVM Overhead size for the JobManager. This is off-heap 
memory reserved for JVM overhead, such as thread stack space, compile cache, 
etc. This includes native memory but not direct memory, and will not be counted 
when Flink calculates JVM max direct memory size parameter. The size of JVM 
Overhead is derived to make up the configured fraction of the Total Process 
Memory. If the derived size is less or greater than the configured min or max 
size, the min or max size will [...]
+        </tr>
+        <tr>
+            <td><h5>jobmanager.memory.jvm-overhead.min</h5></td>
+            <td style="word-wrap: break-word;">192 mb</td>
+            <td>MemorySize</td>
+            <td>Min JVM Overhead size for the JobManager. This is off-heap 
memory reserved for JVM overhead, such as thread stack space, compile cache, 
etc. This includes native memory but not direct memory, and will not be counted 
when Flink calculates JVM max direct memory size parameter. The size of JVM 
Overhead is derived to make up the configured fraction of the Total Process 
Memory. If the derived size is less or greater than the configured min or max 
size, the min or max size will [...]
+        </tr>
+        <tr>
+            <td><h5>jobmanager.memory.off-heap.size</h5></td>
+            <td style="word-wrap: break-word;">128 mb</td>
+            <td>MemorySize</td>
+            <td>Off-heap Memory size for JobManager. The JVM direct memory 
limit of the Job Manager process (-XX:MaxDirectMemorySize) will be set to this 
value. This option covers all off-heap memory usage including direct and native 
memory allocation.</td>
+        </tr>
+        <tr>
+            <td><h5>jobmanager.memory.process.size</h5></td>
+            <td style="word-wrap: break-word;">(none)</td>
+            <td>MemorySize</td>
+            <td>Total Process Memory size for the JobManager. This includes 
all the memory that a JobManager JVM process consumes, consisting of Total 
Flink Memory, JVM Metaspace, and JVM Overhead. In containerized setups, this 
should be set to the container memory. See also 'jobmanager.memory.flink.size' 
for Total Flink Memory size configuration.</td>
         </tr>
         <tr>
             <td><h5>jobmanager.rpc.address</h5></td>
diff --git a/docs/ops/config.md b/docs/ops/config.md
index 992a838..98fe2f0 100644
--- a/docs/ops/config.md
+++ b/docs/ops/config.md
@@ -51,8 +51,10 @@ If you use Flink with 
[Yarn]({{site.baseurl}}/ops/deployment/yarn_setup.html), [
 
 The default memory sizes support simple streaming/batch applications, but are 
too low to yield good performance for more complex applications.
 
-  - `jobmanager.heap.size`: Sets the size of the *Flink Master* (JobManager / 
ResourceManager / Dispatcher) JVM heap.
-  - `taskmanager.memory.process.size`: Total size of the TaskManager process, 
including everything. Flink will subtract some memory for the JVM's own memory 
requirements (metaspace and others), and divide and configure the rest 
automatically between its components (network, managed memory, JVM Heap, etc.).
+  - `jobmanager.memory.process.size`: Total size of the *Flink Master* 
(JobManager / ResourceManager / Dispatcher) process.
+  - `taskmanager.memory.process.size`: Total size of the TaskManager process.
+
+The total sizes include everything. Flink will subtract some memory for the 
JVM's own memory requirements (metaspace and others), and divide and configure 
the rest automatically between its components (JVM Heap, Off-Heap, for Task 
Managers also network, managed memory etc.).
 
 These value are configured as memory sizes, for example *1536m* or *2g*.
 
diff --git a/docs/ops/config.zh.md b/docs/ops/config.zh.md
index 23b7ce1..b2c71da 100644
--- a/docs/ops/config.zh.md
+++ b/docs/ops/config.zh.md
@@ -51,8 +51,10 @@ If you use Flink with 
[Yarn]({{site.baseurl}}/ops/deployment/yarn_setup.html), [
 
 The default memory sizes support simple streaming/batch applications, but are 
too low to yield good performance for more complex applications.
 
-  - `jobmanager.heap.size`: Sets the size of the *Flink Master* (JobManager / 
ResourceManager / Dispatcher) JVM heap.
-  - `taskmanager.memory.process.size`: Total size of the TaskManager process, 
including everything. Flink will subtract some memory for the JVM's own memory 
requirements (metaspace and others), and divide and configure the rest 
automatically between its components (network, managed memory, JVM Heap, etc.).
+  - `jobmanager.memory.process.size`: Total size of the *Flink Master* 
(JobManager / ResourceManager / Dispatcher) process.
+  - `taskmanager.memory.process.size`: Total size of the TaskManager process.
+
+The total sizes include everything. Flink will subtract some memory for the 
JVM's own memory requirements (metaspace and others), and divide and configure 
the rest automatically between its components (JVM Heap, Off-Heap, for Task 
Managers also network, managed memory etc.).
 
 These value are configured as memory sizes, for example *1536m* or *2g*.
 
diff --git a/docs/ops/deployment/cluster_setup.md 
b/docs/ops/deployment/cluster_setup.md
index 5efd38d..75455d2 100644
--- a/docs/ops/deployment/cluster_setup.md
+++ b/docs/ops/deployment/cluster_setup.md
@@ -68,7 +68,7 @@ cd flink-*
 
 After having extracted the system files, you need to configure Flink for the 
cluster by editing *conf/flink-conf.yaml*.
 
-Set the `jobmanager.rpc.address` key to point to your master node. You should 
also define the maximum amount of main memory Flink is allowed to allocate on 
each node by setting the `jobmanager.heap.size` and 
`taskmanager.memory.process.size` keys.
+Set the `jobmanager.rpc.address` key to point to your master node. You should 
also define the maximum amount of main memory Flink is allowed to allocate on 
each node by setting the `jobmanager.memory.process.size` and 
`taskmanager.memory.process.size` keys.
 
 These values are given in MB. If some worker nodes have more main memory which 
you want to allocate to the Flink system you can overwrite the default value by 
setting `taskmanager.memory.process.size` or `taskmanager.memory.flink.size` in 
*conf/flink-conf.yaml* on those specific nodes.
 
@@ -106,7 +106,7 @@ Please see the [configuration page](../config.html) for 
details and additional c
 
 In particular,
 
- * the amount of available memory per JobManager (`jobmanager.heap.size`),
+ * the amount of available memory per JobManager 
(`jobmanager.memory.process.size`),
  * the amount of available memory per TaskManager 
(`taskmanager.memory.process.size` and check [memory setup 
guide](../memory/mem_tuning.html#configure-memory-for-standalone-deployment)),
  * the number of available CPUs per machine (`taskmanager.numberOfTaskSlots`),
  * the total number of CPUs in the cluster (`parallelism.default`) and
diff --git a/docs/ops/deployment/cluster_setup.zh.md 
b/docs/ops/deployment/cluster_setup.zh.md
index 55696f8..f9d6356 100644
--- a/docs/ops/deployment/cluster_setup.zh.md
+++ b/docs/ops/deployment/cluster_setup.zh.md
@@ -68,7 +68,7 @@ cd flink-*
 
 After having extracted the system files, you need to configure Flink for the 
cluster by editing *conf/flink-conf.yaml*.
 
-Set the `jobmanager.rpc.address` key to point to your master node. You should 
also define the maximum amount of main memory Flink is allowed to allocate on 
each node by setting the `jobmanager.heap.size` and 
`taskmanager.memory.process.size` keys.
+Set the `jobmanager.rpc.address` key to point to your master node. You should 
also define the maximum amount of main memory Flink is allowed to allocate on 
each node by setting the `jobmanager.memory.process.size` and 
`taskmanager.memory.process.size` keys.
 
 These values are given in MB. If some worker nodes have more main memory which 
you want to allocate to the Flink system you can overwrite the default value by 
setting setting `taskmanager.memory.process.size` or 
`taskmanager.memory.flink.size` in *conf/flink-conf.yaml* on those specific 
nodes.
 
@@ -106,7 +106,7 @@ Please see the [configuration page](../config.html) for 
details and additional c
 
 In particular,
 
- * the amount of available memory per JobManager (`jobmanager.heap.size`),
+ * the amount of available memory per JobManager 
(`jobmanager.memory.process.size`),
  * the amount of available memory per TaskManager 
(`taskmanager.memory.process.size` and check [memory setup 
guide](../memory/mem_tuning.html#configure-memory-for-standalone-deployment)),
  * the number of available CPUs per machine (`taskmanager.numberOfTaskSlots`),
  * the total number of CPUs in the cluster (`parallelism.default`) and
diff --git a/docs/ops/deployment/kubernetes.md 
b/docs/ops/deployment/kubernetes.md
index d798bff..adce518 100644
--- a/docs/ops/deployment/kubernetes.md
+++ b/docs/ops/deployment/kubernetes.md
@@ -143,7 +143,7 @@ data:
     blob.server.port: 6124
     jobmanager.rpc.port: 6123
     taskmanager.rpc.port: 6122
-    jobmanager.heap.size: 1024m
+    jobmanager.memory.process.size: 1472m
     taskmanager.memory.process.size: 1024m
   log4j.properties: |+
     rootLogger.level = INFO
diff --git a/docs/ops/deployment/kubernetes.zh.md 
b/docs/ops/deployment/kubernetes.zh.md
index f5dd31a..d724075 100644
--- a/docs/ops/deployment/kubernetes.zh.md
+++ b/docs/ops/deployment/kubernetes.zh.md
@@ -143,7 +143,7 @@ data:
     blob.server.port: 6124
     jobmanager.rpc.port: 6123
     taskmanager.rpc.port: 6122
-    jobmanager.heap.size: 1024m
+    jobmanager.memory.process.size: 1472m
     taskmanager.memory.process.size: 1024m
   log4j.properties: |+
     rootLogger.level = INFO
diff --git a/docs/ops/deployment/mesos.md b/docs/ops/deployment/mesos.md
index 1df1838..a273171 100644
--- a/docs/ops/deployment/mesos.md
+++ b/docs/ops/deployment/mesos.md
@@ -215,7 +215,7 @@ For example:
 
     bin/mesos-appmaster.sh \
         -Dmesos.master=master.foobar.org:5050 \
-        -Djobmanager.heap.size=1024m \
+        -Djobmanager.memory.process.size=1472m \
         -Djobmanager.rpc.port=6123 \
         -Drest.port=8081 \
         -Dtaskmanager.memory.process.size=3500m \
@@ -236,7 +236,7 @@ Here is an example configuration for Marathon:
 
     {
         "id": "flink",
-        "cmd": "$FLINK_HOME/bin/mesos-appmaster.sh 
-Djobmanager.heap.size=1024m -Djobmanager.rpc.port=6123 -Drest.port=8081 
-Dtaskmanager.memory.process.size=1024m -Dtaskmanager.numberOfTaskSlots=2 
-Dparallelism.default=2 -Dmesos.resourcemanager.tasks.cpus=1",
+        "cmd": "$FLINK_HOME/bin/mesos-appmaster.sh 
-Djobmanager.memory.process.size=1472m -Djobmanager.rpc.port=6123 
-Drest.port=8081 -Dtaskmanager.memory.process.size=1024m 
-Dtaskmanager.numberOfTaskSlots=2 -Dparallelism.default=2 
-Dmesos.resourcemanager.tasks.cpus=1",
         "cpus": 1.0,
         "mem": 1024
     }
diff --git a/docs/ops/deployment/mesos.zh.md b/docs/ops/deployment/mesos.zh.md
index 83fcd9a..2d08235 100644
--- a/docs/ops/deployment/mesos.zh.md
+++ b/docs/ops/deployment/mesos.zh.md
@@ -215,7 +215,7 @@ For example:
 
     bin/mesos-appmaster.sh \
         -Dmesos.master=master.foobar.org:5050 \
-        -Djobmanager.heap.size=1024m \
+        -Djobmanager.memory.process.size=1472m \
         -Djobmanager.rpc.port=6123 \
         -Drest.port=8081 \
         -Dtaskmanager.memory.process.size=3500m \
@@ -236,7 +236,7 @@ Here is an example configuration for Marathon:
 
     {
         "id": "flink",
-        "cmd": "$FLINK_HOME/bin/mesos-appmaster.sh 
-Djobmanager.heap.size=1024m -Djobmanager.rpc.port=6123 -Drest.port=8081 
-Dtaskmanager.memory.process.size=1024m -Dtaskmanager.numberOfTaskSlots=2 
-Dparallelism.default=2 -Dmesos.resourcemanager.tasks.cpus=1",
+        "cmd": "$FLINK_HOME/bin/mesos-appmaster.sh 
-Djobmanager.memory.process.size=1472m -Djobmanager.rpc.port=6123 
-Drest.port=8081 -Dtaskmanager.memory.process.size=1024m 
-Dtaskmanager.numberOfTaskSlots=2 -Dparallelism.default=2 
-Dmesos.resourcemanager.tasks.cpus=1",
         "cpus": 1.0,
         "mem": 1024
     }
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 8a54f1c..c0a5a7bb1 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
@@ -102,30 +102,31 @@ public class JobManagerOptions {
 
        /**
         * JVM heap size for the JobManager with memory size.
+        * @deprecated use {@link #TOTAL_FLINK_MEMORY} for standalone setups 
and {@link #TOTAL_PROCESS_MEMORY} for containerized setups.
         */
+       @Deprecated
        @Documentation.Section(Documentation.Sections.ALL_JOB_MANAGER)
        public static final ConfigOption<MemorySize> JOB_MANAGER_HEAP_MEMORY =
                key("jobmanager.heap.size")
                .memoryType()
-               .defaultValue(MemorySize.ofMebiBytes(1024))
+               .noDefaultValue()
                .withDescription("JVM heap size for the JobManager.");
 
        /**
         * JVM heap size (in megabytes) for the JobManager.
-        * @deprecated use {@link #JOB_MANAGER_HEAP_MEMORY}
+        * @deprecated use {@link #TOTAL_FLINK_MEMORY} for standalone setups 
and {@link #TOTAL_PROCESS_MEMORY} for containerized setups.
         */
        @Deprecated
        public static final ConfigOption<Integer> JOB_MANAGER_HEAP_MEMORY_MB =
                key("jobmanager.heap.mb")
                .intType()
-               .defaultValue(1024)
+               .noDefaultValue()
                .withDescription("JVM heap size (in megabytes) for the 
JobManager.");
 
        /**
         * Total Process Memory size for the JobManager.
         */
        @Documentation.Section(Documentation.Sections.COMMON_MEMORY)
-       @Documentation.ExcludeFromDocumentation("New JM memory model is still 
in development.")
        public static final ConfigOption<MemorySize> TOTAL_PROCESS_MEMORY =
                key("jobmanager.memory.process.size")
                        .memoryType()
@@ -139,7 +140,6 @@ public class JobManagerOptions {
         * Total Flink Memory size for the JobManager.
         */
        @Documentation.Section(Documentation.Sections.COMMON_MEMORY)
-       @Documentation.ExcludeFromDocumentation("New JM memory model is still 
in development.")
        public static final ConfigOption<MemorySize> TOTAL_FLINK_MEMORY =
                key("jobmanager.memory.flink.size")
                        .memoryType()
@@ -154,7 +154,6 @@ public class JobManagerOptions {
         * JVM Heap Memory size for the JobManager.
         */
        @Documentation.Section(Documentation.Sections.COMMON_MEMORY)
-       @Documentation.ExcludeFromDocumentation("New JM memory model is still 
in development.")
        public static final ConfigOption<MemorySize> JVM_HEAP_MEMORY =
                key("jobmanager.memory.heap.size")
                        .memoryType()
@@ -166,7 +165,6 @@ public class JobManagerOptions {
         * Off-heap Memory size for the JobManager.
         */
        @Documentation.Section(Documentation.Sections.COMMON_MEMORY)
-       @Documentation.ExcludeFromDocumentation("New JM memory model is still 
in development.")
        public static final ConfigOption<MemorySize> OFF_HEAP_MEMORY =
                key("jobmanager.memory.off-heap.size")
                        .memoryType()
@@ -179,7 +177,6 @@ public class JobManagerOptions {
         * JVM Metaspace Size for the JobManager.
         */
        @Documentation.Section(Documentation.Sections.COMMON_MEMORY)
-       @Documentation.ExcludeFromDocumentation("New JM memory model is still 
in development.")
        public static final ConfigOption<MemorySize> JVM_METASPACE =
                key("jobmanager.memory.jvm-metaspace.size")
                        .memoryType()
@@ -197,7 +194,6 @@ public class JobManagerOptions {
         * Min JVM Overhead size for the JobManager.
         */
        @Documentation.Section(Documentation.Sections.COMMON_MEMORY)
-       @Documentation.ExcludeFromDocumentation("New JM memory model is still 
in development.")
        public static final ConfigOption<MemorySize> JVM_OVERHEAD_MIN =
                key("jobmanager.memory.jvm-overhead.min")
                        .memoryType()
@@ -208,7 +204,6 @@ public class JobManagerOptions {
         * Max JVM Overhead size for the TaskExecutors.
         */
        @Documentation.Section(Documentation.Sections.COMMON_MEMORY)
-       @Documentation.ExcludeFromDocumentation("New JM memory model is still 
in development.")
        public static final ConfigOption<MemorySize> JVM_OVERHEAD_MAX =
                key("jobmanager.memory.jvm-overhead.max")
                        .memoryType()
@@ -219,7 +214,6 @@ public class JobManagerOptions {
         * Fraction of Total Process Memory to be reserved for JVM Overhead.
         */
        @Documentation.Section(Documentation.Sections.COMMON_MEMORY)
-       @Documentation.ExcludeFromDocumentation("New JM memory model is still 
in development.")
        public static final ConfigOption<Float> JVM_OVERHEAD_FRACTION =
                key("jobmanager.memory.jvm-overhead.fraction")
                        .floatType()
diff --git a/flink-end-to-end-tests/test-scripts/common.sh 
b/flink-end-to-end-tests/test-scripts/common.sh
index 3aabb3d..8329c12 100644
--- a/flink-end-to-end-tests/test-scripts/common.sh
+++ b/flink-end-to-end-tests/test-scripts/common.sh
@@ -167,7 +167,7 @@ function create_ha_config() {
 
     jobmanager.rpc.address: localhost
     jobmanager.rpc.port: 6123
-    jobmanager.heap.size: 1024m
+    jobmanager.memory.process.size: 1024m
     taskmanager.memory.process.size: 1024m
     taskmanager.numberOfTaskSlots: ${TASK_SLOTS_PER_TM_HA}
 
diff --git a/flink-jepsen/src/jepsen/flink/db.clj 
b/flink-jepsen/src/jepsen/flink/db.clj
index 24738c2..9d8a986 100644
--- a/flink-jepsen/src/jepsen/flink/db.clj
+++ b/flink-jepsen/src/jepsen/flink/db.clj
@@ -41,7 +41,7 @@
   {:high-availability                     "zookeeper"
    :high-availability.zookeeper.quorum    (zookeeper-quorum test)
    :high-availability.storageDir          "hdfs:///flink/ha"
-   :jobmanager.heap.size                  "2048m"
+   :jobmanager.memory.process.size        "2496m"
    :jobmanager.rpc.address                node
    :state.savepoints.dir                  "hdfs:///flink/savepoints"
    :rest.address                          node

Reply via email to