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

tison 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 30311ec  [FLINK-15639][k8s] Support to set tolerations for jobmanager 
and taskmanger pod
30311ec is described below

commit 30311ecdf470db4bb6d23c7883aed6aad901a898
Author: wangyang0918 <danrtsey...@alibaba-inc.com>
AuthorDate: Tue Apr 7 20:10:40 2020 +0800

    [FLINK-15639][k8s] Support to set tolerations for jobmanager and taskmanger 
pod
    
    Taints and tolerations work together to ensure that pods are not scheduled 
onto inappropriate nodes. One or more taints are applied to a node; this marks 
that the node should not accept any pods that do not tolerate the taints. 
Tolerations are applied to pods, and allow (but do not require) the pods to 
schedule onto nodes with matching taints.
    
    This closes #11606 .
---
 .../generated/kubernetes_config_configuration.html | 12 ++++
 .../configuration/KubernetesConfigOptions.java     | 18 ++++++
 .../decorators/InitJobManagerDecorator.java        |  4 ++
 .../decorators/InitTaskManagerDecorator.java       |  4 ++
 .../parameters/KubernetesJobManagerParameters.java |  6 ++
 .../parameters/KubernetesParameters.java           |  7 +++
 .../KubernetesTaskManagerParameters.java           |  6 ++
 .../kubeclient/resources/KubernetesToleration.java | 65 ++++++++++++++++++++++
 .../decorators/InitJobManagerDecoratorTest.java    | 13 +++++
 .../decorators/InitTaskManagerDecoratorTest.java   | 13 +++++
 10 files changed, 148 insertions(+)

diff --git a/docs/_includes/generated/kubernetes_config_configuration.html 
b/docs/_includes/generated/kubernetes_config_configuration.html
index 6ef559f..114a05b 100644
--- a/docs/_includes/generated/kubernetes_config_configuration.html
+++ b/docs/_includes/generated/kubernetes_config_configuration.html
@@ -105,6 +105,12 @@
             <td>Service account that is used by jobmanager within kubernetes 
cluster. The job manager uses this service account when requesting taskmanager 
pods from the API server.</td>
         </tr>
         <tr>
+            <td><h5>kubernetes.jobmanager.tolerations</h5></td>
+            <td style="word-wrap: break-word;">(none)</td>
+            <td>List&lt;Map&gt;</td>
+            <td>The user-specified tolerations to be set to the JobManager 
pod. The value should be in the form of 
key:key1,operator:Equal,value:value1,effect:NoSchedule;key:key2,operator:Exists,effect:NoExecute,tolerationSeconds:6000</td>
+        </tr>
+        <tr>
             <td><h5>kubernetes.namespace</h5></td>
             <td style="word-wrap: break-word;">"default"</td>
             <td>String</td>
@@ -140,5 +146,11 @@
             <td>Map</td>
             <td>The node selector to be set for TaskManager pods. Specified as 
key:value pairs separated by commas. For example, 
environment:production,disk:ssd.</td>
         </tr>
+        <tr>
+            <td><h5>kubernetes.taskmanager.tolerations</h5></td>
+            <td style="word-wrap: break-word;">(none)</td>
+            <td>List&lt;Map&gt;</td>
+            <td>The user-specified tolerations to be set to the TaskManager 
pod. The value should be in the form of 
key:key1,operator:Equal,value:value1,effect:NoSchedule;key:key2,operator:Exists,effect:NoExecute,tolerationSeconds:6000</td>
+        </tr>
     </tbody>
 </table>
diff --git 
a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java
 
b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java
index 39b2cad..70969b7 100644
--- 
a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java
+++ 
b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java
@@ -186,6 +186,24 @@ public class KubernetesConfigOptions {
                .withDescription("The user-specified annotations that are set 
to the TaskManager pod. The value could be " +
                        "in the form of a1:v1,a2:v2");
 
+       public static final ConfigOption<List<Map<String, String>>> 
JOB_MANAGER_TOLERATIONS =
+               key("kubernetes.jobmanager.tolerations")
+                       .mapType()
+                       .asList()
+                       .noDefaultValue()
+                       .withDescription("The user-specified tolerations to be 
set to the JobManager pod. The value should be " +
+                               "in the form of 
key:key1,operator:Equal,value:value1,effect:NoSchedule;" +
+                               
"key:key2,operator:Exists,effect:NoExecute,tolerationSeconds:6000");
+
+       public static final ConfigOption<List<Map<String, String>>> 
TASK_MANAGER_TOLERATIONS =
+               key("kubernetes.taskmanager.tolerations")
+                       .mapType()
+                       .asList()
+                       .noDefaultValue()
+                       .withDescription("The user-specified tolerations to be 
set to the TaskManager pod. The value should be " +
+                               "in the form of 
key:key1,operator:Equal,value:value1,effect:NoSchedule;" +
+                               
"key:key2,operator:Exists,effect:NoExecute,tolerationSeconds:6000");
+
        /**
         * The flink rest service exposed type.
         */
diff --git 
a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecorator.java
 
b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecorator.java
index 45fa10d..ad54d71 100644
--- 
a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecorator.java
+++ 
b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecorator.java
@@ -20,6 +20,7 @@ package org.apache.flink.kubernetes.kubeclient.decorators;
 
 import org.apache.flink.kubernetes.kubeclient.FlinkPod;
 import 
org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesToleration;
 import org.apache.flink.kubernetes.utils.Constants;
 import org.apache.flink.kubernetes.utils.KubernetesUtils;
 
@@ -66,6 +67,9 @@ public class InitJobManagerDecorator extends 
AbstractKubernetesStepDecorator {
                                
.withServiceAccountName(kubernetesJobManagerParameters.getServiceAccount())
                                
.withImagePullSecrets(kubernetesJobManagerParameters.getImagePullSecrets())
                                
.withNodeSelector(kubernetesJobManagerParameters.getNodeSelector())
+                               
.withTolerations(kubernetesJobManagerParameters.getTolerations().stream()
+                                       .map(e -> 
KubernetesToleration.fromMap(e).getInternalResource())
+                                       .collect(Collectors.toList()))
                                .endSpec()
                        .build();
 
diff --git 
a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecorator.java
 
b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecorator.java
index e789b60..f968b75 100644
--- 
a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecorator.java
+++ 
b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecorator.java
@@ -20,6 +20,7 @@ package org.apache.flink.kubernetes.kubeclient.decorators;
 
 import org.apache.flink.kubernetes.kubeclient.FlinkPod;
 import 
org.apache.flink.kubernetes.kubeclient.parameters.KubernetesTaskManagerParameters;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesToleration;
 import org.apache.flink.kubernetes.utils.Constants;
 import org.apache.flink.kubernetes.utils.KubernetesUtils;
 
@@ -60,6 +61,9 @@ public class InitTaskManagerDecorator extends 
AbstractKubernetesStepDecorator {
                        .editOrNewSpec()
                                
.withImagePullSecrets(kubernetesTaskManagerParameters.getImagePullSecrets())
                                
.withNodeSelector(kubernetesTaskManagerParameters.getNodeSelector())
+                               
.withTolerations(kubernetesTaskManagerParameters.getTolerations().stream()
+                                       .map(e -> 
KubernetesToleration.fromMap(e).getInternalResource())
+                                       .collect(Collectors.toList()))
                                .endSpec()
                        .build();
 
diff --git 
a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java
 
b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java
index 89f15f3..6540cdb 100644
--- 
a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java
+++ 
b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java
@@ -33,6 +33,7 @@ import 
org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
 
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 
 import static org.apache.flink.util.Preconditions.checkArgument;
@@ -79,6 +80,11 @@ public class KubernetesJobManagerParameters extends 
AbstractKubernetesParameters
                return 
flinkConfig.getOptional(KubernetesConfigOptions.JOB_MANAGER_ANNOTATIONS).orElse(Collections.emptyMap());
        }
 
+       @Override
+       public List<Map<String, String>> getTolerations() {
+               return 
flinkConfig.getOptional(KubernetesConfigOptions.JOB_MANAGER_TOLERATIONS).orElse(Collections.emptyList());
+       }
+
        public String getJobManagerMainContainerName() {
                return JOB_MANAGER_MAIN_CONTAINER_NAME;
        }
diff --git 
a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesParameters.java
 
b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesParameters.java
index 2f66101..44443c4 100644
--- 
a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesParameters.java
+++ 
b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesParameters.java
@@ -22,6 +22,7 @@ import 
org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
 
 import io.fabric8.kubernetes.api.model.LocalObjectReference;
 
+import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 
@@ -68,6 +69,12 @@ public interface KubernetesParameters {
        Map<String, String> getAnnotations();
 
        /**
+        * A collection of tolerations that are set to the JobManager and 
TaskManager Pod(s). Kubernetes taints and
+        * tolerations work together to ensure that pods are not scheduled onto 
inappropriate nodes.
+        */
+       List<Map<String, String>> getTolerations();
+
+       /**
         * Directory in Pod that stores the flink-conf.yaml, log4j.properties, 
and the logback.xml.
         */
        String getFlinkConfDirInPod();
diff --git 
a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParameters.java
 
b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParameters.java
index 6929d3a..663db57 100644
--- 
a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParameters.java
+++ 
b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParameters.java
@@ -26,6 +26,7 @@ import 
org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameter
 
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 
 import static org.apache.flink.util.Preconditions.checkArgument;
@@ -84,6 +85,11 @@ public class KubernetesTaskManagerParameters extends 
AbstractKubernetesParameter
                return 
flinkConfig.getOptional(KubernetesConfigOptions.TASK_MANAGER_ANNOTATIONS).orElse(Collections.emptyMap());
        }
 
+       @Override
+       public List<Map<String, String>> getTolerations() {
+               return 
flinkConfig.getOptional(KubernetesConfigOptions.TASK_MANAGER_TOLERATIONS).orElse(Collections.emptyList());
+       }
+
        public String getTaskManagerMainContainerName() {
                return TASK_MANAGER_MAIN_CONTAINER_NAME;
        }
diff --git 
a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesToleration.java
 
b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesToleration.java
new file mode 100644
index 0000000..2d43208
--- /dev/null
+++ 
b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesToleration.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes.kubeclient.resources;
+
+import io.fabric8.kubernetes.api.model.Toleration;
+import io.fabric8.kubernetes.api.model.TolerationBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+
+/**
+ * Represent Toleration resource in kubernetes.
+ */
+public class KubernetesToleration extends KubernetesResource<Toleration> {
+
+       private static final Logger LOG = 
LoggerFactory.getLogger(KubernetesToleration.class);
+
+       private KubernetesToleration(Toleration toleration) {
+               super(toleration);
+       }
+
+       public static KubernetesToleration fromMap(Map<String, String> 
stringMap) {
+               final TolerationBuilder tolerationBuilder = new 
TolerationBuilder();
+               stringMap.forEach((k, v) -> {
+                       switch (k.toLowerCase()) {
+                               case "effect":
+                                       tolerationBuilder.withEffect(v);
+                                       break;
+                               case "key":
+                                       tolerationBuilder.withKey(v);
+                                       break;
+                               case "operator":
+                                       tolerationBuilder.withOperator(v);
+                                       break;
+                               case "tolerationseconds":
+                                       
tolerationBuilder.withTolerationSeconds(Long.valueOf(v));
+                                       break;
+                               case "value":
+                                       tolerationBuilder.withValue(v);
+                                       break;
+                               default:
+                                       LOG.warn("Unrecognized key({}) of 
toleration, will ignore.", k);
+                                       break;
+                       }
+               });
+               return new KubernetesToleration(tolerationBuilder.build());
+       }
+}
diff --git 
a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecoratorTest.java
 
b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecoratorTest.java
index 78d1976..0298129 100644
--- 
a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecoratorTest.java
+++ 
b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecoratorTest.java
@@ -32,6 +32,8 @@ import io.fabric8.kubernetes.api.model.LocalObjectReference;
 import io.fabric8.kubernetes.api.model.Pod;
 import io.fabric8.kubernetes.api.model.Quantity;
 import io.fabric8.kubernetes.api.model.ResourceRequirements;
+import io.fabric8.kubernetes.api.model.Toleration;
+import org.hamcrest.Matchers;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -60,6 +62,11 @@ public class InitJobManagerDecoratorTest extends 
KubernetesJobManagerTestBase {
                        put("a2", "v2");
                }
        };
+       private static final String TOLERATION_STRING = 
"key:key1,operator:Equal,value:value1,effect:NoSchedule;" +
+               
"KEY:key2,operator:Exists,Effect:NoExecute,tolerationSeconds:6000";
+       private static final List<Toleration> TOLERATION = Arrays.asList(
+               new Toleration("NoSchedule", "key1", "Equal", null, "value1"),
+               new Toleration("NoExecute", "key2", "Exists", 6000L, null));
 
        private Pod resultPod;
        private Container resultMainContainer;
@@ -70,6 +77,7 @@ public class InitJobManagerDecoratorTest extends 
KubernetesJobManagerTestBase {
                
this.flinkConfig.set(KubernetesConfigOptions.JOB_MANAGER_SERVICE_ACCOUNT, 
SERVICE_ACCOUNT_NAME);
                
this.flinkConfig.set(KubernetesConfigOptions.CONTAINER_IMAGE_PULL_SECRETS, 
IMAGE_PULL_SECRETS);
                
this.flinkConfig.set(KubernetesConfigOptions.JOB_MANAGER_ANNOTATIONS, 
ANNOTATIONS);
+               
this.flinkConfig.setString(KubernetesConfigOptions.JOB_MANAGER_TOLERATIONS.key(),
 TOLERATION_STRING);
 
                final InitJobManagerDecorator initJobManagerDecorator =
                        new 
InitJobManagerDecorator(this.kubernetesJobManagerParameters);
@@ -174,4 +182,9 @@ public class InitJobManagerDecoratorTest extends 
KubernetesJobManagerTestBase {
        public void testNodeSelector() {
                assertThat(this.resultPod.getSpec().getNodeSelector(), 
is(equalTo(nodeSelector)));
        }
+
+       @Test
+       public void testPodTolerations() {
+               assertThat(this.resultPod.getSpec().getTolerations(), 
Matchers.containsInAnyOrder(TOLERATION.toArray()));
+       }
 }
diff --git 
a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecoratorTest.java
 
b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecoratorTest.java
index 287548c..1b91bc2 100644
--- 
a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecoratorTest.java
+++ 
b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecoratorTest.java
@@ -31,6 +31,8 @@ import io.fabric8.kubernetes.api.model.LocalObjectReference;
 import io.fabric8.kubernetes.api.model.Pod;
 import io.fabric8.kubernetes.api.model.Quantity;
 import io.fabric8.kubernetes.api.model.ResourceRequirements;
+import io.fabric8.kubernetes.api.model.Toleration;
+import org.hamcrest.Matchers;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -58,6 +60,11 @@ public class InitTaskManagerDecoratorTest extends 
KubernetesTaskManagerTestBase
                        put("a2", "v2");
                }
        };
+       private static final String TOLERATION_STRING = 
"key:key1,operator:Equal,value:value1,effect:NoSchedule;" +
+               
"KEY:key2,operator:Exists,Effect:NoExecute,tolerationSeconds:6000";
+       private static final List<Toleration> TOLERATION = Arrays.asList(
+               new Toleration("NoSchedule", "key1", "Equal", null, "value1"),
+               new Toleration("NoExecute", "key2", "Exists", 6000L, null));
 
        private Pod resultPod;
        private Container resultMainContainer;
@@ -67,6 +74,7 @@ public class InitTaskManagerDecoratorTest extends 
KubernetesTaskManagerTestBase
                super.setup();
                
this.flinkConfig.set(KubernetesConfigOptions.CONTAINER_IMAGE_PULL_SECRETS, 
IMAGE_PULL_SECRETS);
                
this.flinkConfig.set(KubernetesConfigOptions.TASK_MANAGER_ANNOTATIONS, 
ANNOTATIONS);
+               
this.flinkConfig.setString(KubernetesConfigOptions.TASK_MANAGER_TOLERATIONS.key(),
 TOLERATION_STRING);
 
                final InitTaskManagerDecorator initTaskManagerDecorator =
                        new 
InitTaskManagerDecorator(kubernetesTaskManagerParameters);
@@ -168,4 +176,9 @@ public class InitTaskManagerDecoratorTest extends 
KubernetesTaskManagerTestBase
        public void testNodeSelector() {
                assertThat(this.resultPod.getSpec().getNodeSelector(), 
is(equalTo(nodeSelector)));
        }
+
+       @Test
+       public void testPodTolerations() {
+               assertThat(this.resultPod.getSpec().getTolerations(), 
Matchers.containsInAnyOrder(TOLERATION.toArray()));
+       }
 }

Reply via email to