You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/08/18 06:51:34 UTC

[GitHub] [flink] xintongsong opened a new pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

xintongsong opened a new pull request #13186:
URL: https://github.com/apache/flink/pull/13186


   ## What is the purpose of the change
   
   This PR migrate Kubernetes deployment from the legacy KubernetesResourceManager to the new ActiveResourceManager + KubernetesResourceManagerDriver approach.
   
   ## Brief change log
   
   - e47c0e1be9a776d08b8e80c84c37280fff0b5b49: Initialize ResourceManagerDriver with ScheduledExecutor.
   - ca9c57db0b1eb5d0f36e119ac6c9c38d109a5449: Introduce AbstractResourceManagerDriver.
   - 0dbbc255908166cda090e830acccf4a40bc0de7f: Introduce KubernetesResourceManagerDriver.
   - 5e1135f030a936638448a1ccf986f88be9d39797: Enable pod creation interval.
   - 3a8705a4c617a0f76847c5c3f262043fe75b8942: Switch to the new KubernetesResourceManagerDriver.
   
   ## Verifying this change
   
   - Added KuberetesResourceManagerDriverTest.
   - Some of the original test cases in KubernetesResourceManagerTest is now covered by ActiveResourceManagerTest. 
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (no)
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (no)
     - The serializers: (no)
     - The runtime per-record code paths (performance sensitive): (no)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn/Mesos, ZooKeeper: (yes)
     - The S3 file system connector: (no)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (no)
     - If yes, how is the feature documented? (not applicable)
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13186:
URL: https://github.com/apache/flink/pull/13186#issuecomment-675299227


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5652",
       "triggerID" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5668",
       "triggerID" : "675358004",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f2e824b2daf427f49409e1ea5c7e9704a84c76d9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5676",
       "triggerID" : "f2e824b2daf427f49409e1ea5c7e9704a84c76d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd4e2812d845277e7f4bc68e97f4759e11334ca3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5680",
       "triggerID" : "bd4e2812d845277e7f4bc68e97f4759e11334ca3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * bd4e2812d845277e7f4bc68e97f4759e11334ca3 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5680) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13186:
URL: https://github.com/apache/flink/pull/13186#discussion_r477080583



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManagerDriver.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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.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.KubernetesResourceManagerConfiguration;
+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 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;
+
+	public KubernetesResourceManagerDriver(
+			Configuration flinkConfig,
+			FlinkKubeClient kubeClient,
+			KubernetesResourceManagerConfiguration configuration) {
+		super(flinkConfig, GlobalConfiguration.loadConfiguration());

Review comment:
       Moreover, relying on env variables to read the right things will make the testing of this component much harder if we want to control the "base" configuration.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13186:
URL: https://github.com/apache/flink/pull/13186#discussion_r477104459



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/AbstractResourceManagerDriver.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.runtime.resourcemanager.active;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Abstract common base class for implementations of {@link ResourceManagerDriver}.
+ */
+public abstract class AbstractResourceManagerDriver<WorkerType extends ResourceIDRetrievable>
+	implements ResourceManagerDriver<WorkerType> {
+
+	protected final Logger log = LoggerFactory.getLogger(getClass());
+
+	protected final Configuration flinkConfig;
+	protected final Configuration flinkClientConfig;

Review comment:
       Hi @tillrohrmann thanks for pointing out this concern. 
   
   > Why we are trying to only pass the diff config options, rather that the full configuration?
   
   That will make the start commands very long. Also it is hard to distinguish the config options only for TaskManager. So currently, we compare the effective configuration in ResourceManager and client shipped configuration. And then pass the differences.
   
   > Is `flinkClientConfig` the original configuration?
   
   I think yes. For Yarn and K8s entrypoints(e.g. `YarnJobClusterEntrypoint`, `KubernetesApplicationClusterEntrypoint`, etc.), we do not accept the configured configuration directory(via `--configDir`). Instead we load them from a specific directory(`$PWD` for Yarn and `ENV_FLINK_CONF_DIR` for K8s), both of them are guaranteed correctly setting in the Flink client.
   
   However, if we could find out which config options should be passed the TaskManagers, i agree that `flinkClientConfig` could be removed.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on pull request #13186:
URL: https://github.com/apache/flink/pull/13186#issuecomment-680927647


   I'm fine with merging this PR and addressing the `flinkClientConfig` in a follow up issue.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] XComp commented on a change in pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #13186:
URL: https://github.com/apache/flink/pull/13186#discussion_r472002192



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/active/TestingResourceEventHandler.java
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.runtime.resourcemanager.active;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Collection;
+import java.util.function.Consumer;
+
+/**
+ * Testing implementation of {@link ResourceEventHandler}.
+ */
+public class TestingResourceEventHandler<WorkerType extends ResourceIDRetrievable> implements ResourceEventHandler<WorkerType> {
+
+	private final Consumer<Collection<WorkerType>> onPreviousAttemptWorkersRecoveredConsumer;
+	private final Consumer<ResourceID> onWorkerTerminatedConsumer;
+	private final Consumer<Throwable> onErrorConsumer;
+
+	private TestingResourceEventHandler(
+			Consumer<Collection<WorkerType>> onPreviousAttemptWorkersRecoveredConsumer,
+			Consumer<ResourceID> onWorkerTerminatedConsumer,
+			Consumer<Throwable> onErrorConsumer) {
+		this.onPreviousAttemptWorkersRecoveredConsumer = onPreviousAttemptWorkersRecoveredConsumer;
+		this.onWorkerTerminatedConsumer = onWorkerTerminatedConsumer;
+		this.onErrorConsumer = onErrorConsumer;
+	}
+
+	@Override
+	public void onPreviousAttemptWorkersRecovered(Collection<WorkerType> recoveredWorkers) {
+		onPreviousAttemptWorkersRecoveredConsumer.accept(recoveredWorkers);
+	}
+
+	@Override
+	public void onWorkerTerminated(ResourceID resourceId) {
+		onWorkerTerminatedConsumer.accept(resourceId);
+	}
+
+	@Override
+	public void onError(Throwable exception) {
+		onErrorConsumer.accept(exception);
+	}
+
+	public static class Builder<WorkerType extends ResourceIDRetrievable> {

Review comment:
       Checkstyle complains about missing JavaDoc here.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManagerDriver.java
##########
@@ -0,0 +1,324 @@
+/*
+ * 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 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 = configuration.getClusterId();
+		this.podCreationRetryInterval = configuration.getPodCreationRetryInterval();
+		this.kubeClient = kubeClient;

Review comment:
       Should we add a `null` check here?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/AbstractResourceManagerDriver.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.runtime.resourcemanager.active;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Abstract common base class for implementations of {@link ResourceManagerDriver}.
+ */
+public abstract class AbstractResourceManagerDriver<WorkerType extends ResourceIDRetrievable>
+	implements ResourceManagerDriver<WorkerType> {
+
+	protected final Logger log = LoggerFactory.getLogger(getClass());
+
+	protected final Configuration flinkConfig;
+	protected final Configuration flinkClientConfig;
+
+	private ResourceEventHandler<WorkerType> resourceEventHandler = null;
+	private ScheduledExecutor mainThreadExecutor = null;
+
+	public AbstractResourceManagerDriver(
+			final Configuration flinkConfig,
+			final Configuration flinkClientConfig) {
+		this.flinkConfig = Preconditions.checkNotNull(flinkConfig);
+		this.flinkClientConfig = Preconditions.checkNotNull(flinkClientConfig);
+	}
+
+	protected final ResourceEventHandler<WorkerType> getResourceEventHandler() {
+		return Preconditions.checkNotNull(

Review comment:
       How about using `IllegalStateException` instead of `NullPointerException` here? Wouldn't that describe the error class in a better way?

##########
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 final static String CLUSTER_ID = "testing-flink-cluster";
+	private final static Time POD_CREATION_INTERVAL = Time.milliseconds(50L);
+	private final static KubernetesResourceManagerDriverConfiguration KUBERNETES_RESOURCE_MANAGER_CONFIGURATION =
+			new KubernetesResourceManagerDriverConfiguration(CLUSTER_ID, POD_CREATION_INTERVAL);

Review comment:
       Checkstyle is complaining about the order of keywords here: `private static final` is the way to go.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManagerDriver.java
##########
@@ -0,0 +1,324 @@
+/*
+ * 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 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 = configuration.getClusterId();
+		this.podCreationRetryInterval = configuration.getPodCreationRetryInterval();
+		this.kubeClient = kubeClient;
+		requestResourceFutures = new HashMap<>();
+		podCreationCoolDown = FutureUtils.completedVoidFuture();

Review comment:
       Maybe, align the member access: either use `this.` in all cases or in non.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/TestingFlinkKubeClient.java
##########
@@ -0,0 +1,170 @@
+/*
+ * 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;
+
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesPod;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesService;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.BiFunction;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * Testing implementation of {@link FlinkKubeClient}.
+ */
+public class TestingFlinkKubeClient implements FlinkKubeClient {
+
+	private final Function<KubernetesPod, CompletableFuture<Void>> createTaskManagerPodFunction;
+	private final Function<String, CompletableFuture<Void>> stopPodFunction;
+	private final Consumer<String> stopAndCleanupClusterConsumer;
+	private final Function<Map<String, String>, List<KubernetesPod>> getPodsWithLabelsFunction;
+	private final BiFunction<Map<String, String>, PodCallbackHandler, KubernetesWatch> watchPodsAndDoCallbackFunction;
+
+	private TestingFlinkKubeClient(
+			Function<KubernetesPod, CompletableFuture<Void>> createTaskManagerPodFunction,
+			Function<String, CompletableFuture<Void>> stopPodFunction,
+			Consumer<String> stopAndCleanupClusterConsumer,
+			Function<Map<String, String>, List<KubernetesPod>> getPodsWithLabelsFunction,
+			BiFunction<Map<String, String>, PodCallbackHandler, KubernetesWatch> watchPodsAndDoCallbackFunction) {
+
+		this.createTaskManagerPodFunction = createTaskManagerPodFunction;
+		this.stopPodFunction = stopPodFunction;
+		this.stopAndCleanupClusterConsumer = stopAndCleanupClusterConsumer;
+		this.getPodsWithLabelsFunction = getPodsWithLabelsFunction;
+		this.watchPodsAndDoCallbackFunction = watchPodsAndDoCallbackFunction;
+	}
+
+	@Override
+	public void createJobManagerComponent(KubernetesJobManagerSpecification kubernetesJMSpec) {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public CompletableFuture<Void> createTaskManagerPod(KubernetesPod kubernetesPod) {
+		return createTaskManagerPodFunction.apply(kubernetesPod);
+	}
+
+	@Override
+	public CompletableFuture<Void> stopPod(String podName) {
+		return stopPodFunction.apply(podName);
+	}
+
+	@Override
+	public void stopAndCleanupCluster(String clusterId) {
+		stopAndCleanupClusterConsumer.accept(clusterId);
+	}
+
+	@Override
+	public Optional<KubernetesService> getRestService(String clusterId) {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public Optional<Endpoint> getRestEndpoint(String clusterId) {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public List<KubernetesPod> getPodsWithLabels(Map<String, String> labels) {
+		return getPodsWithLabelsFunction.apply(labels);
+	}
+
+	@Override
+	public void handleException(Exception e) {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public KubernetesWatch watchPodsAndDoCallback(Map<String, String> labels, PodCallbackHandler podCallbackHandler) {
+		return watchPodsAndDoCallbackFunction.apply(labels, podCallbackHandler);
+	}
+
+	@Override
+	public void close() throws Exception {
+		// noop
+	}
+
+	public static class Builder {

Review comment:
       Syntactic sugar: Using a `static` method `TestingFlinkKubeClient.builder()` instead of calling `new TestingFlinkKubeClient.Builder()` could be used here as well.

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/TestingFlinkKubeClient.java
##########
@@ -0,0 +1,170 @@
+/*
+ * 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;
+
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesPod;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesService;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.BiFunction;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * Testing implementation of {@link FlinkKubeClient}.
+ */
+public class TestingFlinkKubeClient implements FlinkKubeClient {
+
+	private final Function<KubernetesPod, CompletableFuture<Void>> createTaskManagerPodFunction;
+	private final Function<String, CompletableFuture<Void>> stopPodFunction;
+	private final Consumer<String> stopAndCleanupClusterConsumer;
+	private final Function<Map<String, String>, List<KubernetesPod>> getPodsWithLabelsFunction;
+	private final BiFunction<Map<String, String>, PodCallbackHandler, KubernetesWatch> watchPodsAndDoCallbackFunction;
+
+	private TestingFlinkKubeClient(
+			Function<KubernetesPod, CompletableFuture<Void>> createTaskManagerPodFunction,
+			Function<String, CompletableFuture<Void>> stopPodFunction,
+			Consumer<String> stopAndCleanupClusterConsumer,
+			Function<Map<String, String>, List<KubernetesPod>> getPodsWithLabelsFunction,
+			BiFunction<Map<String, String>, PodCallbackHandler, KubernetesWatch> watchPodsAndDoCallbackFunction) {
+
+		this.createTaskManagerPodFunction = createTaskManagerPodFunction;
+		this.stopPodFunction = stopPodFunction;
+		this.stopAndCleanupClusterConsumer = stopAndCleanupClusterConsumer;
+		this.getPodsWithLabelsFunction = getPodsWithLabelsFunction;
+		this.watchPodsAndDoCallbackFunction = watchPodsAndDoCallbackFunction;
+	}
+
+	@Override
+	public void createJobManagerComponent(KubernetesJobManagerSpecification kubernetesJMSpec) {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public CompletableFuture<Void> createTaskManagerPod(KubernetesPod kubernetesPod) {
+		return createTaskManagerPodFunction.apply(kubernetesPod);
+	}
+
+	@Override
+	public CompletableFuture<Void> stopPod(String podName) {
+		return stopPodFunction.apply(podName);
+	}
+
+	@Override
+	public void stopAndCleanupCluster(String clusterId) {
+		stopAndCleanupClusterConsumer.accept(clusterId);
+	}
+
+	@Override
+	public Optional<KubernetesService> getRestService(String clusterId) {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public Optional<Endpoint> getRestEndpoint(String clusterId) {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public List<KubernetesPod> getPodsWithLabels(Map<String, String> labels) {
+		return getPodsWithLabelsFunction.apply(labels);
+	}
+
+	@Override
+	public void handleException(Exception e) {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public KubernetesWatch watchPodsAndDoCallback(Map<String, String> labels, PodCallbackHandler podCallbackHandler) {
+		return watchPodsAndDoCallbackFunction.apply(labels, podCallbackHandler);
+	}
+
+	@Override
+	public void close() throws Exception {
+		// noop
+	}
+
+	public static class Builder {
+		private Function<KubernetesPod, CompletableFuture<Void>> createTaskManagerPodFunction =
+				(ignore) -> FutureUtils.completedVoidFuture();
+		private Function<String, CompletableFuture<Void>> stopPodFunction =
+				(ignore) -> FutureUtils.completedVoidFuture();
+		private Consumer<String> stopAndCleanupClusterConsumer =
+				(ignore) -> {};
+		private Function<Map<String, String>, List<KubernetesPod>> getPodsWithLabelsFunction =
+				(ignore) -> Collections.emptyList();
+		private Consumer<Exception> handleExceptionConsumer =
+				(ignore) -> {};

Review comment:
       The `handleExceptionConsumer` is never used. Did you miss adding it to the `TestingFlinkKubeClient` constructor. `TestingFlinkKubeClient.handleException(Exception)` is throwing an `UnsupportedOperationException()` right now. This needs to be fixed as well, if you decide to actually use the `handleExceptionConsumer`. Otherwise, the member can be removed.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/active/ResourceManagerDriverTestBase.java
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.runtime.resourcemanager.active;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.clusterframework.ApplicationStatus;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils;
+import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.Test;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Common test cases for implementations of {@link ResourceManagerDriver}.
+ */
+public abstract class ResourceManagerDriverTestBase<WorkerType extends ResourceIDRetrievable> extends TestLogger {
+
+	protected static final long TIMEOUT_SEC = 5L;
+
+	protected static final TaskExecutorProcessSpec TASK_EXECUTOR_PROCESS_SPEC = TaskExecutorProcessUtils
+			.processSpecFromWorkerResourceSpec(new Configuration(), WorkerResourceSpec.ZERO);
+
+	private static final String MAIN_THREAD_NAME = "testing-rpc-main-thread";
+	private static final ScheduledExecutor MAIN_THREAD_EXECUTOR =
+			new ScheduledExecutorServiceAdapter(Executors.newSingleThreadScheduledExecutor(runnable -> new Thread(runnable, MAIN_THREAD_NAME)));
+
+	@Test
+	public void testInitialize() throws Exception {
+		final Context context = createContext();
+		context.runTest(context::validateInitialization);
+	}
+
+	@Test
+	public void testRecoverPreviousAttemptWorkers() throws Exception {
+		final CompletableFuture<Collection<WorkerType>> recoveredWorkersFuture = new CompletableFuture<>();
+		final Context context = createContext();
+		context.resourceEventHandlerBuilder.setOnPreviousAttemptWorkersRecoveredConsumer(recoveredWorkersFuture::complete);
+		context.preparePreviousAttemptWorkers();
+		context.runTest(() -> context.validateWorkersRecoveredFromPreviousAttempt(recoveredWorkersFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS)));
+	}
+
+	@Test
+	public void testTerminate() throws Exception {
+		final Context context = createContext();
+		context.runTest(() -> {
+			context.getDriver().terminate();
+			context.validateTermination();
+		});
+	}
+
+	@Test
+	public void testDeregisterApplication() throws Exception {
+		final Context context = createContext();
+		context.runTest(() -> {
+			context.getDriver().deregisterApplication(ApplicationStatus.SUCCEEDED, null);

Review comment:
       I realize that the value is not really used in the actual implementation. But shouldn't that be reflected in the test in a way that all values of the enum are tested having the same outcome?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/AbstractResourceManagerDriver.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.runtime.resourcemanager.active;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Abstract common base class for implementations of {@link ResourceManagerDriver}.
+ */
+public abstract class AbstractResourceManagerDriver<WorkerType extends ResourceIDRetrievable>
+	implements ResourceManagerDriver<WorkerType> {
+
+	protected final Logger log = LoggerFactory.getLogger(getClass());
+
+	protected final Configuration flinkConfig;
+	protected final Configuration flinkClientConfig;
+
+	private ResourceEventHandler<WorkerType> resourceEventHandler = null;
+	private ScheduledExecutor mainThreadExecutor = null;
+
+	public AbstractResourceManagerDriver(
+			final Configuration flinkConfig,
+			final Configuration flinkClientConfig) {
+		this.flinkConfig = Preconditions.checkNotNull(flinkConfig);
+		this.flinkClientConfig = Preconditions.checkNotNull(flinkClientConfig);
+	}
+
+	protected final ResourceEventHandler<WorkerType> getResourceEventHandler() {
+		return Preconditions.checkNotNull(
+				this.resourceEventHandler,
+				"Cannot get resource event handler. Resource manager driver is not initialized.");
+	}
+
+	protected final ScheduledExecutor getMainThreadExecutor() {
+		return Preconditions.checkNotNull(

Review comment:
       Same here: I'd say that throwing an `IllegalStateException` would be a better match for the actual error here since having the private member being set to `null` is more like an implementation detail that does not need to be exposed.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/active/TestingResourceEventHandler.java
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.runtime.resourcemanager.active;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Collection;
+import java.util.function.Consumer;
+
+/**
+ * Testing implementation of {@link ResourceEventHandler}.
+ */
+public class TestingResourceEventHandler<WorkerType extends ResourceIDRetrievable> implements ResourceEventHandler<WorkerType> {
+
+	private final Consumer<Collection<WorkerType>> onPreviousAttemptWorkersRecoveredConsumer;
+	private final Consumer<ResourceID> onWorkerTerminatedConsumer;
+	private final Consumer<Throwable> onErrorConsumer;
+
+	private TestingResourceEventHandler(
+			Consumer<Collection<WorkerType>> onPreviousAttemptWorkersRecoveredConsumer,
+			Consumer<ResourceID> onWorkerTerminatedConsumer,
+			Consumer<Throwable> onErrorConsumer) {
+		this.onPreviousAttemptWorkersRecoveredConsumer = onPreviousAttemptWorkersRecoveredConsumer;
+		this.onWorkerTerminatedConsumer = onWorkerTerminatedConsumer;
+		this.onErrorConsumer = onErrorConsumer;
+	}
+
+	@Override
+	public void onPreviousAttemptWorkersRecovered(Collection<WorkerType> recoveredWorkers) {
+		onPreviousAttemptWorkersRecoveredConsumer.accept(recoveredWorkers);
+	}
+
+	@Override
+	public void onWorkerTerminated(ResourceID resourceId) {
+		onWorkerTerminatedConsumer.accept(resourceId);
+	}
+
+	@Override
+	public void onError(Throwable exception) {
+		onErrorConsumer.accept(exception);
+	}
+
+	public static class Builder<WorkerType extends ResourceIDRetrievable> {
+		private Consumer<Collection<WorkerType>> onPreviousAttemptWorkersRecoveredConsumer = (ignore) -> {};
+		private Consumer<ResourceID> onWorkerTerminatedConsumer = (ignore) -> {};
+		private Consumer<Throwable> onErrorConsumer = (ignore) -> {};
+
+		public Builder<WorkerType> setOnPreviousAttemptWorkersRecoveredConsumer(

Review comment:
       That's just syntactic sugar, but: I always like having a static `builder()` method in the parent class instead of a public `Builder` constructor. This way, you won't need to use `new` when instantiating the `Builder` (like it's done [here](https://github.com/apache/flink/blob/master/flink-runtime/src/test/java/org/apache/flink/runtime/rest/util/TestRestServerEndpoint.java#L39-L53)).
   
   But that's just a suggestion...

##########
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 final static String CLUSTER_ID = "testing-flink-cluster";
+	private final static Time POD_CREATION_INTERVAL = Time.milliseconds(50L);
+	private final static 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 PREVIOUS_ATTEMPT_POD = new TestingKubernetesPod(CLUSTER_ID + "-taskmanager-1-1");

Review comment:
       Checkstyle: non-`static` members should be camel-cased instead of snake-cased.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/active/ResourceManagerDriverTestBase.java
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.runtime.resourcemanager.active;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.clusterframework.ApplicationStatus;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils;
+import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.Test;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Common test cases for implementations of {@link ResourceManagerDriver}.
+ */
+public abstract class ResourceManagerDriverTestBase<WorkerType extends ResourceIDRetrievable> extends TestLogger {
+
+	protected static final long TIMEOUT_SEC = 5L;
+
+	protected static final TaskExecutorProcessSpec TASK_EXECUTOR_PROCESS_SPEC = TaskExecutorProcessUtils
+			.processSpecFromWorkerResourceSpec(new Configuration(), WorkerResourceSpec.ZERO);
+
+	private static final String MAIN_THREAD_NAME = "testing-rpc-main-thread";
+	private static final ScheduledExecutor MAIN_THREAD_EXECUTOR =
+			new ScheduledExecutorServiceAdapter(Executors.newSingleThreadScheduledExecutor(runnable -> new Thread(runnable, MAIN_THREAD_NAME)));
+
+	@Test
+	public void testInitialize() throws Exception {
+		final Context context = createContext();
+		context.runTest(context::validateInitialization);
+	}
+
+	@Test
+	public void testRecoverPreviousAttemptWorkers() throws Exception {
+		final CompletableFuture<Collection<WorkerType>> recoveredWorkersFuture = new CompletableFuture<>();
+		final Context context = createContext();
+		context.resourceEventHandlerBuilder.setOnPreviousAttemptWorkersRecoveredConsumer(recoveredWorkersFuture::complete);
+		context.preparePreviousAttemptWorkers();
+		context.runTest(() -> context.validateWorkersRecoveredFromPreviousAttempt(recoveredWorkersFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS)));
+	}
+
+	@Test
+	public void testTerminate() throws Exception {
+		final Context context = createContext();
+		context.runTest(() -> {
+			context.getDriver().terminate();
+			context.validateTermination();
+		});
+	}
+
+	@Test
+	public void testDeregisterApplication() throws Exception {
+		final Context context = createContext();
+		context.runTest(() -> {
+			context.getDriver().deregisterApplication(ApplicationStatus.SUCCEEDED, null);
+			context.validateDeregisterApplication();
+		});
+	}
+
+	@Test
+	public void testRequestResource() throws Exception {
+		final Context context = createContext();
+		context.runTest(() -> {
+			context.runInMainThread(() -> context.getDriver().requestResource(TASK_EXECUTOR_PROCESS_SPEC));
+			context.validateRequestedResources(Collections.singleton(TASK_EXECUTOR_PROCESS_SPEC));
+		});
+	}
+
+	@Test
+	public void testReleaseResource() throws Exception {
+		final CompletableFuture<WorkerType> requestResourceFuture = new CompletableFuture<>();
+		final CompletableFuture<WorkerType> releaseResourceFuture = new CompletableFuture<>();
+		final Context context = createContext();
+		context.runTest(() -> {
+			context.runInMainThread(() -> context.getDriver()
+					.requestResource(TASK_EXECUTOR_PROCESS_SPEC)
+					.thenAccept(requestResourceFuture::complete));
+			requestResourceFuture.thenApply((workerNode) ->
+					context.runInMainThread(() -> {
+						context.getDriver().releaseResource(workerNode);
+						releaseResourceFuture.complete(workerNode);
+					}));
+			final WorkerType worker = releaseResourceFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS);
+			context.validateReleaseResources(Collections.singleton(worker));
+		});
+	}
+
+	protected abstract Context createContext();
+
+	protected abstract class Context {

Review comment:
       Checkstyle: Missing JavaDoc

##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/TestingFlinkKubeClient.java
##########
@@ -0,0 +1,170 @@
+/*
+ * 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;
+
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesPod;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesService;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.BiFunction;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * Testing implementation of {@link FlinkKubeClient}.
+ */
+public class TestingFlinkKubeClient implements FlinkKubeClient {
+
+	private final Function<KubernetesPod, CompletableFuture<Void>> createTaskManagerPodFunction;
+	private final Function<String, CompletableFuture<Void>> stopPodFunction;
+	private final Consumer<String> stopAndCleanupClusterConsumer;
+	private final Function<Map<String, String>, List<KubernetesPod>> getPodsWithLabelsFunction;
+	private final BiFunction<Map<String, String>, PodCallbackHandler, KubernetesWatch> watchPodsAndDoCallbackFunction;
+
+	private TestingFlinkKubeClient(
+			Function<KubernetesPod, CompletableFuture<Void>> createTaskManagerPodFunction,
+			Function<String, CompletableFuture<Void>> stopPodFunction,
+			Consumer<String> stopAndCleanupClusterConsumer,
+			Function<Map<String, String>, List<KubernetesPod>> getPodsWithLabelsFunction,
+			BiFunction<Map<String, String>, PodCallbackHandler, KubernetesWatch> watchPodsAndDoCallbackFunction) {
+
+		this.createTaskManagerPodFunction = createTaskManagerPodFunction;
+		this.stopPodFunction = stopPodFunction;
+		this.stopAndCleanupClusterConsumer = stopAndCleanupClusterConsumer;
+		this.getPodsWithLabelsFunction = getPodsWithLabelsFunction;
+		this.watchPodsAndDoCallbackFunction = watchPodsAndDoCallbackFunction;
+	}
+
+	@Override
+	public void createJobManagerComponent(KubernetesJobManagerSpecification kubernetesJMSpec) {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public CompletableFuture<Void> createTaskManagerPod(KubernetesPod kubernetesPod) {
+		return createTaskManagerPodFunction.apply(kubernetesPod);
+	}
+
+	@Override
+	public CompletableFuture<Void> stopPod(String podName) {
+		return stopPodFunction.apply(podName);
+	}
+
+	@Override
+	public void stopAndCleanupCluster(String clusterId) {
+		stopAndCleanupClusterConsumer.accept(clusterId);
+	}
+
+	@Override
+	public Optional<KubernetesService> getRestService(String clusterId) {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public Optional<Endpoint> getRestEndpoint(String clusterId) {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public List<KubernetesPod> getPodsWithLabels(Map<String, String> labels) {
+		return getPodsWithLabelsFunction.apply(labels);
+	}
+
+	@Override
+	public void handleException(Exception e) {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public KubernetesWatch watchPodsAndDoCallback(Map<String, String> labels, PodCallbackHandler podCallbackHandler) {
+		return watchPodsAndDoCallbackFunction.apply(labels, podCallbackHandler);
+	}
+
+	@Override
+	public void close() throws Exception {
+		// noop
+	}
+
+	public static class Builder {

Review comment:
       CheckStyle: Missing JavaDoc

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/active/ResourceManagerDriverTestBase.java
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.runtime.resourcemanager.active;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.clusterframework.ApplicationStatus;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils;
+import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.Test;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Common test cases for implementations of {@link ResourceManagerDriver}.
+ */
+public abstract class ResourceManagerDriverTestBase<WorkerType extends ResourceIDRetrievable> extends TestLogger {
+
+	protected static final long TIMEOUT_SEC = 5L;
+
+	protected static final TaskExecutorProcessSpec TASK_EXECUTOR_PROCESS_SPEC = TaskExecutorProcessUtils
+			.processSpecFromWorkerResourceSpec(new Configuration(), WorkerResourceSpec.ZERO);
+
+	private static final String MAIN_THREAD_NAME = "testing-rpc-main-thread";
+	private static final ScheduledExecutor MAIN_THREAD_EXECUTOR =
+			new ScheduledExecutorServiceAdapter(Executors.newSingleThreadScheduledExecutor(runnable -> new Thread(runnable, MAIN_THREAD_NAME)));
+
+	@Test
+	public void testInitialize() throws Exception {
+		final Context context = createContext();
+		context.runTest(context::validateInitialization);
+	}
+
+	@Test
+	public void testRecoverPreviousAttemptWorkers() throws Exception {
+		final CompletableFuture<Collection<WorkerType>> recoveredWorkersFuture = new CompletableFuture<>();
+		final Context context = createContext();
+		context.resourceEventHandlerBuilder.setOnPreviousAttemptWorkersRecoveredConsumer(recoveredWorkersFuture::complete);
+		context.preparePreviousAttemptWorkers();
+		context.runTest(() -> context.validateWorkersRecoveredFromPreviousAttempt(recoveredWorkersFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS)));
+	}
+
+	@Test
+	public void testTerminate() throws Exception {
+		final Context context = createContext();
+		context.runTest(() -> {
+			context.getDriver().terminate();
+			context.validateTermination();
+		});
+	}
+
+	@Test
+	public void testDeregisterApplication() throws Exception {
+		final Context context = createContext();
+		context.runTest(() -> {
+			context.getDriver().deregisterApplication(ApplicationStatus.SUCCEEDED, null);

Review comment:
       Is it enough to test for `ApplicationStatus.SUCCEEDED` here? I'm asking since there are other values in the `ApplicationStatus` enum that are not tested.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/ActiveResourceManagerFactory.java
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.runtime.resourcemanager.active;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable;
+import org.apache.flink.runtime.entrypoint.ClusterInformation;
+import org.apache.flink.runtime.heartbeat.HeartbeatServices;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.io.network.partition.ResourceManagerPartitionTrackerImpl;
+import org.apache.flink.runtime.metrics.MetricRegistry;
+import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManager;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerFactory;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServices;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.rpc.RpcService;
+
+import javax.annotation.Nullable;
+
+/**
+ * Factory class for creating {@link ActiveResourceManager} with various implementations of {@link ResourceManagerDriver}.
+ */
+public abstract class ActiveResourceManagerFactory<WorkerType extends ResourceIDRetrievable>
+		extends ResourceManagerFactory<WorkerType> {
+
+	@Override
+	public ResourceManager<WorkerType> createResourceManager(
+			Configuration configuration,
+			ResourceID resourceId,
+			RpcService rpcService,
+			HighAvailabilityServices highAvailabilityServices,
+			HeartbeatServices heartbeatServices,
+			FatalErrorHandler fatalErrorHandler,
+			ClusterInformation clusterInformation,
+			@Nullable String webInterfaceUrl,
+			MetricRegistry metricRegistry,
+			String hostname) throws Exception {
+		return super.createResourceManager(
+				createActiveResourceManagerConfiguration(configuration),
+				resourceId,
+				rpcService,
+				highAvailabilityServices,
+				heartbeatServices,
+				fatalErrorHandler,
+				clusterInformation,
+				webInterfaceUrl,
+				metricRegistry,
+				hostname);
+	}
+
+	private Configuration createActiveResourceManagerConfiguration(Configuration originalConfiguration) {
+		final Configuration copiedConfig = new Configuration(originalConfiguration);
+		// In active mode, it's depend on the ResourceManager to set the ResourceID of TaskManagers.

Review comment:
       ```suggestion
   		// In active mode, it depends on the ResourceManager to set the ResourceID of TaskManagers.
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13186:
URL: https://github.com/apache/flink/pull/13186#issuecomment-675299227


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5652",
       "triggerID" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5668",
       "triggerID" : "675358004",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 3a8705a4c617a0f76847c5c3f262043fe75b8942 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5652) Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5668) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13186:
URL: https://github.com/apache/flink/pull/13186#discussion_r477359862



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/AbstractResourceManagerDriver.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.runtime.resourcemanager.active;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Abstract common base class for implementations of {@link ResourceManagerDriver}.
+ */
+public abstract class AbstractResourceManagerDriver<WorkerType extends ResourceIDRetrievable>
+	implements ResourceManagerDriver<WorkerType> {
+
+	protected final Logger log = LoggerFactory.getLogger(getClass());
+
+	protected final Configuration flinkConfig;
+	protected final Configuration flinkClientConfig;

Review comment:
       It might be ok at the moment because the current entrypoints ensure that the assumptions are respected. However, nobody says that this won't change in the future. The farther these assumptions are away from each other (e.g. `KubernetesResourceManagerDriver` assumes that the corresponding entrypoint sets up the right env variables), the more brittle it is. Hence, I would suggest to rethink the usage of `GlobalConfiguration` in the future.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13186:
URL: https://github.com/apache/flink/pull/13186#issuecomment-675299227


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5652",
       "triggerID" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5668",
       "triggerID" : "675358004",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f2e824b2daf427f49409e1ea5c7e9704a84c76d9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5676",
       "triggerID" : "f2e824b2daf427f49409e1ea5c7e9704a84c76d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd4e2812d845277e7f4bc68e97f4759e11334ca3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5680",
       "triggerID" : "bd4e2812d845277e7f4bc68e97f4759e11334ca3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "523cf170fe07e4b32e47653af46bf2290935d87a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5692",
       "triggerID" : "523cf170fe07e4b32e47653af46bf2290935d87a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * bd4e2812d845277e7f4bc68e97f4759e11334ca3 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5680) 
   * 523cf170fe07e4b32e47653af46bf2290935d87a Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5692) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] xintongsong commented on pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
xintongsong commented on pull request #13186:
URL: https://github.com/apache/flink/pull/13186#issuecomment-681302816


   Thanks all. I've created FLINK-19057 for tracking the `flinkClientConfig` issue.
   Merging this.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13186:
URL: https://github.com/apache/flink/pull/13186#issuecomment-675299227


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5652",
       "triggerID" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5668",
       "triggerID" : "675358004",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f2e824b2daf427f49409e1ea5c7e9704a84c76d9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5676",
       "triggerID" : "f2e824b2daf427f49409e1ea5c7e9704a84c76d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd4e2812d845277e7f4bc68e97f4759e11334ca3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5680",
       "triggerID" : "bd4e2812d845277e7f4bc68e97f4759e11334ca3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "523cf170fe07e4b32e47653af46bf2290935d87a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5692",
       "triggerID" : "523cf170fe07e4b32e47653af46bf2290935d87a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "63c2f658cc309661c434451a5d8c0b38d2e748d3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5848",
       "triggerID" : "679880992",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "63c2f658cc309661c434451a5d8c0b38d2e748d3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5848",
       "triggerID" : "63c2f658cc309661c434451a5d8c0b38d2e748d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "679880992",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 63c2f658cc309661c434451a5d8c0b38d2e748d3 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5848) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13186:
URL: https://github.com/apache/flink/pull/13186#issuecomment-675299227


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5652",
       "triggerID" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5668",
       "triggerID" : "675358004",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f2e824b2daf427f49409e1ea5c7e9704a84c76d9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5676",
       "triggerID" : "f2e824b2daf427f49409e1ea5c7e9704a84c76d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd4e2812d845277e7f4bc68e97f4759e11334ca3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5680",
       "triggerID" : "bd4e2812d845277e7f4bc68e97f4759e11334ca3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "523cf170fe07e4b32e47653af46bf2290935d87a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5692",
       "triggerID" : "523cf170fe07e4b32e47653af46bf2290935d87a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "63c2f658cc309661c434451a5d8c0b38d2e748d3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5848",
       "triggerID" : "679880992",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "63c2f658cc309661c434451a5d8c0b38d2e748d3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5848",
       "triggerID" : "63c2f658cc309661c434451a5d8c0b38d2e748d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "679880992",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "952d92954bb418aa30820b811bc5a41f76e9ab21",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5867",
       "triggerID" : "952d92954bb418aa30820b811bc5a41f76e9ab21",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 63c2f658cc309661c434451a5d8c0b38d2e748d3 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5848) 
   * 952d92954bb418aa30820b811bc5a41f76e9ab21 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5867) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] xintongsong commented on pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
xintongsong commented on pull request #13186:
URL: https://github.com/apache/flink/pull/13186#issuecomment-679880992


   @flinkbot run azure


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



[GitHub] [flink] xintongsong commented on a change in pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
xintongsong commented on a change in pull request #13186:
URL: https://github.com/apache/flink/pull/13186#discussion_r476260518



##########
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:
       I believe there is another ticket (FLINK-17273) tracking this issue.
   Since this is also a behavior change, and is not necessarily related to this refracting effort, I would be in favor of making it in a separate PR.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] xintongsong commented on pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
xintongsong commented on pull request #13186:
URL: https://github.com/apache/flink/pull/13186#issuecomment-675426724


   @XComp, thanks for the review and suggestions. Comments addressed.
   Sorry for the many checkstyle problems. I just realized that my last checkstyle fix commit was not successfully pushed.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13186:
URL: https://github.com/apache/flink/pull/13186#issuecomment-675299227


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5652",
       "triggerID" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3a8705a4c617a0f76847c5c3f262043fe75b8942 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5652) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13186:
URL: https://github.com/apache/flink/pull/13186#issuecomment-675299227


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5652",
       "triggerID" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5668",
       "triggerID" : "675358004",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f2e824b2daf427f49409e1ea5c7e9704a84c76d9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5676",
       "triggerID" : "f2e824b2daf427f49409e1ea5c7e9704a84c76d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd4e2812d845277e7f4bc68e97f4759e11334ca3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5680",
       "triggerID" : "bd4e2812d845277e7f4bc68e97f4759e11334ca3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "523cf170fe07e4b32e47653af46bf2290935d87a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5692",
       "triggerID" : "523cf170fe07e4b32e47653af46bf2290935d87a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "63c2f658cc309661c434451a5d8c0b38d2e748d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5848",
       "triggerID" : "679880992",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "63c2f658cc309661c434451a5d8c0b38d2e748d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5848",
       "triggerID" : "63c2f658cc309661c434451a5d8c0b38d2e748d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "679880992",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "952d92954bb418aa30820b811bc5a41f76e9ab21",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5867",
       "triggerID" : "952d92954bb418aa30820b811bc5a41f76e9ab21",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 952d92954bb418aa30820b811bc5a41f76e9ab21 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5867) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13186:
URL: https://github.com/apache/flink/pull/13186#issuecomment-675299227


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5652",
       "triggerID" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5668",
       "triggerID" : "675358004",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f2e824b2daf427f49409e1ea5c7e9704a84c76d9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f2e824b2daf427f49409e1ea5c7e9704a84c76d9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3a8705a4c617a0f76847c5c3f262043fe75b8942 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5652) Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5668) 
   * f2e824b2daf427f49409e1ea5c7e9704a84c76d9 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] xintongsong commented on pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
xintongsong commented on pull request #13186:
URL: https://github.com/apache/flink/pull/13186#issuecomment-680741939


   It seems to me that the only open issue is regarding the `flinkClientConfig`, which was already there before this PR.
   If there's no other comment, I would like to merge this PR as it is, and open a follow-up issue where we can keep the discussion.
   WDYT? @tillrohrmann @wangyang0918 


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] xintongsong commented on pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
xintongsong commented on pull request #13186:
URL: https://github.com/apache/flink/pull/13186#issuecomment-679875405


   Thanks @wangyang0918 for the verification and review.
   Addressed comments, and rebased for resolving conflicts.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] xintongsong closed pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
xintongsong closed pull request #13186:
URL: https://github.com/apache/flink/pull/13186


   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] XComp commented on a change in pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #13186:
URL: https://github.com/apache/flink/pull/13186#discussion_r472090011



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManagerDriver.java
##########
@@ -0,0 +1,324 @@
+/*
+ * 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 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 = configuration.getClusterId();
+		this.podCreationRetryInterval = configuration.getPodCreationRetryInterval();
+		this.kubeClient = kubeClient;

Review comment:
       I realized that I selected too many lines here. I'm referring to line 94.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13186:
URL: https://github.com/apache/flink/pull/13186#issuecomment-675299227


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5652",
       "triggerID" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5668",
       "triggerID" : "675358004",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f2e824b2daf427f49409e1ea5c7e9704a84c76d9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5676",
       "triggerID" : "f2e824b2daf427f49409e1ea5c7e9704a84c76d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd4e2812d845277e7f4bc68e97f4759e11334ca3",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bd4e2812d845277e7f4bc68e97f4759e11334ca3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3a8705a4c617a0f76847c5c3f262043fe75b8942 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5652) Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5668) 
   * f2e824b2daf427f49409e1ea5c7e9704a84c76d9 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5676) 
   * bd4e2812d845277e7f4bc68e97f4759e11334ca3 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13186:
URL: https://github.com/apache/flink/pull/13186#issuecomment-675299227


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5652",
       "triggerID" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3a8705a4c617a0f76847c5c3f262043fe75b8942 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5652) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on a change in pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on a change in pull request #13186:
URL: https://github.com/apache/flink/pull/13186#discussion_r477905760



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/AbstractResourceManagerDriver.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.runtime.resourcemanager.active;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Abstract common base class for implementations of {@link ResourceManagerDriver}.
+ */
+public abstract class AbstractResourceManagerDriver<WorkerType extends ResourceIDRetrievable>
+	implements ResourceManagerDriver<WorkerType> {
+
+	protected final Logger log = LoggerFactory.getLogger(getClass());
+
+	protected final Configuration flinkConfig;
+	protected final Configuration flinkClientConfig;

Review comment:
       Yeah, i agree with you to rethink the usage of `GlobalConfiguration` and how to ship the dynamic config options to TaskManagers.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] xintongsong commented on a change in pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
xintongsong commented on a change in pull request #13186:
URL: https://github.com/apache/flink/pull/13186#discussion_r476973166



##########
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:
       My answer would be no.
   
   On Yarn, when calling `AMRMClientAsync.addContainerRequest`, the request is first added to the local client, then sent to Yarn RM via heartbeats. This prevents the requests from being failed and re-requested immediately.
   
   Not sure about Mesos though.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13186:
URL: https://github.com/apache/flink/pull/13186#discussion_r476473271



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManagerDriver.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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.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.KubernetesResourceManagerConfiguration;
+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 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;
+
+	public KubernetesResourceManagerDriver(
+			Configuration flinkConfig,
+			FlinkKubeClient kubeClient,
+			KubernetesResourceManagerConfiguration configuration) {
+		super(flinkConfig, GlobalConfiguration.loadConfiguration());
+
+		this.clusterId = Preconditions.checkNotNull(configuration.getClusterId());
+		this.kubeClient = Preconditions.checkNotNull(kubeClient);
+		this.requestResourceFutures = new HashMap<>();
+	}
+
+	// ------------------------------------------------------------------------
+	//  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());
+
+		// TODO: enable pod creation interval
+		// 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.
+
+		FutureUtils.assertNoException(
+				kubeClient.createTaskManagerPod(taskManagerPod)
+					.handleAsync((ignore, exception) -> {
+						if (exception != null) {
+							log.warn("Could not create pod {}, exception: {}", podName, exception);
+							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);

Review comment:
       Why is it necessary to reconstruct the `dynamicProperties` for `KubernetesTaskManagerParameters`? Wouldn't it be enough to pass on the `taskManagerConfig`?

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManagerDriver.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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.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.KubernetesResourceManagerConfiguration;
+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 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;
+
+	public KubernetesResourceManagerDriver(
+			Configuration flinkConfig,
+			FlinkKubeClient kubeClient,
+			KubernetesResourceManagerConfiguration configuration) {
+		super(flinkConfig, GlobalConfiguration.loadConfiguration());
+
+		this.clusterId = Preconditions.checkNotNull(configuration.getClusterId());
+		this.kubeClient = Preconditions.checkNotNull(kubeClient);
+		this.requestResourceFutures = new HashMap<>();
+	}
+
+	// ------------------------------------------------------------------------
+	//  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());
+
+		// TODO: enable pod creation interval
+		// 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.
+
+		FutureUtils.assertNoException(
+				kubeClient.createTaskManagerPod(taskManagerPod)
+					.handleAsync((ignore, exception) -> {
+						if (exception != null) {
+							log.warn("Could not create pod {}, exception: {}", podName, exception);
+							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));
+	}

Review comment:
       Nit: One could also introduce an inner class for `FlinkKubeClient.PodCallbackHandler` to keep the interface of the `KubernetesResourceManagerDriver` a bit leaner. 

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/AbstractResourceManagerDriver.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.runtime.resourcemanager.active;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Abstract common base class for implementations of {@link ResourceManagerDriver}.
+ */
+public abstract class AbstractResourceManagerDriver<WorkerType extends ResourceIDRetrievable>
+	implements ResourceManagerDriver<WorkerType> {
+
+	protected final Logger log = LoggerFactory.getLogger(getClass());
+
+	protected final Configuration flinkConfig;
+	protected final Configuration flinkClientConfig;

Review comment:
       Why do we need two configurations in the base class?

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManagerDriver.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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.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.KubernetesResourceManagerConfiguration;
+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 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;
+
+	public KubernetesResourceManagerDriver(
+			Configuration flinkConfig,
+			FlinkKubeClient kubeClient,
+			KubernetesResourceManagerConfiguration configuration) {
+		super(flinkConfig, GlobalConfiguration.loadConfiguration());
+
+		this.clusterId = Preconditions.checkNotNull(configuration.getClusterId());
+		this.kubeClient = Preconditions.checkNotNull(kubeClient);
+		this.requestResourceFutures = new HashMap<>();
+	}
+
+	// ------------------------------------------------------------------------
+	//  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());
+
+		// TODO: enable pod creation interval
+		// 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.

Review comment:
       Let's file right away a follow up issue for this problem.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManagerDriver.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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.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.KubernetesResourceManagerConfiguration;
+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 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;
+
+	public KubernetesResourceManagerDriver(
+			Configuration flinkConfig,
+			FlinkKubeClient kubeClient,
+			KubernetesResourceManagerConfiguration configuration) {
+		super(flinkConfig, GlobalConfiguration.loadConfiguration());
+
+		this.clusterId = Preconditions.checkNotNull(configuration.getClusterId());
+		this.kubeClient = Preconditions.checkNotNull(kubeClient);
+		this.requestResourceFutures = new HashMap<>();
+	}
+
+	// ------------------------------------------------------------------------
+	//  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: {}",

Review comment:
       ```suggestion
   		log.info("Deregistering Flink Kubernetes cluster, clusterId: {}, diagnostics: {}",
   ```

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManagerDriver.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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.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.KubernetesResourceManagerConfiguration;
+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 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;
+
+	public KubernetesResourceManagerDriver(
+			Configuration flinkConfig,
+			FlinkKubeClient kubeClient,
+			KubernetesResourceManagerConfiguration configuration) {
+		super(flinkConfig, GlobalConfiguration.loadConfiguration());
+
+		this.clusterId = Preconditions.checkNotNull(configuration.getClusterId());
+		this.kubeClient = Preconditions.checkNotNull(kubeClient);
+		this.requestResourceFutures = new HashMap<>();
+	}
+
+	// ------------------------------------------------------------------------
+	//  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());
+
+		// TODO: enable pod creation interval
+		// 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.
+
+		FutureUtils.assertNoException(
+				kubeClient.createTaskManagerPod(taskManagerPod)
+					.handleAsync((ignore, exception) -> {
+						if (exception != null) {
+							log.warn("Could not create pod {}, exception: {}", podName, exception);
+							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));

Review comment:
       Fatal error should be directly forwarded since they should lead to an immediate shut down of the system.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManagerDriver.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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.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.KubernetesResourceManagerConfiguration;
+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 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;
+
+	public KubernetesResourceManagerDriver(
+			Configuration flinkConfig,
+			FlinkKubeClient kubeClient,
+			KubernetesResourceManagerConfiguration configuration) {
+		super(flinkConfig, GlobalConfiguration.loadConfiguration());

Review comment:
       `GlobalConfiguration.loadConfiguration()` should only be used in the in the entrypoints of process and not somewhere in the code. The problem is that it is not guaranteed at this point that you can read a proper Flink configuration. Moreover, you are missing all dynamically configured properties which are processed in the entrypoints. Hence, I would suggest to not load the configuration here.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManagerDriver.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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.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.KubernetesResourceManagerConfiguration;
+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 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;
+
+	public KubernetesResourceManagerDriver(
+			Configuration flinkConfig,
+			FlinkKubeClient kubeClient,
+			KubernetesResourceManagerConfiguration configuration) {
+		super(flinkConfig, GlobalConfiguration.loadConfiguration());
+
+		this.clusterId = Preconditions.checkNotNull(configuration.getClusterId());
+		this.kubeClient = Preconditions.checkNotNull(kubeClient);
+		this.requestResourceFutures = new HashMap<>();
+	}
+
+	// ------------------------------------------------------------------------
+	//  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());
+
+		// TODO: enable pod creation interval
+		// 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.
+
+		FutureUtils.assertNoException(
+				kubeClient.createTaskManagerPod(taskManagerPod)
+					.handleAsync((ignore, exception) -> {
+						if (exception != null) {
+							log.warn("Could not create pod {}, exception: {}", podName, exception);
+							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);

Review comment:
       Let's add to `ActiveResourceManager.onPreviousAttemptWorkersRecovered` `getMainThreadExecutor().assertRunningInMainThread();` in order to make sure that we are indeed in the main thread when calling this method.

##########
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 might think about whether such a mechanism would make sense for all `ActiveResourceManagers`. If the answer is yes, then we could move the retry logic into the `ActiveResourceManager` class.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManagerDriver.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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.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.KubernetesResourceManagerConfiguration;
+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 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;
+
+	public KubernetesResourceManagerDriver(
+			Configuration flinkConfig,
+			FlinkKubeClient kubeClient,
+			KubernetesResourceManagerConfiguration configuration) {
+		super(flinkConfig, GlobalConfiguration.loadConfiguration());
+
+		this.clusterId = Preconditions.checkNotNull(configuration.getClusterId());
+		this.kubeClient = Preconditions.checkNotNull(kubeClient);
+		this.requestResourceFutures = new HashMap<>();
+	}
+
+	// ------------------------------------------------------------------------
+	//  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());
+
+		// TODO: enable pod creation interval
+		// 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.
+
+		FutureUtils.assertNoException(
+				kubeClient.createTaskManagerPod(taskManagerPod)
+					.handleAsync((ignore, exception) -> {
+						if (exception != null) {
+							log.warn("Could not create pod {}, exception: {}", podName, exception);
+							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 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));
+					removePod(podName);
+				}
+			}
+		});
+	}
+
+	private void removePod(String podName) {

Review comment:
       Maybe rename into `stopPod`.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] xintongsong commented on a change in pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
xintongsong commented on a change in pull request #13186:
URL: https://github.com/apache/flink/pull/13186#discussion_r476973166



##########
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:
       My answer would be no.
   
   On Yarn, when calling `AMRMClientAsync.addContainerRequest`, the request is first added to the local client, then sent to Yarn RM via heartbeats. This prevents the requests from being failed and re-requested immediately.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] xintongsong commented on pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
xintongsong commented on pull request #13186:
URL: https://github.com/apache/flink/pull/13186#issuecomment-680452263


   Thanks for the review, @tillrohrmann.
   I replied about the `flinkClientConfig`. The rest of the comments are addressed.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] xintongsong commented on pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
xintongsong commented on pull request #13186:
URL: https://github.com/apache/flink/pull/13186#issuecomment-675358004


   @flinkbot run azure


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] XComp commented on a change in pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #13186:
URL: https://github.com/apache/flink/pull/13186#discussion_r472222803



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/TestingFlinkKubeClient.java
##########
@@ -0,0 +1,180 @@
+/*
+ * 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;
+
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesPod;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesService;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.BiFunction;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * Testing implementation of {@link FlinkKubeClient}.
+ */
+public class TestingFlinkKubeClient implements FlinkKubeClient {
+
+	private final Function<KubernetesPod, CompletableFuture<Void>> createTaskManagerPodFunction;
+	private final Function<String, CompletableFuture<Void>> stopPodFunction;
+	private final Consumer<String> stopAndCleanupClusterConsumer;
+	private final Function<Map<String, String>, List<KubernetesPod>> getPodsWithLabelsFunction;
+	private final BiFunction<Map<String, String>, PodCallbackHandler, KubernetesWatch> watchPodsAndDoCallbackFunction;
+
+	private TestingFlinkKubeClient(
+			Function<KubernetesPod, CompletableFuture<Void>> createTaskManagerPodFunction,
+			Function<String, CompletableFuture<Void>> stopPodFunction,
+			Consumer<String> stopAndCleanupClusterConsumer,
+			Function<Map<String, String>, List<KubernetesPod>> getPodsWithLabelsFunction,
+			BiFunction<Map<String, String>, PodCallbackHandler, KubernetesWatch> watchPodsAndDoCallbackFunction) {
+
+		this.createTaskManagerPodFunction = createTaskManagerPodFunction;
+		this.stopPodFunction = stopPodFunction;
+		this.stopAndCleanupClusterConsumer = stopAndCleanupClusterConsumer;
+		this.getPodsWithLabelsFunction = getPodsWithLabelsFunction;
+		this.watchPodsAndDoCallbackFunction = watchPodsAndDoCallbackFunction;
+	}
+
+	@Override
+	public void createJobManagerComponent(KubernetesJobManagerSpecification kubernetesJMSpec) {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public CompletableFuture<Void> createTaskManagerPod(KubernetesPod kubernetesPod) {
+		return createTaskManagerPodFunction.apply(kubernetesPod);
+	}
+
+	@Override
+	public CompletableFuture<Void> stopPod(String podName) {
+		return stopPodFunction.apply(podName);
+	}
+
+	@Override
+	public void stopAndCleanupCluster(String clusterId) {
+		stopAndCleanupClusterConsumer.accept(clusterId);
+	}
+
+	@Override
+	public Optional<KubernetesService> getRestService(String clusterId) {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public Optional<Endpoint> getRestEndpoint(String clusterId) {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public List<KubernetesPod> getPodsWithLabels(Map<String, String> labels) {
+		return getPodsWithLabelsFunction.apply(labels);
+	}
+
+	@Override
+	public void handleException(Exception e) {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public KubernetesWatch watchPodsAndDoCallback(Map<String, String> labels, PodCallbackHandler podCallbackHandler) {
+		return watchPodsAndDoCallbackFunction.apply(labels, podCallbackHandler);
+	}
+
+	@Override
+	public void close() throws Exception {
+		// noop
+	}
+
+	public static Builder builder() {
+		return new Builder();
+	}
+
+	/**
+	 * Builder class for {@link TestingFlinkKubeClient}.
+	 */
+	public static class Builder {
+		private Function<KubernetesPod, CompletableFuture<Void>> createTaskManagerPodFunction =
+				(ignore) -> FutureUtils.completedVoidFuture();
+		private Function<String, CompletableFuture<Void>> stopPodFunction =
+				(ignore) -> FutureUtils.completedVoidFuture();
+		private Consumer<String> stopAndCleanupClusterConsumer =
+				(ignore) -> {};
+		private Function<Map<String, String>, List<KubernetesPod>> getPodsWithLabelsFunction =
+				(ignore) -> Collections.emptyList();
+		private BiFunction<Map<String, String>, PodCallbackHandler, KubernetesWatch> watchPodsAndDoCallbackFunction =
+				(ignore1, ignore2) -> new MockKubernetesWatch();
+
+		private Builder() {};

Review comment:
       The semicolon is not necessary.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wangyang0918 commented on pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on pull request #13186:
URL: https://github.com/apache/flink/pull/13186#issuecomment-681282597


   It also makes sense to me for merging the PR first and leave the discussion about `GlobalConfiguration` and `flinkClientConfig ` in a separate ticket.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13186:
URL: https://github.com/apache/flink/pull/13186#issuecomment-675299227


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5652",
       "triggerID" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5668",
       "triggerID" : "675358004",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f2e824b2daf427f49409e1ea5c7e9704a84c76d9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5676",
       "triggerID" : "f2e824b2daf427f49409e1ea5c7e9704a84c76d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd4e2812d845277e7f4bc68e97f4759e11334ca3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5680",
       "triggerID" : "bd4e2812d845277e7f4bc68e97f4759e11334ca3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "523cf170fe07e4b32e47653af46bf2290935d87a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "523cf170fe07e4b32e47653af46bf2290935d87a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * bd4e2812d845277e7f4bc68e97f4759e11334ca3 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5680) 
   * 523cf170fe07e4b32e47653af46bf2290935d87a UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] xintongsong commented on a change in pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
xintongsong commented on a change in pull request #13186:
URL: https://github.com/apache/flink/pull/13186#discussion_r472102547



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/active/ResourceManagerDriverTestBase.java
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.runtime.resourcemanager.active;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.clusterframework.ApplicationStatus;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils;
+import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.Test;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Common test cases for implementations of {@link ResourceManagerDriver}.
+ */
+public abstract class ResourceManagerDriverTestBase<WorkerType extends ResourceIDRetrievable> extends TestLogger {
+
+	protected static final long TIMEOUT_SEC = 5L;
+
+	protected static final TaskExecutorProcessSpec TASK_EXECUTOR_PROCESS_SPEC = TaskExecutorProcessUtils
+			.processSpecFromWorkerResourceSpec(new Configuration(), WorkerResourceSpec.ZERO);
+
+	private static final String MAIN_THREAD_NAME = "testing-rpc-main-thread";
+	private static final ScheduledExecutor MAIN_THREAD_EXECUTOR =
+			new ScheduledExecutorServiceAdapter(Executors.newSingleThreadScheduledExecutor(runnable -> new Thread(runnable, MAIN_THREAD_NAME)));
+
+	@Test
+	public void testInitialize() throws Exception {
+		final Context context = createContext();
+		context.runTest(context::validateInitialization);
+	}
+
+	@Test
+	public void testRecoverPreviousAttemptWorkers() throws Exception {
+		final CompletableFuture<Collection<WorkerType>> recoveredWorkersFuture = new CompletableFuture<>();
+		final Context context = createContext();
+		context.resourceEventHandlerBuilder.setOnPreviousAttemptWorkersRecoveredConsumer(recoveredWorkersFuture::complete);
+		context.preparePreviousAttemptWorkers();
+		context.runTest(() -> context.validateWorkersRecoveredFromPreviousAttempt(recoveredWorkersFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS)));
+	}
+
+	@Test
+	public void testTerminate() throws Exception {
+		final Context context = createContext();
+		context.runTest(() -> {
+			context.getDriver().terminate();
+			context.validateTermination();
+		});
+	}
+
+	@Test
+	public void testDeregisterApplication() throws Exception {
+		final Context context = createContext();
+		context.runTest(() -> {
+			context.getDriver().deregisterApplication(ApplicationStatus.SUCCEEDED, null);

Review comment:
       True, it does no harm to be too careful.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13186:
URL: https://github.com/apache/flink/pull/13186#discussion_r477077315



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManagerDriver.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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.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.KubernetesResourceManagerConfiguration;
+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 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;
+
+	public KubernetesResourceManagerDriver(
+			Configuration flinkConfig,
+			FlinkKubeClient kubeClient,
+			KubernetesResourceManagerConfiguration configuration) {
+		super(flinkConfig, GlobalConfiguration.loadConfiguration());

Review comment:
       The method relies on the fact that some environment variable is set. Depending on the how the process is setup this might be set or not. In the entrypoints we actually also consider a potentially configured configuration directory which is passed in as an argument. Hence, we might read completely different configurations here.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] xintongsong commented on pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
xintongsong commented on pull request #13186:
URL: https://github.com/apache/flink/pull/13186#issuecomment-675292244


   cc @tillrohrmann @XComp @wangyang0918 


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13186:
URL: https://github.com/apache/flink/pull/13186#issuecomment-675299227


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5652",
       "triggerID" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5668",
       "triggerID" : "675358004",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f2e824b2daf427f49409e1ea5c7e9704a84c76d9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5676",
       "triggerID" : "f2e824b2daf427f49409e1ea5c7e9704a84c76d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd4e2812d845277e7f4bc68e97f4759e11334ca3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5680",
       "triggerID" : "bd4e2812d845277e7f4bc68e97f4759e11334ca3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3a8705a4c617a0f76847c5c3f262043fe75b8942 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5652) Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5668) 
   * f2e824b2daf427f49409e1ea5c7e9704a84c76d9 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5676) 
   * bd4e2812d845277e7f4bc68e97f4759e11334ca3 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5680) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13186:
URL: https://github.com/apache/flink/pull/13186#issuecomment-675299227


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5652",
       "triggerID" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5668",
       "triggerID" : "675358004",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f2e824b2daf427f49409e1ea5c7e9704a84c76d9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5676",
       "triggerID" : "f2e824b2daf427f49409e1ea5c7e9704a84c76d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd4e2812d845277e7f4bc68e97f4759e11334ca3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5680",
       "triggerID" : "bd4e2812d845277e7f4bc68e97f4759e11334ca3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "523cf170fe07e4b32e47653af46bf2290935d87a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5692",
       "triggerID" : "523cf170fe07e4b32e47653af46bf2290935d87a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "63c2f658cc309661c434451a5d8c0b38d2e748d3",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "679880992",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "679880992",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "63c2f658cc309661c434451a5d8c0b38d2e748d3",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "63c2f658cc309661c434451a5d8c0b38d2e748d3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   *  Unknown: [CANCELED](TBD) 
   * 63c2f658cc309661c434451a5d8c0b38d2e748d3 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] xintongsong commented on a change in pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
xintongsong commented on a change in pull request #13186:
URL: https://github.com/apache/flink/pull/13186#discussion_r476996403



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManagerDriver.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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.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.KubernetesResourceManagerConfiguration;
+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 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;
+
+	public KubernetesResourceManagerDriver(
+			Configuration flinkConfig,
+			FlinkKubeClient kubeClient,
+			KubernetesResourceManagerConfiguration configuration) {
+		super(flinkConfig, GlobalConfiguration.loadConfiguration());
+
+		this.clusterId = Preconditions.checkNotNull(configuration.getClusterId());
+		this.kubeClient = Preconditions.checkNotNull(kubeClient);
+		this.requestResourceFutures = new HashMap<>();
+	}
+
+	// ------------------------------------------------------------------------
+	//  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());
+
+		// TODO: enable pod creation interval
+		// 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.
+
+		FutureUtils.assertNoException(
+				kubeClient.createTaskManagerPod(taskManagerPod)
+					.handleAsync((ignore, exception) -> {
+						if (exception != null) {
+							log.warn("Could not create pod {}, exception: {}", podName, exception);
+							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);

Review comment:
       It compares `taskManagerConfig` with `flinkClientConfig` (the original version shipped from client), and send only the differences.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13186:
URL: https://github.com/apache/flink/pull/13186#issuecomment-675299227


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5652",
       "triggerID" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5668",
       "triggerID" : "675358004",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f2e824b2daf427f49409e1ea5c7e9704a84c76d9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5676",
       "triggerID" : "f2e824b2daf427f49409e1ea5c7e9704a84c76d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd4e2812d845277e7f4bc68e97f4759e11334ca3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5680",
       "triggerID" : "bd4e2812d845277e7f4bc68e97f4759e11334ca3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "523cf170fe07e4b32e47653af46bf2290935d87a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5692",
       "triggerID" : "523cf170fe07e4b32e47653af46bf2290935d87a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "679880992",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "63c2f658cc309661c434451a5d8c0b38d2e748d3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5848",
       "triggerID" : "679880992",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "63c2f658cc309661c434451a5d8c0b38d2e748d3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5848",
       "triggerID" : "63c2f658cc309661c434451a5d8c0b38d2e748d3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   *  Unknown: [CANCELED](TBD) 
   * 63c2f658cc309661c434451a5d8c0b38d2e748d3 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5848) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13186:
URL: https://github.com/apache/flink/pull/13186#issuecomment-675299227


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5652",
       "triggerID" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5668",
       "triggerID" : "675358004",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f2e824b2daf427f49409e1ea5c7e9704a84c76d9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5676",
       "triggerID" : "f2e824b2daf427f49409e1ea5c7e9704a84c76d9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3a8705a4c617a0f76847c5c3f262043fe75b8942 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5652) Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5668) 
   * f2e824b2daf427f49409e1ea5c7e9704a84c76d9 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5676) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] xintongsong commented on a change in pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
xintongsong commented on a change in pull request #13186:
URL: https://github.com/apache/flink/pull/13186#discussion_r472105037



##########
File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/TestingFlinkKubeClient.java
##########
@@ -0,0 +1,170 @@
+/*
+ * 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;
+
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesPod;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesService;
+import org.apache.flink.kubernetes.kubeclient.resources.KubernetesWatch;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.BiFunction;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * Testing implementation of {@link FlinkKubeClient}.
+ */
+public class TestingFlinkKubeClient implements FlinkKubeClient {
+
+	private final Function<KubernetesPod, CompletableFuture<Void>> createTaskManagerPodFunction;
+	private final Function<String, CompletableFuture<Void>> stopPodFunction;
+	private final Consumer<String> stopAndCleanupClusterConsumer;
+	private final Function<Map<String, String>, List<KubernetesPod>> getPodsWithLabelsFunction;
+	private final BiFunction<Map<String, String>, PodCallbackHandler, KubernetesWatch> watchPodsAndDoCallbackFunction;
+
+	private TestingFlinkKubeClient(
+			Function<KubernetesPod, CompletableFuture<Void>> createTaskManagerPodFunction,
+			Function<String, CompletableFuture<Void>> stopPodFunction,
+			Consumer<String> stopAndCleanupClusterConsumer,
+			Function<Map<String, String>, List<KubernetesPod>> getPodsWithLabelsFunction,
+			BiFunction<Map<String, String>, PodCallbackHandler, KubernetesWatch> watchPodsAndDoCallbackFunction) {
+
+		this.createTaskManagerPodFunction = createTaskManagerPodFunction;
+		this.stopPodFunction = stopPodFunction;
+		this.stopAndCleanupClusterConsumer = stopAndCleanupClusterConsumer;
+		this.getPodsWithLabelsFunction = getPodsWithLabelsFunction;
+		this.watchPodsAndDoCallbackFunction = watchPodsAndDoCallbackFunction;
+	}
+
+	@Override
+	public void createJobManagerComponent(KubernetesJobManagerSpecification kubernetesJMSpec) {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public CompletableFuture<Void> createTaskManagerPod(KubernetesPod kubernetesPod) {
+		return createTaskManagerPodFunction.apply(kubernetesPod);
+	}
+
+	@Override
+	public CompletableFuture<Void> stopPod(String podName) {
+		return stopPodFunction.apply(podName);
+	}
+
+	@Override
+	public void stopAndCleanupCluster(String clusterId) {
+		stopAndCleanupClusterConsumer.accept(clusterId);
+	}
+
+	@Override
+	public Optional<KubernetesService> getRestService(String clusterId) {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public Optional<Endpoint> getRestEndpoint(String clusterId) {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public List<KubernetesPod> getPodsWithLabels(Map<String, String> labels) {
+		return getPodsWithLabelsFunction.apply(labels);
+	}
+
+	@Override
+	public void handleException(Exception e) {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public KubernetesWatch watchPodsAndDoCallback(Map<String, String> labels, PodCallbackHandler podCallbackHandler) {
+		return watchPodsAndDoCallbackFunction.apply(labels, podCallbackHandler);
+	}
+
+	@Override
+	public void close() throws Exception {
+		// noop
+	}
+
+	public static class Builder {
+		private Function<KubernetesPod, CompletableFuture<Void>> createTaskManagerPodFunction =
+				(ignore) -> FutureUtils.completedVoidFuture();
+		private Function<String, CompletableFuture<Void>> stopPodFunction =
+				(ignore) -> FutureUtils.completedVoidFuture();
+		private Consumer<String> stopAndCleanupClusterConsumer =
+				(ignore) -> {};
+		private Function<Map<String, String>, List<KubernetesPod>> getPodsWithLabelsFunction =
+				(ignore) -> Collections.emptyList();
+		private Consumer<Exception> handleExceptionConsumer =
+				(ignore) -> {};

Review comment:
       Nice catch. I meant to not support `handleException` because it's never used on the resource manager side, but forgot to remove the member.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #13186:
URL: https://github.com/apache/flink/pull/13186#issuecomment-675292490


   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit 3a8705a4c617a0f76847c5c3f262043fe75b8942 (Tue Aug 18 06:54:36 UTC 2020)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] xintongsong commented on a change in pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
xintongsong commented on a change in pull request #13186:
URL: https://github.com/apache/flink/pull/13186#discussion_r476988674



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManagerDriver.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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.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.KubernetesResourceManagerConfiguration;
+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 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;
+
+	public KubernetesResourceManagerDriver(
+			Configuration flinkConfig,
+			FlinkKubeClient kubeClient,
+			KubernetesResourceManagerConfiguration configuration) {
+		super(flinkConfig, GlobalConfiguration.loadConfiguration());

Review comment:
       > The problem is that it is not guaranteed at this point that you can read a proper Flink configuration.
   That sounds to be a problem. Could you explain what might prevent reading a proper Flink configuration?
   
   > Moreover, you are missing all dynamically configured properties which are processed in the entrypoints.
   I think that is exactly what we want. Please see my other comment about `flinkClientConfig`.

##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManagerDriver.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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.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.KubernetesResourceManagerConfiguration;
+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 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;
+
+	public KubernetesResourceManagerDriver(
+			Configuration flinkConfig,
+			FlinkKubeClient kubeClient,
+			KubernetesResourceManagerConfiguration configuration) {
+		super(flinkConfig, GlobalConfiguration.loadConfiguration());

Review comment:
       > The problem is that it is not guaranteed at this point that you can read a proper Flink configuration.
   
   That sounds to be a problem. Could you explain what might prevent reading a proper Flink configuration?
   
   > Moreover, you are missing all dynamically configured properties which are processed in the entrypoints.
   
   I think that is exactly what we want. Please see my other comment about `flinkClientConfig`.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13186:
URL: https://github.com/apache/flink/pull/13186#issuecomment-675299227


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5652",
       "triggerID" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5668",
       "triggerID" : "675358004",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f2e824b2daf427f49409e1ea5c7e9704a84c76d9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5676",
       "triggerID" : "f2e824b2daf427f49409e1ea5c7e9704a84c76d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd4e2812d845277e7f4bc68e97f4759e11334ca3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5680",
       "triggerID" : "bd4e2812d845277e7f4bc68e97f4759e11334ca3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "523cf170fe07e4b32e47653af46bf2290935d87a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5692",
       "triggerID" : "523cf170fe07e4b32e47653af46bf2290935d87a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 523cf170fe07e4b32e47653af46bf2290935d87a Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5692) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13186:
URL: https://github.com/apache/flink/pull/13186#issuecomment-675299227


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5652",
       "triggerID" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5668",
       "triggerID" : "675358004",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f2e824b2daf427f49409e1ea5c7e9704a84c76d9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5676",
       "triggerID" : "f2e824b2daf427f49409e1ea5c7e9704a84c76d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd4e2812d845277e7f4bc68e97f4759e11334ca3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5680",
       "triggerID" : "bd4e2812d845277e7f4bc68e97f4759e11334ca3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f2e824b2daf427f49409e1ea5c7e9704a84c76d9 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5676) 
   * bd4e2812d845277e7f4bc68e97f4759e11334ca3 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5680) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13186:
URL: https://github.com/apache/flink/pull/13186#discussion_r477080285



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/AbstractResourceManagerDriver.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.runtime.resourcemanager.active;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Abstract common base class for implementations of {@link ResourceManagerDriver}.
+ */
+public abstract class AbstractResourceManagerDriver<WorkerType extends ResourceIDRetrievable>
+	implements ResourceManagerDriver<WorkerType> {
+
+	protected final Logger log = LoggerFactory.getLogger(getClass());
+
+	protected final Configuration flinkConfig;
+	protected final Configuration flinkClientConfig;

Review comment:
       I think in the general case the assumption is wrong that `flinkClientConfig` is the original configuration because we don't load the configuration file from a potentially configured configuration directory. It might hold true wrt to the current start up scripts but it is brittle since it ties very specific assumptions together.
   
   Moreover, where is the difference between sending the full configuration vs. a diff which is applied to the presumably same base version? Shouldn't it result in the same effective configuration? Of course we should send the full `taskManagerConfig` and not the JM configuration.
   
   




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] xintongsong commented on a change in pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
xintongsong commented on a change in pull request #13186:
URL: https://github.com/apache/flink/pull/13186#discussion_r476986171



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/AbstractResourceManagerDriver.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.runtime.resourcemanager.active;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Abstract common base class for implementations of {@link ResourceManagerDriver}.
+ */
+public abstract class AbstractResourceManagerDriver<WorkerType extends ResourceIDRetrievable>
+	implements ResourceManagerDriver<WorkerType> {
+
+	protected final Logger log = LoggerFactory.getLogger(getClass());
+
+	protected final Configuration flinkConfig;
+	protected final Configuration flinkClientConfig;

Review comment:
       The background is that, RM will not send the complete configuration to the TMs, but only the differences compared to the original configuration submitted by the client.
   
   Currently, when submitting the job, client will ship the local `flink-conf.yaml` file. This file is available to both JM and TMs, through Yarn Shared Cache and Kubernetes ConfigMap.
   
   TM should not use that original `flink-conf.yaml` directly, because the configuration might be overwritten on the JM side (e.g., dynamic properties, JM address, TM resource, etc.). Therefore, RM will compare the effective configuration (`flinkConfig`) with the original one shipped from the client side (`flinkClientConfig`), and send the differences to TMs as dynamic properties, which overwrites the original configuration on the TM side.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #13186:
URL: https://github.com/apache/flink/pull/13186#issuecomment-675299227


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3a8705a4c617a0f76847c5c3f262043fe75b8942 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13186:
URL: https://github.com/apache/flink/pull/13186#issuecomment-675299227


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5652",
       "triggerID" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5668",
       "triggerID" : "675358004",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f2e824b2daf427f49409e1ea5c7e9704a84c76d9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5676",
       "triggerID" : "f2e824b2daf427f49409e1ea5c7e9704a84c76d9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd4e2812d845277e7f4bc68e97f4759e11334ca3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5680",
       "triggerID" : "bd4e2812d845277e7f4bc68e97f4759e11334ca3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "523cf170fe07e4b32e47653af46bf2290935d87a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5692",
       "triggerID" : "523cf170fe07e4b32e47653af46bf2290935d87a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "63c2f658cc309661c434451a5d8c0b38d2e748d3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5848",
       "triggerID" : "679880992",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "63c2f658cc309661c434451a5d8c0b38d2e748d3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5848",
       "triggerID" : "63c2f658cc309661c434451a5d8c0b38d2e748d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "679880992",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "952d92954bb418aa30820b811bc5a41f76e9ab21",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "952d92954bb418aa30820b811bc5a41f76e9ab21",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 63c2f658cc309661c434451a5d8c0b38d2e748d3 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5848) 
   * 952d92954bb418aa30820b811bc5a41f76e9ab21 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] xintongsong commented on a change in pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
xintongsong commented on a change in pull request #13186:
URL: https://github.com/apache/flink/pull/13186#discussion_r476259405



##########
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:
       True, there is indeed a behavior change.
   Unfortunately, I don't find good way to not introducing such behavior changes with the refactoring, since the re-request logic has been abstracted to the `ActiveResourceManager`.
   As far as I can see, this behavior change should not introduce any regression.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] xintongsong commented on a change in pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
xintongsong commented on a change in pull request #13186:
URL: https://github.com/apache/flink/pull/13186#discussion_r476256982



##########
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:
       Well, I was not intended to prevent these variables from being accessed by the test cases.
   But I think you are right, it does not hurt to make the `private`. They are not accessed from anywhere else, and one could change that once such accesses are needed.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] xintongsong commented on a change in pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
xintongsong commented on a change in pull request #13186:
URL: https://github.com/apache/flink/pull/13186#discussion_r476975877



##########
File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManagerDriver.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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.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.KubernetesResourceManagerConfiguration;
+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 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;
+
+	public KubernetesResourceManagerDriver(
+			Configuration flinkConfig,
+			FlinkKubeClient kubeClient,
+			KubernetesResourceManagerConfiguration configuration) {
+		super(flinkConfig, GlobalConfiguration.loadConfiguration());
+
+		this.clusterId = Preconditions.checkNotNull(configuration.getClusterId());
+		this.kubeClient = Preconditions.checkNotNull(kubeClient);
+		this.requestResourceFutures = new HashMap<>();
+	}
+
+	// ------------------------------------------------------------------------
+	//  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());
+
+		// TODO: enable pod creation interval
+		// 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.

Review comment:
       This is addressed right away in another commit in this PR.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13186: [FLINK-18720][k8s] Migrate KubernetesResourceManager to the new KubernetesResourceManagerDriver

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13186:
URL: https://github.com/apache/flink/pull/13186#issuecomment-675299227


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5652",
       "triggerID" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3a8705a4c617a0f76847c5c3f262043fe75b8942",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5668",
       "triggerID" : "675358004",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 3a8705a4c617a0f76847c5c3f262043fe75b8942 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5652) Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5668) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org