wangyang0918 commented on a change in pull request #13186:
URL: https://github.com/apache/flink/pull/13186#discussion_r476203060



##########
File path: 
flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManagerDriver.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
+import 
org.apache.flink.kubernetes.configuration.KubernetesResourceManagerDriverConfiguration;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import 
org.apache.flink.kubernetes.kubeclient.factory.KubernetesTaskManagerFactory;
+import 
org.apache.flink.kubernetes.kubeclient.parameters.KubernetesTaskManagerParameters;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesPod;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.runtime.clusterframework.ApplicationStatus;
+import org.apache.flink.runtime.clusterframework.BootstrapTools;
+import 
org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.externalresource.ExternalResourceUtils;
+import 
org.apache.flink.runtime.resourcemanager.active.AbstractResourceManagerDriver;
+import org.apache.flink.runtime.resourcemanager.active.ResourceManagerDriver;
+import 
org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Implementation of {@link ResourceManagerDriver} for Kubernetes deployment.
+ */
+public class KubernetesResourceManagerDriver extends 
AbstractResourceManagerDriver<KubernetesWorkerNode>
+       implements FlinkKubeClient.PodCallbackHandler {
+
+       /** The taskmanager pod name pattern is 
{clusterId}-{taskmanager}-{attemptId}-{podIndex}. */
+       private static final String TASK_MANAGER_POD_FORMAT = 
"%s-taskmanager-%d-%d";
+
+       private final String clusterId;
+
+       private final Time podCreationRetryInterval;
+
+       private final FlinkKubeClient kubeClient;
+
+       /** Request resource futures, keyed by pod names. */
+       private final Map<String, CompletableFuture<KubernetesWorkerNode>> 
requestResourceFutures;
+
+       /** When ResourceManager failover, the max attempt should recover. */
+       private long currentMaxAttemptId = 0;
+
+       /** Current max pod index. When creating a new pod, it should increase 
one. */
+       private long currentMaxPodId = 0;
+
+       private KubernetesWatch podsWatch;
+
+       /**
+        * Incompletion of this future indicates that there was a pod creation 
failure recently and the driver should not
+        * retry creating pods until the future become completed again. It's 
guaranteed to be modified in main thread.
+        */
+       private CompletableFuture<Void> podCreationCoolDown;
+
+       public KubernetesResourceManagerDriver(
+                       Configuration flinkConfig,
+                       FlinkKubeClient kubeClient,
+                       KubernetesResourceManagerDriverConfiguration 
configuration) {
+               super(flinkConfig, GlobalConfiguration.loadConfiguration());
+
+               this.clusterId = 
Preconditions.checkNotNull(configuration.getClusterId());
+               this.podCreationRetryInterval = 
Preconditions.checkNotNull(configuration.getPodCreationRetryInterval());
+               this.kubeClient = Preconditions.checkNotNull(kubeClient);
+               this.requestResourceFutures = new HashMap<>();
+               this.podCreationCoolDown = FutureUtils.completedVoidFuture();
+       }
+
+       // 
------------------------------------------------------------------------
+       //  ResourceManagerDriver
+       // 
------------------------------------------------------------------------
+
+       @Override
+       protected void initializeInternal() throws Exception {
+               recoverWorkerNodesFromPreviousAttempts();
+
+               podsWatch = kubeClient.watchPodsAndDoCallback(
+                               KubernetesUtils.getTaskManagerLabels(clusterId),
+                               this);
+       }
+
+       @Override
+       public CompletableFuture<Void> terminate() {
+               // shut down all components
+               Exception exception = null;
+
+               try {
+                       podsWatch.close();
+               } catch (Exception e) {
+                       exception = e;
+               }
+
+               try {
+                       kubeClient.close();
+               } catch (Exception e) {
+                       exception = ExceptionUtils.firstOrSuppressed(e, 
exception);
+               }
+
+               return exception == null ?
+                               FutureUtils.completedVoidFuture() :
+                               FutureUtils.completedExceptionally(exception);
+       }
+
+       @Override
+       public void deregisterApplication(ApplicationStatus finalStatus, 
@Nullable String optionalDiagnostics) {
+               log.info("Deregistering flink kubernetes cluster, clusterId: 
{}, diagnostics: {}",
+                               clusterId,
+                               optionalDiagnostics == null ? "" : 
optionalDiagnostics);
+               kubeClient.stopAndCleanupCluster(clusterId);
+       }
+
+       @Override
+       public CompletableFuture<KubernetesWorkerNode> 
requestResource(TaskExecutorProcessSpec taskExecutorProcessSpec) {
+               final KubernetesTaskManagerParameters parameters =
+                               
createKubernetesTaskManagerParameters(taskExecutorProcessSpec);
+               final KubernetesPod taskManagerPod =
+                               
KubernetesTaskManagerFactory.buildTaskManagerKubernetesPod(parameters);
+               final String podName = taskManagerPod.getName();
+               final CompletableFuture<KubernetesWorkerNode> 
requestResourceFuture = new CompletableFuture<>();
+
+               requestResourceFutures.put(podName, requestResourceFuture);
+
+               log.info("Creating new TaskManager pod with name {} and 
resource <{},{}>.",
+                               podName,
+                               parameters.getTaskManagerMemoryMB(),
+                               parameters.getTaskManagerCPU());
+
+               // When K8s API Server is temporary unavailable, 
`kubeClient.createTaskManagerPod` might fail immediately.
+               // In case of pod creation failures, we should wait for an 
interval before trying to create new pods.
+               // Otherwise, ActiveResourceManager will always re-requesting 
the worker, which keeps the main thread busy.
+               final CompletableFuture<Void> createPodFuture =
+                               podCreationCoolDown.thenCompose((ignore) -> 
kubeClient.createTaskManagerPod(taskManagerPod));
+
+               FutureUtils.assertNoException(
+                               createPodFuture.handleAsync((ignore, exception) 
-> {
+                                       if (exception != null) {
+                                               log.warn("Could not create pod 
{}, exception: {}", podName, exception);
+                                               tryResetPodCreationCoolDown();
+                                               
CompletableFuture<KubernetesWorkerNode> future =
+                                                               
requestResourceFutures.remove(taskManagerPod.getName());
+                                               if (future != null) {
+                                                       
future.completeExceptionally(exception);
+                                               }
+                                       } else {
+                                               log.info("Pod {} is created.", 
podName);
+                                       }
+                                       return null;
+                               }, getMainThreadExecutor()));
+
+               return requestResourceFuture;
+       }
+
+       @Override
+       public void releaseResource(KubernetesWorkerNode worker) {
+               final String podName = worker.getResourceID().toString();
+
+               log.info("Stopping TaskManager pod {}.", podName);
+
+               removePod(podName);
+       }
+
+       // 
------------------------------------------------------------------------
+       //  FlinkKubeClient.PodCallbackHandler
+       // 
------------------------------------------------------------------------
+
+       @Override
+       public void onAdded(List<KubernetesPod> pods) {
+               getMainThreadExecutor().execute(() -> {
+                       for (KubernetesPod pod : pods) {
+                               final String podName = pod.getName();
+                               final CompletableFuture<KubernetesWorkerNode> 
requestResourceFuture = requestResourceFutures.remove(podName);
+
+                               if (requestResourceFuture == null) {
+                                       log.debug("Ignore TaskManager pod that 
is already added: {}", podName);
+                                       continue;
+                               }
+
+                               log.info("Received new TaskManager pod: {}", 
podName);
+                               requestResourceFuture.complete(new 
KubernetesWorkerNode(new ResourceID(podName)));
+                       }
+               });
+       }
+
+       @Override
+       public void onModified(List<KubernetesPod> pods) {
+               terminatedPodsInMainThread(pods);
+       }
+
+       @Override
+       public void onDeleted(List<KubernetesPod> pods) {
+               terminatedPodsInMainThread(pods);
+       }
+
+       @Override
+       public void onError(List<KubernetesPod> pods) {
+               terminatedPodsInMainThread(pods);
+       }
+
+       @Override
+       public void handleFatalError(Throwable throwable) {
+               getMainThreadExecutor().execute(() -> 
getResourceEventHandler().onError(throwable));
+       }
+
+       // 
------------------------------------------------------------------------
+       //  Internal
+       // 
------------------------------------------------------------------------
+
+       private void recoverWorkerNodesFromPreviousAttempts() throws 
ResourceManagerException {
+               final List<KubernetesPod> podList = 
kubeClient.getPodsWithLabels(KubernetesUtils.getTaskManagerLabels(clusterId));
+               final List<KubernetesWorkerNode> recoveredWorkers = new 
ArrayList<>();
+
+               for (KubernetesPod pod : podList) {
+                       final KubernetesWorkerNode worker = new 
KubernetesWorkerNode(new ResourceID(pod.getName()));
+                       recoveredWorkers.add(worker);
+                       final long attempt = worker.getAttempt();
+                       if (attempt > currentMaxAttemptId) {
+                               currentMaxAttemptId = attempt;
+                       }
+               }
+
+               log.info("Recovered {} pods from previous attempts, current 
attempt id is {}.",
+                               recoveredWorkers.size(),
+                               ++currentMaxAttemptId);
+
+               // Should not invoke resource event handler on the main thread 
executor.
+               // We are in the initializing thread. The main thread executor 
is not yet ready.
+               
getResourceEventHandler().onPreviousAttemptWorkersRecovered(recoveredWorkers);
+       }
+
+       private KubernetesTaskManagerParameters 
createKubernetesTaskManagerParameters(TaskExecutorProcessSpec 
taskExecutorProcessSpec) {
+               final String podName = String.format(
+                               TASK_MANAGER_POD_FORMAT,
+                               clusterId,
+                               currentMaxAttemptId,
+                               ++currentMaxPodId);
+
+               final ContaineredTaskManagerParameters taskManagerParameters =
+                               
ContaineredTaskManagerParameters.create(flinkConfig, taskExecutorProcessSpec);
+
+               final Configuration taskManagerConfig = new 
Configuration(flinkConfig);
+               
taskManagerConfig.set(TaskManagerOptions.TASK_MANAGER_RESOURCE_ID, podName);
+
+               final String dynamicProperties =
+                               
BootstrapTools.getDynamicPropertiesAsString(flinkClientConfig, 
taskManagerConfig);
+
+               return new KubernetesTaskManagerParameters(
+                               flinkConfig,
+                               podName,
+                               dynamicProperties,
+                               taskManagerParameters,
+                               
ExternalResourceUtils.getExternalResources(flinkConfig, 
KubernetesConfigOptions.EXTERNAL_RESOURCE_KUBERNETES_CONFIG_KEY_SUFFIX));
+       }
+
+       private void tryResetPodCreationCoolDown() {
+               if (podCreationCoolDown.isDone()) {
+                       log.info("Pod creation failed. Will not retry creating 
pods in {}.", podCreationRetryInterval);
+                       podCreationCoolDown = new CompletableFuture<>();
+                       getMainThreadExecutor().schedule(
+                                       () -> 
podCreationCoolDown.complete(null),
+                                       podCreationRetryInterval.getSize(),
+                                       podCreationRetryInterval.getUnit());
+               }
+       }
+
+       private void terminatedPodsInMainThread(List<KubernetesPod> pods) {
+               getMainThreadExecutor().execute(() -> {
+                       for (KubernetesPod pod : pods) {
+                               if (pod.isTerminated()) {
+                                       final String podName = pod.getName();
+                                       log.info("TaskManager pod {} is 
terminated.", podName);
+
+                                       // this is a safe net, in case 
onModified/onDeleted/onError is received before onAdded
+                                       final 
CompletableFuture<KubernetesWorkerNode> requestResourceFuture = 
requestResourceFutures.remove(podName);
+                                       if (requestResourceFuture != null) {
+                                               log.warn("Pod {} is terminated 
before receiving the ADDED event.", podName);
+                                               
requestResourceFuture.completeExceptionally(new FlinkException("Pod is 
terminated."));
+                                       }
+
+                                       
getResourceEventHandler().onWorkerTerminated(new ResourceID(podName));

Review comment:
       Do we need to call `closeTaskManagerConnection` in 
`ActiveResourceManager#onWorkerTerminated`?

##########
File path: 
flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesResourceManagerDriverTest.java
##########
@@ -0,0 +1,295 @@
+/*
+ * 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;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
+import 
org.apache.flink.kubernetes.configuration.KubernetesResourceManagerDriverConfiguration;
+import 
org.apache.flink.kubernetes.kubeclient.FlinkKubeClient.PodCallbackHandler;
+import org.apache.flink.kubernetes.kubeclient.TestingFlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesPod;
+import org.apache.flink.kubernetes.kubeclient.resources.TestingKubernetesPod;
+import org.apache.flink.kubernetes.utils.Constants;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.resourcemanager.active.ResourceManagerDriver;
+import 
org.apache.flink.runtime.resourcemanager.active.ResourceManagerDriverTestBase;
+
+import io.fabric8.kubernetes.api.model.ResourceRequirements;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+
+import static org.hamcrest.Matchers.greaterThanOrEqualTo;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for {@link KubernetesResourceManagerDriver}.
+ */
+public class KubernetesResourceManagerDriverTest extends 
ResourceManagerDriverTestBase<KubernetesWorkerNode> {
+
+       private static final String CLUSTER_ID = "testing-flink-cluster";
+       private static final Time POD_CREATION_INTERVAL = 
Time.milliseconds(50L);
+       private static final KubernetesResourceManagerDriverConfiguration 
KUBERNETES_RESOURCE_MANAGER_CONFIGURATION =
+                       new 
KubernetesResourceManagerDriverConfiguration(CLUSTER_ID, POD_CREATION_INTERVAL);
+
+       @Test
+       public void testOnPodAdded() throws Exception {
+               new Context() {{
+                       final CompletableFuture<KubernetesPod> createPodFuture 
= new CompletableFuture<>();
+                       final CompletableFuture<KubernetesWorkerNode> 
requestResourceFuture = new CompletableFuture<>();
+
+                       
flinkKubeClientBuilder.setCreateTaskManagerPodFunction((pod) -> {
+                               createPodFuture.complete(pod);
+                               return FutureUtils.completedVoidFuture();
+                       });
+
+                       runTest(() -> {
+                               // request new pod
+                               runInMainThread(() -> 
getDriver().requestResource(TASK_EXECUTOR_PROCESS_SPEC).thenAccept(requestResourceFuture::complete));
+                               final KubernetesPod pod = 
createPodFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+                               // prepare validation:
+                               // - complete requestResourceFuture in main 
thread with correct KubernetesWorkerNode
+                               final CompletableFuture<Void> validationFuture 
= requestResourceFuture.thenAccept((workerNode) -> {
+                                       validateInMainThread();
+                                       
assertThat(workerNode.getResourceID().toString(), is(pod.getName()));
+                               });
+
+                               // send onAdded event
+                               
getPodCallbackHandler().onAdded(Collections.singletonList(pod));
+
+                               // make sure finishing validation
+                               validationFuture.get(TIMEOUT_SEC, 
TimeUnit.SECONDS);
+                       });
+               }};
+       }
+
+       @Test
+       public void testOnPodModified() throws Exception {
+               new Context() {{
+                       testOnPodTerminated((pod) -> 
getPodCallbackHandler().onModified(pod));
+               }};
+       }
+
+       @Test
+       public void testOnPodDeleted() throws Exception {
+               new Context() {{
+                       testOnPodTerminated((pod) -> 
getPodCallbackHandler().onDeleted(pod));
+               }};
+       }
+
+       @Test
+       public void testOnError() throws Exception {
+               new Context() {{
+                       testOnPodTerminated((pod) -> 
getPodCallbackHandler().onError(pod));
+               }};
+       }
+
+       @Test
+       public void testFatalHandleError() throws Exception {
+               new Context() {{
+                       final CompletableFuture<Throwable> onErrorFuture = new 
CompletableFuture<>();
+                       
resourceEventHandlerBuilder.setOnErrorConsumer(onErrorFuture::complete);
+
+                       runTest(() -> {
+                               final Throwable testingError = new 
Throwable("testing error");
+                               
getPodCallbackHandler().handleFatalError(testingError);
+                               assertThat(onErrorFuture.get(TIMEOUT_SEC, 
TimeUnit.SECONDS), is(testingError));
+                       });
+               }};
+       }
+
+       @Test
+       public void testPodCreationInterval() throws Exception {
+               new Context() {{
+                       final AtomicInteger createPodCount = new 
AtomicInteger(0);
+                       final List<CompletableFuture<Long>> 
createPodTimeFutures = new ArrayList<>();
+                       createPodTimeFutures.add(new CompletableFuture<>());
+                       createPodTimeFutures.add(new CompletableFuture<>());
+
+                       
flinkKubeClientBuilder.setCreateTaskManagerPodFunction((ignore) -> {
+                               int idx = createPodCount.getAndIncrement();
+                               if (idx < createPodTimeFutures.size()) {
+                                       
createPodTimeFutures.get(idx).complete(System.currentTimeMillis());
+                               }
+                               return FutureUtils.completedExceptionally(new 
Throwable("testing error"));
+                       });
+
+                       runTest(() -> {
+                               // re-request resource on pod creation failed
+                               runInMainThread(() -> 
getDriver().requestResource(TASK_EXECUTOR_PROCESS_SPEC)
+                                               .whenComplete((ignore1, 
ignore2) -> getDriver().requestResource(TASK_EXECUTOR_PROCESS_SPEC)));
+
+                               // validate trying creating pod twice, with 
proper interval
+                               long t1 = 
createPodTimeFutures.get(0).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+                               long t2 = 
createPodTimeFutures.get(1).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+                               assertThat((t2 - t1), 
greaterThanOrEqualTo(POD_CREATION_INTERVAL.toMilliseconds()));
+                       });
+               }};
+       }
+
+       @Override
+       protected ResourceManagerDriverTestBase<KubernetesWorkerNode>.Context 
createContext() {
+               return new Context();
+       }
+
+       private class Context extends 
ResourceManagerDriverTestBase<KubernetesWorkerNode>.Context {
+               private final KubernetesPod previousAttemptPod = new 
TestingKubernetesPod(CLUSTER_ID + "-taskmanager-1-1");
+
+               final CompletableFuture<PodCallbackHandler> 
setWatchPodsAndDoCallbackFuture = new CompletableFuture<>();

Review comment:
       These variables could be private to avoid unexpected use.

##########
File path: 
flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesResourceManagerDriverTest.java
##########
@@ -0,0 +1,295 @@
+/*
+ * 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;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
+import 
org.apache.flink.kubernetes.configuration.KubernetesResourceManagerDriverConfiguration;
+import 
org.apache.flink.kubernetes.kubeclient.FlinkKubeClient.PodCallbackHandler;
+import org.apache.flink.kubernetes.kubeclient.TestingFlinkKubeClient;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesPod;
+import org.apache.flink.kubernetes.kubeclient.resources.TestingKubernetesPod;
+import org.apache.flink.kubernetes.utils.Constants;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.resourcemanager.active.ResourceManagerDriver;
+import 
org.apache.flink.runtime.resourcemanager.active.ResourceManagerDriverTestBase;
+
+import io.fabric8.kubernetes.api.model.ResourceRequirements;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+
+import static org.hamcrest.Matchers.greaterThanOrEqualTo;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for {@link KubernetesResourceManagerDriver}.
+ */
+public class KubernetesResourceManagerDriverTest extends 
ResourceManagerDriverTestBase<KubernetesWorkerNode> {
+
+       private static final String CLUSTER_ID = "testing-flink-cluster";
+       private static final Time POD_CREATION_INTERVAL = 
Time.milliseconds(50L);
+       private static final KubernetesResourceManagerDriverConfiguration 
KUBERNETES_RESOURCE_MANAGER_CONFIGURATION =
+                       new 
KubernetesResourceManagerDriverConfiguration(CLUSTER_ID, POD_CREATION_INTERVAL);
+
+       @Test
+       public void testOnPodAdded() throws Exception {
+               new Context() {{
+                       final CompletableFuture<KubernetesPod> createPodFuture 
= new CompletableFuture<>();
+                       final CompletableFuture<KubernetesWorkerNode> 
requestResourceFuture = new CompletableFuture<>();
+
+                       
flinkKubeClientBuilder.setCreateTaskManagerPodFunction((pod) -> {
+                               createPodFuture.complete(pod);
+                               return FutureUtils.completedVoidFuture();
+                       });
+
+                       runTest(() -> {
+                               // request new pod
+                               runInMainThread(() -> 
getDriver().requestResource(TASK_EXECUTOR_PROCESS_SPEC).thenAccept(requestResourceFuture::complete));
+                               final KubernetesPod pod = 
createPodFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+                               // prepare validation:
+                               // - complete requestResourceFuture in main 
thread with correct KubernetesWorkerNode
+                               final CompletableFuture<Void> validationFuture 
= requestResourceFuture.thenAccept((workerNode) -> {
+                                       validateInMainThread();
+                                       
assertThat(workerNode.getResourceID().toString(), is(pod.getName()));
+                               });
+
+                               // send onAdded event
+                               
getPodCallbackHandler().onAdded(Collections.singletonList(pod));
+
+                               // make sure finishing validation
+                               validationFuture.get(TIMEOUT_SEC, 
TimeUnit.SECONDS);
+                       });
+               }};
+       }
+
+       @Test
+       public void testOnPodModified() throws Exception {
+               new Context() {{
+                       testOnPodTerminated((pod) -> 
getPodCallbackHandler().onModified(pod));
+               }};
+       }
+
+       @Test
+       public void testOnPodDeleted() throws Exception {
+               new Context() {{
+                       testOnPodTerminated((pod) -> 
getPodCallbackHandler().onDeleted(pod));
+               }};
+       }
+
+       @Test
+       public void testOnError() throws Exception {
+               new Context() {{
+                       testOnPodTerminated((pod) -> 
getPodCallbackHandler().onError(pod));
+               }};
+       }
+
+       @Test
+       public void testFatalHandleError() throws Exception {
+               new Context() {{
+                       final CompletableFuture<Throwable> onErrorFuture = new 
CompletableFuture<>();
+                       
resourceEventHandlerBuilder.setOnErrorConsumer(onErrorFuture::complete);
+
+                       runTest(() -> {
+                               final Throwable testingError = new 
Throwable("testing error");
+                               
getPodCallbackHandler().handleFatalError(testingError);
+                               assertThat(onErrorFuture.get(TIMEOUT_SEC, 
TimeUnit.SECONDS), is(testingError));
+                       });
+               }};
+       }
+
+       @Test
+       public void testPodCreationInterval() throws Exception {
+               new Context() {{
+                       final AtomicInteger createPodCount = new 
AtomicInteger(0);
+                       final List<CompletableFuture<Long>> 
createPodTimeFutures = new ArrayList<>();
+                       createPodTimeFutures.add(new CompletableFuture<>());
+                       createPodTimeFutures.add(new CompletableFuture<>());
+
+                       
flinkKubeClientBuilder.setCreateTaskManagerPodFunction((ignore) -> {
+                               int idx = createPodCount.getAndIncrement();
+                               if (idx < createPodTimeFutures.size()) {
+                                       
createPodTimeFutures.get(idx).complete(System.currentTimeMillis());
+                               }
+                               return FutureUtils.completedExceptionally(new 
Throwable("testing error"));
+                       });
+
+                       runTest(() -> {
+                               // re-request resource on pod creation failed
+                               runInMainThread(() -> 
getDriver().requestResource(TASK_EXECUTOR_PROCESS_SPEC)
+                                               .whenComplete((ignore1, 
ignore2) -> getDriver().requestResource(TASK_EXECUTOR_PROCESS_SPEC)));
+
+                               // validate trying creating pod twice, with 
proper interval
+                               long t1 = 
createPodTimeFutures.get(0).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+                               long t2 = 
createPodTimeFutures.get(1).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+                               assertThat((t2 - t1), 
greaterThanOrEqualTo(POD_CREATION_INTERVAL.toMilliseconds()));
+                       });
+               }};
+       }
+
+       @Override
+       protected ResourceManagerDriverTestBase<KubernetesWorkerNode>.Context 
createContext() {
+               return new Context();
+       }
+
+       private class Context extends 
ResourceManagerDriverTestBase<KubernetesWorkerNode>.Context {
+               private final KubernetesPod previousAttemptPod = new 
TestingKubernetesPod(CLUSTER_ID + "-taskmanager-1-1");
+
+               final CompletableFuture<PodCallbackHandler> 
setWatchPodsAndDoCallbackFuture = new CompletableFuture<>();
+               final CompletableFuture<Void> closeKubernetesWatchFuture = new 
CompletableFuture<>();
+               final CompletableFuture<String> stopAndCleanupClusterFuture =  
new CompletableFuture<>();
+               final CompletableFuture<KubernetesPod> 
createTaskManagerPodFuture = new CompletableFuture<>();
+               final CompletableFuture<String> stopPodFuture = new 
CompletableFuture<>();
+
+               final TestingFlinkKubeClient.Builder flinkKubeClientBuilder = 
TestingFlinkKubeClient.builder()
+                               .setWatchPodsAndDoCallbackFunction((ignore, 
handler) -> {
+                                       
setWatchPodsAndDoCallbackFuture.complete(handler);
+                                       return new 
TestingFlinkKubeClient.MockKubernetesWatch() {
+                                               @Override
+                                               public void close() {
+                                                       
closeKubernetesWatchFuture.complete(null);
+                                               }
+                                       };
+                               })
+                               
.setStopAndCleanupClusterConsumer(stopAndCleanupClusterFuture::complete)
+                               .setCreateTaskManagerPodFunction((pod) -> {
+                                       
createTaskManagerPodFuture.complete(pod);
+                                       
getPodCallbackHandler().onAdded(Collections.singletonList(pod));
+                                       return 
FutureUtils.completedVoidFuture();
+                               })
+                               .setStopPodFunction((podname) -> {

Review comment:
       nit: maybe `podName` is better.

##########
File path: 
flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManagerDriver.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
+import 
org.apache.flink.kubernetes.configuration.KubernetesResourceManagerDriverConfiguration;
+import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient;
+import 
org.apache.flink.kubernetes.kubeclient.factory.KubernetesTaskManagerFactory;
+import 
org.apache.flink.kubernetes.kubeclient.parameters.KubernetesTaskManagerParameters;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesPod;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.kubernetes.utils.KubernetesUtils;
+import org.apache.flink.runtime.clusterframework.ApplicationStatus;
+import org.apache.flink.runtime.clusterframework.BootstrapTools;
+import 
org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.externalresource.ExternalResourceUtils;
+import 
org.apache.flink.runtime.resourcemanager.active.AbstractResourceManagerDriver;
+import org.apache.flink.runtime.resourcemanager.active.ResourceManagerDriver;
+import 
org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Implementation of {@link ResourceManagerDriver} for Kubernetes deployment.
+ */
+public class KubernetesResourceManagerDriver extends 
AbstractResourceManagerDriver<KubernetesWorkerNode>
+       implements FlinkKubeClient.PodCallbackHandler {
+
+       /** The taskmanager pod name pattern is 
{clusterId}-{taskmanager}-{attemptId}-{podIndex}. */
+       private static final String TASK_MANAGER_POD_FORMAT = 
"%s-taskmanager-%d-%d";
+
+       private final String clusterId;
+
+       private final Time podCreationRetryInterval;
+
+       private final FlinkKubeClient kubeClient;
+
+       /** Request resource futures, keyed by pod names. */
+       private final Map<String, CompletableFuture<KubernetesWorkerNode>> 
requestResourceFutures;
+
+       /** When ResourceManager failover, the max attempt should recover. */
+       private long currentMaxAttemptId = 0;
+
+       /** Current max pod index. When creating a new pod, it should increase 
one. */
+       private long currentMaxPodId = 0;
+
+       private KubernetesWatch podsWatch;
+
+       /**
+        * Incompletion of this future indicates that there was a pod creation 
failure recently and the driver should not
+        * retry creating pods until the future become completed again. It's 
guaranteed to be modified in main thread.
+        */
+       private CompletableFuture<Void> podCreationCoolDown;
+
+       public KubernetesResourceManagerDriver(
+                       Configuration flinkConfig,
+                       FlinkKubeClient kubeClient,
+                       KubernetesResourceManagerDriverConfiguration 
configuration) {
+               super(flinkConfig, GlobalConfiguration.loadConfiguration());
+
+               this.clusterId = 
Preconditions.checkNotNull(configuration.getClusterId());
+               this.podCreationRetryInterval = 
Preconditions.checkNotNull(configuration.getPodCreationRetryInterval());
+               this.kubeClient = Preconditions.checkNotNull(kubeClient);
+               this.requestResourceFutures = new HashMap<>();
+               this.podCreationCoolDown = FutureUtils.completedVoidFuture();
+       }
+
+       // 
------------------------------------------------------------------------
+       //  ResourceManagerDriver
+       // 
------------------------------------------------------------------------
+
+       @Override
+       protected void initializeInternal() throws Exception {
+               recoverWorkerNodesFromPreviousAttempts();
+
+               podsWatch = kubeClient.watchPodsAndDoCallback(
+                               KubernetesUtils.getTaskManagerLabels(clusterId),
+                               this);
+       }
+
+       @Override
+       public CompletableFuture<Void> terminate() {
+               // shut down all components
+               Exception exception = null;
+
+               try {
+                       podsWatch.close();
+               } catch (Exception e) {
+                       exception = e;
+               }
+
+               try {
+                       kubeClient.close();
+               } catch (Exception e) {
+                       exception = ExceptionUtils.firstOrSuppressed(e, 
exception);
+               }
+
+               return exception == null ?
+                               FutureUtils.completedVoidFuture() :
+                               FutureUtils.completedExceptionally(exception);
+       }
+
+       @Override
+       public void deregisterApplication(ApplicationStatus finalStatus, 
@Nullable String optionalDiagnostics) {
+               log.info("Deregistering flink kubernetes cluster, clusterId: 
{}, diagnostics: {}",
+                               clusterId,
+                               optionalDiagnostics == null ? "" : 
optionalDiagnostics);
+               kubeClient.stopAndCleanupCluster(clusterId);
+       }
+
+       @Override
+       public CompletableFuture<KubernetesWorkerNode> 
requestResource(TaskExecutorProcessSpec taskExecutorProcessSpec) {
+               final KubernetesTaskManagerParameters parameters =
+                               
createKubernetesTaskManagerParameters(taskExecutorProcessSpec);
+               final KubernetesPod taskManagerPod =
+                               
KubernetesTaskManagerFactory.buildTaskManagerKubernetesPod(parameters);
+               final String podName = taskManagerPod.getName();
+               final CompletableFuture<KubernetesWorkerNode> 
requestResourceFuture = new CompletableFuture<>();
+
+               requestResourceFutures.put(podName, requestResourceFuture);
+
+               log.info("Creating new TaskManager pod with name {} and 
resource <{},{}>.",
+                               podName,
+                               parameters.getTaskManagerMemoryMB(),
+                               parameters.getTaskManagerCPU());
+
+               // When K8s API Server is temporary unavailable, 
`kubeClient.createTaskManagerPod` might fail immediately.
+               // In case of pod creation failures, we should wait for an 
interval before trying to create new pods.
+               // Otherwise, ActiveResourceManager will always re-requesting 
the worker, which keeps the main thread busy.
+               final CompletableFuture<Void> createPodFuture =

Review comment:
       We have behavior change about retrying interval here. But i think is 
reasonable and do not increase the pressure of K8s apiserver.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to