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

zhijiang pushed a commit to branch release-1.10
in repository https://gitbox.apache.org/repos/asf/flink.git

commit a4a9f83427416fdbe5989ca8050fd3eca9146da7
Author: wangyang0918 <danrtsey...@alibaba-inc.com>
AuthorDate: Tue Dec 17 14:56:12 2019 +0800

    [FLINK-15288][kubernetes] Starting jobmanager pod should respect 
containerized heap-cutoff
    
    The cutoff is to leave some memory for jvm non-heap, for example meta 
space, thread native memory and etc.
---
 docs/_includes/generated/resource_manager_configuration.html          | 2 +-
 .../src/main/java/org/apache/flink/configuration/ConfigConstants.java | 2 +-
 .../java/org/apache/flink/configuration/ResourceManagerOptions.java   | 4 ++--
 .../main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java  | 3 ++-
 .../org/apache/flink/kubernetes/KubernetesClusterDescriptorTest.java  | 4 ----
 .../test/java/org/apache/flink/kubernetes/KubernetesUtilsTest.java    | 2 +-
 6 files changed, 7 insertions(+), 10 deletions(-)

diff --git a/docs/_includes/generated/resource_manager_configuration.html 
b/docs/_includes/generated/resource_manager_configuration.html
index d5c1131..b9fab66 100644
--- a/docs/_includes/generated/resource_manager_configuration.html
+++ b/docs/_includes/generated/resource_manager_configuration.html
@@ -18,7 +18,7 @@
             <td><h5>containerized.heap-cutoff-ratio</h5></td>
             <td style="word-wrap: break-word;">0.25</td>
             <td>Float</td>
-            <td>Percentage of heap space to remove from containers (YARN / 
Mesos), to compensate for other JVM memory usage.</td>
+            <td>Percentage of heap space to remove from containers (YARN / 
Mesos / Kubernetes), to compensate for other JVM memory usage.</td>
         </tr>
         <tr>
             <td><h5>local.number-resourcemanager</h5></td>
diff --git 
a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java 
b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
index 1a88eb3..f1bcd78 100644
--- 
a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
+++ 
b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
@@ -370,7 +370,7 @@ public final class ConfigConstants {
        // -------- Common Resource Framework Configuration (YARN & Mesos) 
--------
 
        /**
-        * Percentage of heap space to remove from containers (YARN / Mesos), 
to compensate
+        * Percentage of heap space to remove from containers (YARN / Mesos / 
Kubernetes), to compensate
         * for other JVM memory usage.
         * @deprecated Use {@link 
ResourceManagerOptions#CONTAINERIZED_HEAP_CUTOFF_RATIO} instead.
         */
diff --git 
a/flink-core/src/main/java/org/apache/flink/configuration/ResourceManagerOptions.java
 
b/flink-core/src/main/java/org/apache/flink/configuration/ResourceManagerOptions.java
index af8e43d..4bf354e 100644
--- 
a/flink-core/src/main/java/org/apache/flink/configuration/ResourceManagerOptions.java
+++ 
b/flink-core/src/main/java/org/apache/flink/configuration/ResourceManagerOptions.java
@@ -56,14 +56,14 @@ public class ResourceManagerOptions {
                        " Its not possible to use this configuration key to 
define port ranges.");
 
        /**
-        * Percentage of heap space to remove from containers (YARN / Mesos), 
to compensate
+        * Percentage of heap space to remove from containers (YARN / Mesos/ 
Kubernetes), to compensate
         * for other JVM memory usage.
         */
        public static final ConfigOption<Float> CONTAINERIZED_HEAP_CUTOFF_RATIO 
= ConfigOptions
                .key("containerized.heap-cutoff-ratio")
                .defaultValue(0.25f)
                .withDeprecatedKeys("yarn.heap-cutoff-ratio")
-               .withDescription("Percentage of heap space to remove from 
containers (YARN / Mesos), to compensate" +
+               .withDescription("Percentage of heap space to remove from 
containers (YARN / Mesos / Kubernetes), to compensate" +
                        " for other JVM memory usage.");
 
        /**
diff --git 
a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java
 
b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java
index f1eac1b..0a8d8da 100644
--- 
a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java
+++ 
b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java
@@ -126,7 +126,8 @@ public class KubernetesUtils {
                        boolean hasLog4j,
                        String mainClass,
                        @Nullable String mainArgs) {
-               final String jvmMemOpts = String.format("-Xms%sm -Xmx%sm", 
jobManagerMemoryMb, jobManagerMemoryMb);
+               final int heapSize = 
BootstrapTools.calculateHeapSize(jobManagerMemoryMb, flinkConfig);
+               final String jvmMemOpts = String.format("-Xms%sm -Xmx%sm", 
heapSize, heapSize);
                return getCommonStartCommand(
                        flinkConfig,
                        ClusterComponent.JOB_MANAGER,
diff --git 
a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesClusterDescriptorTest.java
 
b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesClusterDescriptorTest.java
index c1bdc6d..ae77505 100644
--- 
a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesClusterDescriptorTest.java
+++ 
b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesClusterDescriptorTest.java
@@ -102,10 +102,6 @@ public class KubernetesClusterDescriptorTest extends 
KubernetesTestBase {
                final KubernetesClusterDescriptor descriptor = new 
KubernetesClusterDescriptor(FLINK_CONFIG, flinkKubeClient);
 
                final ClusterSpecification clusterSpecification = new 
ClusterSpecification.ClusterSpecificationBuilder()
-                       .setMasterMemoryMB(1)
-                       .setTaskManagerMemoryMB(1)
-                       .setNumberTaskManagers(1)
-                       .setSlotsPerTaskManager(1)
                        .createClusterSpecification();
 
                descriptor.deploySessionCluster(clusterSpecification);
diff --git 
a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesUtilsTest.java
 
b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesUtilsTest.java
index d375ece..c9d7d6a 100644
--- 
a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesUtilsTest.java
+++ 
b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesUtilsTest.java
@@ -65,7 +65,7 @@ public class KubernetesUtilsTest extends TestLogger {
 
        // Memory variables
        private static final int jobManagerMem = 768;
-       private static final String jmJvmMem = String.format("-Xms%dm -Xmx%dm", 
jobManagerMem, jobManagerMem);
+       private static final String jmJvmMem = "-Xms168m -Xmx168m";
 
        private static final TaskExecutorResourceSpec taskExecutorResourceSpec 
= new TaskExecutorResourceSpec(
                new CPUResource(1.0),

Reply via email to