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/07/29 07:26:42 UTC

[GitHub] [flink] xintongsong opened a new pull request #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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


   ## What is the purpose of the change
   
   This PR introduces interfaces for the new design of active resource managers, and introduces the new `ActiveResourceManager`.
   
   ## Brief change log
   
   - d0c48eb2a65062685bb2a3242dd545a2ff601bd9..e0df80a3d746cf92721b37fbdbb3c63cfaf1122d: Code clean-ups.
   - 5218c8125d72b2b781ddb7926e8713d522c79aea: Introduce util class `ConsumerWithException`, which will be used by the new test cases.
   - c5f4c3f22347a3032f7f5b5e9f75fdfbf61322f5: Rename the original `ActiveResourceManager` to legacy.
   - 80697b59602b7ca75f68cbb5cca4df61edb1dac1: Introduce interfaces `ResourceManagerDriver` and `ResourceEventHandler`
   - ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef: Implement the new `ActiveResourceManager`.
   
   ## Verifying this change
   
   - Add `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] xintongsong commented on a change in pull request #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/taskmanager/TaskExecutorFlinkMemory.java
##########
@@ -128,4 +130,31 @@ public MemorySize getJvmDirectMemorySize() {
 	public MemorySize getTotalFlinkMemorySize() {
 		return frameworkHeap.add(frameworkOffHeap).add(taskHeap).add(taskOffHeap).add(network).add(managed);
 	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (obj == this) {
+			return true;
+		} else if (obj instanceof TaskExecutorFlinkMemory) {
+			TaskExecutorFlinkMemory that = (TaskExecutorFlinkMemory) obj;
+			return Objects.equals(this.frameworkHeap, that.frameworkHeap) &&
+					Objects.equals(this. frameworkOffHeap, that.frameworkOffHeap) &&
+					Objects.equals(this. taskHeap, that.taskHeap)&&
+					Objects.equals(this. taskOffHeap, that.taskOffHeap)&&
+					Objects.equals(this. network, that.network)&&
+					Objects.equals(this. managed, that.managed);

Review comment:
       Nice catch. That should be a typo + copy/paste problem.




----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4957",
       "triggerID" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5404",
       "triggerID" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5458363056979d6be8594af4ac469c57ee8f9b15",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5409",
       "triggerID" : "5458363056979d6be8594af4ac469c57ee8f9b15",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 906162e1f7a7988fd1f70bae81a76d6172497c63 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5404) 
   * 5458363056979d6be8594af4ac469c57ee8f9b15 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5409) 
   
   <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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/ResourceManagerDriver.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.ApplicationStatus;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec;
+import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable;
+import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException;
+
+import javax.annotation.Nullable;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * A {@link ResourceManagerDriver} is responsible for requesting and releasing resources from/to a particular external
+ * resource manager.
+ */
+public interface ResourceManagerDriver<WorkerType extends ResourceIDRetrievable> {
+
+	/**
+	 * Initialize the deployment specific components.
+	 *
+	 * @param resourceEventHandler Handler that handles resource events.
+	 */
+	void initialize(ResourceEventHandler<WorkerType> resourceEventHandler) throws Throwable;
+
+	/**
+	 * Terminate the deployment specific components.
+	 *
+	 * @return A future that will be completed successfully when the driver is terminated, or exceptionally if cannot be
+	 * terminated.
+	 */
+	CompletableFuture<Void> terminate();
+
+	/**
+	 * The deployment specific code to deregister the application. This should report the application's final status and
+	 * shut down the resource manager driver cleanly.
+	 *
+	 * <p>This method also needs to make sure all pending containers that are not registered yet are returned.
+	 *
+	 * @param finalStatus The application status to report.
+	 * @param optionalDiagnostics A diagnostics message or {@code null}.
+	 * @throws ResourceManagerException if the application could not be shut down.
+	 */
+	void deregisterApplication(ApplicationStatus finalStatus, @Nullable String optionalDiagnostics) throws Throwable;

Review comment:
       `terminate` corresponds to the deployment specific part of original `RpcEndpoint#onStop`, while `deregisterApplication` corresponds to `ResourceManager#internalDeregisterApplication`. It seems to me that the former releases resources before shutting down the RM, while the later simply communicate with the external system and unregister the application. (Please see `KubernetesResourceManagerDriver` in the POC branch [here](https://github.com/xintongsong/flink/blob/26b293c768d4e31cd6e7f6ac0fbf79591a515b83/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManagerDriver.java#L108) for what these two methods do.)
   
   I'm not entirely sure why we had these two interfaces at the first place, and whether they can be unified. Since this PR is a pure refactoring effort, I tried to preserve the original codes to avoid potential behavior changes.




----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4957",
       "triggerID" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5404",
       "triggerID" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5458363056979d6be8594af4ac469c57ee8f9b15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5409",
       "triggerID" : "5458363056979d6be8594af4ac469c57ee8f9b15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5428",
       "triggerID" : "672548350",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5428",
       "triggerID" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "672548350",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "4847c551d75f244418947031dfe8b53bcf34073e",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5449",
       "triggerID" : "4847c551d75f244418947031dfe8b53bcf34073e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "afc7db9e33b3996dd6b859b1ed1e9c418a7a0ef0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "afc7db9e33b3996dd6b859b1ed1e9c418a7a0ef0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4847c551d75f244418947031dfe8b53bcf34073e Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5449) 
   * afc7db9e33b3996dd6b859b1ed1e9c418a7a0ef0 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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4957",
       "triggerID" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5404",
       "triggerID" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5458363056979d6be8594af4ac469c57ee8f9b15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5409",
       "triggerID" : "5458363056979d6be8594af4ac469c57ee8f9b15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5428",
       "triggerID" : "672548350",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5428",
       "triggerID" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "672548350",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * f1bfc953a17f24a74ac3da5db2d9dd790f90de8c Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5428) 
   
   <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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4957",
       "triggerID" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5404",
       "triggerID" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5458363056979d6be8594af4ac469c57ee8f9b15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5409",
       "triggerID" : "5458363056979d6be8594af4ac469c57ee8f9b15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5428",
       "triggerID" : "672548350",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5428",
       "triggerID" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "672548350",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "4847c551d75f244418947031dfe8b53bcf34073e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5449",
       "triggerID" : "4847c551d75f244418947031dfe8b53bcf34073e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "afc7db9e33b3996dd6b859b1ed1e9c418a7a0ef0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5502",
       "triggerID" : "afc7db9e33b3996dd6b859b1ed1e9c418a7a0ef0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "afc7db9e33b3996dd6b859b1ed1e9c418a7a0ef0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5505",
       "triggerID" : "673474288",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * afc7db9e33b3996dd6b859b1ed1e9c418a7a0ef0 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5502) Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5505) 
   
   <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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4957",
       "triggerID" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5404",
       "triggerID" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5458363056979d6be8594af4ac469c57ee8f9b15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5409",
       "triggerID" : "5458363056979d6be8594af4ac469c57ee8f9b15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5428",
       "triggerID" : "672548350",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5428",
       "triggerID" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "672548350",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "4847c551d75f244418947031dfe8b53bcf34073e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5449",
       "triggerID" : "4847c551d75f244418947031dfe8b53bcf34073e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "afc7db9e33b3996dd6b859b1ed1e9c418a7a0ef0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5502",
       "triggerID" : "afc7db9e33b3996dd6b859b1ed1e9c418a7a0ef0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "afc7db9e33b3996dd6b859b1ed1e9c418a7a0ef0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5505",
       "triggerID" : "673474288",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "afc7db9e33b3996dd6b859b1ed1e9c418a7a0ef0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5502",
       "triggerID" : "673474288",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * afc7db9e33b3996dd6b859b1ed1e9c418a7a0ef0 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5505) Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5502) 
   
   <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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4957",
       "triggerID" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5404",
       "triggerID" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5458363056979d6be8594af4ac469c57ee8f9b15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5409",
       "triggerID" : "5458363056979d6be8594af4ac469c57ee8f9b15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5428",
       "triggerID" : "672548350",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5428",
       "triggerID" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "672548350",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "4847c551d75f244418947031dfe8b53bcf34073e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4847c551d75f244418947031dfe8b53bcf34073e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f1bfc953a17f24a74ac3da5db2d9dd790f90de8c Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5428) 
   * 4847c551d75f244418947031dfe8b53bcf34073e 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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/ActiveResourceManager.java
##########
@@ -0,0 +1,287 @@
+/*
+ * 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.annotation.VisibleForTesting;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.akka.AkkaUtils;
+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.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.ResourceManagerPartitionTrackerFactory;
+import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.JobLeaderIdService;
+import org.apache.flink.runtime.resourcemanager.ResourceManager;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException;
+import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * An active implementation of {@link ResourceManager}.
+ *
+ * <p>This resource manager actively requests and releases resources from/to the external resource management frameworks.
+ * With different {@link ResourceManagerDriver} provided, this resource manager can work with various frameworks.
+ */
+public class ActiveResourceManager<WorkerType extends ResourceIDRetrievable>
+		extends ResourceManager<WorkerType> implements ResourceEventHandler<WorkerType> {
+
+	protected final Configuration flinkConfig;
+
+	private final ResourceManagerDriver<WorkerType> resourceManagerDriver;
+
+	/** All workers maintained by {@link ActiveResourceManager}. */
+	private final Map<ResourceID, WorkerType> workerNodeMap;
+
+	/** Number of requested and not registered workers per worker resource spec. */
+	private final PendingWorkerCounter pendingWorkerCounter;
+
+	/** Identifiers and worker resource spec of requested not registered workers. */
+	private final Map<ResourceID, WorkerResourceSpec> currentAttemptUnregisteredWorkers;
+
+	public ActiveResourceManager(
+			ResourceManagerDriver<WorkerType> resourceManagerDriver,
+			Configuration flinkConfig,
+			RpcService rpcService,
+			ResourceID resourceId,
+			HighAvailabilityServices highAvailabilityServices,
+			HeartbeatServices heartbeatServices,
+			SlotManager slotManager,
+			ResourceManagerPartitionTrackerFactory clusterPartitionTrackerFactory,
+			JobLeaderIdService jobLeaderIdService,
+			ClusterInformation clusterInformation,
+			FatalErrorHandler fatalErrorHandler,
+			ResourceManagerMetricGroup resourceManagerMetricGroup) {
+		super(
+				rpcService,
+				resourceId,
+				highAvailabilityServices,
+				heartbeatServices,
+				slotManager,
+				clusterPartitionTrackerFactory,
+				jobLeaderIdService,
+				clusterInformation,
+				fatalErrorHandler,
+				resourceManagerMetricGroup,
+				AkkaUtils.getTimeoutAsTime(Preconditions.checkNotNull(flinkConfig)));
+
+		this.flinkConfig = flinkConfig;
+		this.resourceManagerDriver = Preconditions.checkNotNull(resourceManagerDriver);
+		this.workerNodeMap = new HashMap<>();
+		this.pendingWorkerCounter = new PendingWorkerCounter();
+		this.currentAttemptUnregisteredWorkers = new HashMap<>();
+	}
+
+	// ------------------------------------------------------------------------
+	//  ResourceManager
+	// ------------------------------------------------------------------------
+
+	@Override
+	protected void initialize() throws ResourceManagerException {
+		try {
+			resourceManagerDriver.initialize(this);
+		} catch (Throwable t) {
+			throw new ResourceManagerException("Cannot initialize resource provider.", t);
+		}
+	}
+
+	@Override
+	protected void terminate() throws ResourceManagerException {
+		try {
+			resourceManagerDriver.terminate().get();
+		} catch (Throwable t) {
+			throw new ResourceManagerException("Cannot terminate resource provider.", t);
+		}
+	}
+
+	@Override
+	protected void internalDeregisterApplication(ApplicationStatus finalStatus, @Nullable String optionalDiagnostics)
+			throws ResourceManagerException {
+		try {
+			resourceManagerDriver.deregisterApplication(finalStatus, optionalDiagnostics);
+		} catch (Throwable t) {
+			throw new ResourceManagerException("Cannot deregister application.", t);
+		}
+	}
+
+	@Override
+	public boolean startNewWorker(WorkerResourceSpec workerResourceSpec) {
+		requestNewWorker(workerResourceSpec);
+		return true;
+	}
+
+	@Override
+	protected WorkerType workerStarted(ResourceID resourceID) {
+		return workerNodeMap.get(resourceID);
+	}
+
+	@Override
+	public boolean stopWorker(WorkerType worker) {
+		final ResourceID resourceId = worker.getResourceID();
+		resourceManagerDriver.releaseResource(worker);
+
+		log.info("Stopping worker {}.", resourceId);
+
+		clearStateForWorker(resourceId);
+
+		return true;
+	}
+
+	@Override
+	protected void onWorkerRegistered(WorkerType worker) {
+		final ResourceID resourceId = worker.getResourceID();
+		log.info("Worker {} is registered.", resourceId);
+
+		final WorkerResourceSpec workerResourceSpec = currentAttemptUnregisteredWorkers.remove(resourceId);
+		if (workerResourceSpec != null) {
+			final int count = pendingWorkerCounter.decreaseAndGet(workerResourceSpec);
+			log.info("Worker {} with resource spec {} was requested in current attempt." +
+							" Current pending count after registering: {}.",
+					resourceId,
+					workerResourceSpec,
+					count);
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  ResourceEventListener
+	// ------------------------------------------------------------------------
+
+	@Override
+	public void onPreviousAttemptWorkersRecovered(Collection<WorkerType> recoveredWorkers) {
+		log.info("Recovered {} workers from previous attempt.", recoveredWorkers.size());
+		for (WorkerType worker : recoveredWorkers) {
+			final ResourceID resourceId = worker.getResourceID();
+			workerNodeMap.put(resourceId, worker);
+			log.info("Worker {} recovered from previous attempt.", resourceId);
+		}
+	}
+
+	@Override
+	public void onWorkerTerminated(ResourceID resourceId) {
+		log.info("Worker {} is terminated.", resourceId);
+		if (clearStateForWorker(resourceId)) {
+			requestWorkerIfRequired();
+		}
+	}
+
+	@Override
+	public void onError(Throwable exception) {
+		onFatalError(exception);
+	}
+
+	@Override
+	public void handleInMainThread(Runnable runnable) {
+		runAsync(runnable);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Internal
+	// ------------------------------------------------------------------------
+
+	private void requestNewWorker(WorkerResourceSpec workerResourceSpec) {
+		final TaskExecutorProcessSpec taskExecutorProcessSpec =
+				TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, workerResourceSpec);
+		final int pendingCount = pendingWorkerCounter.increaseAndGet(workerResourceSpec);
+
+		log.info("Requesting new worker with resource spec {}, current pending count: {}.",
+				workerResourceSpec,
+				pendingCount);
+
+		resourceManagerDriver.requestResource(taskExecutorProcessSpec)
+				.whenComplete((worker, exception) -> {
+					if (exception != null) {
+						final int count = pendingWorkerCounter.decreaseAndGet(workerResourceSpec);
+						log.warn("Failed requesting worker with resource spec {}, current pending count: {}, exception: {}",
+								workerResourceSpec,
+								count,
+								exception);
+						requestWorkerIfRequired();
+					} else {
+						final ResourceID resourceId = worker.getResourceID();
+						workerNodeMap.put(resourceId, worker);
+						currentAttemptUnregisteredWorkers.put(resourceId, workerResourceSpec);
+						log.info("Requested worker {} with resource spec {}.",
+								resourceId,
+								workerResourceSpec);
+					}
+				});

Review comment:
       I'm not sure whether I have understand your comment correctly.
   It seems `FutureUtils.assertNoException` will treat the exception as a fatal error and exit the process, while here what we want is to try re-request a worker if there's an exception.




----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4957",
       "triggerID" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5404",
       "triggerID" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5458363056979d6be8594af4ac469c57ee8f9b15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5409",
       "triggerID" : "5458363056979d6be8594af4ac469c57ee8f9b15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "672548350",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "672548350",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   *  Unknown: [CANCELED](TBD) 
   * f1bfc953a17f24a74ac3da5db2d9dd790f90de8c 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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4957",
       "triggerID" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5404",
       "triggerID" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5458363056979d6be8594af4ac469c57ee8f9b15",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5409",
       "triggerID" : "5458363056979d6be8594af4ac469c57ee8f9b15",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4957) 
   * 906162e1f7a7988fd1f70bae81a76d6172497c63 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5404) 
   * 5458363056979d6be8594af4ac469c57ee8f9b15 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5409) 
   
   <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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/ActiveResourceManager.java
##########
@@ -0,0 +1,287 @@
+/*
+ * 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.annotation.VisibleForTesting;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.akka.AkkaUtils;
+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.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+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.ResourceManagerPartitionTrackerFactory;
+import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.JobLeaderIdService;
+import org.apache.flink.runtime.resourcemanager.ResourceManager;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException;
+import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * An active implementation of {@link ResourceManager}.
+ *
+ * <p>This resource manager actively requests and releases resources from/to the external resource management frameworks.
+ * With different {@link ResourceManagerDriver} provided, this resource manager can work with various frameworks.
+ */
+public class ActiveResourceManager<WorkerType extends ResourceIDRetrievable>
+		extends ResourceManager<WorkerType> implements ResourceEventHandler<WorkerType> {
+
+	protected final Configuration flinkConfig;
+
+	private final ResourceManagerDriver<WorkerType> resourceManagerDriver;
+
+	/** All workers maintained by {@link ActiveResourceManager}. */
+	private final Map<ResourceID, WorkerType> workerNodeMap;
+
+	/** Number of requested and not registered workers per worker resource spec. */
+	private final PendingWorkerCounter pendingWorkerCounter;
+
+	/** Identifiers and worker resource spec of requested not registered workers. */
+	private final Map<ResourceID, WorkerResourceSpec> currentAttemptUnregisteredWorkers;
+
+	public ActiveResourceManager(
+			ResourceManagerDriver<WorkerType> resourceManagerDriver,
+			Configuration flinkConfig,
+			RpcService rpcService,
+			ResourceID resourceId,
+			HighAvailabilityServices highAvailabilityServices,
+			HeartbeatServices heartbeatServices,
+			SlotManager slotManager,
+			ResourceManagerPartitionTrackerFactory clusterPartitionTrackerFactory,
+			JobLeaderIdService jobLeaderIdService,
+			ClusterInformation clusterInformation,
+			FatalErrorHandler fatalErrorHandler,
+			ResourceManagerMetricGroup resourceManagerMetricGroup) {
+		super(
+				rpcService,
+				resourceId,
+				highAvailabilityServices,
+				heartbeatServices,
+				slotManager,
+				clusterPartitionTrackerFactory,
+				jobLeaderIdService,
+				clusterInformation,
+				fatalErrorHandler,
+				resourceManagerMetricGroup,
+				AkkaUtils.getTimeoutAsTime(Preconditions.checkNotNull(flinkConfig)));
+
+		this.flinkConfig = flinkConfig;
+		this.resourceManagerDriver = resourceManagerDriver;
+		this.workerNodeMap = new HashMap<>();
+		this.pendingWorkerCounter = new PendingWorkerCounter();
+		this.currentAttemptUnregisteredWorkers = new HashMap<>();
+	}
+
+	// ------------------------------------------------------------------------
+	//  ResourceManager
+	// ------------------------------------------------------------------------
+
+	@Override
+	protected void initialize() throws ResourceManagerException {
+		try {
+			resourceManagerDriver.initialize(
+					this,
+					command -> getMainThreadExecutor().execute(command)); // always execute on the current main thread executor.

Review comment:
       Yes, double checked.




----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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






----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
##########
@@ -1064,6 +1076,13 @@ public void handleError(final Exception exception) {
 	 */
 	protected abstract void initialize() throws ResourceManagerException;
 
+	/**
+	 * Terminates the framework specific components.
+	 *
+	 * @throws Throwable with occurs during termination.
+	 */
+	protected abstract void terminate() throws Throwable;

Review comment:
       I kept it `Throwable` because this was what `KubernetesResourceManager#onStop` and `YarnResourceManager#onStop` catch.
   Weaken to throwing `Exception` also makes sense to me.




----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4957",
       "triggerID" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5404",
       "triggerID" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5458363056979d6be8594af4ac469c57ee8f9b15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5409",
       "triggerID" : "5458363056979d6be8594af4ac469c57ee8f9b15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5428",
       "triggerID" : "672548350",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5428",
       "triggerID" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "672548350",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "4847c551d75f244418947031dfe8b53bcf34073e",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5449",
       "triggerID" : "4847c551d75f244418947031dfe8b53bcf34073e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4847c551d75f244418947031dfe8b53bcf34073e Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5449) 
   
   <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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
##########
@@ -1064,6 +1076,13 @@ public void handleError(final Exception exception) {
 	 */
 	protected abstract void initialize() throws ResourceManagerException;
 
+	/**
+	 * Terminates the framework specific components.
+	 *
+	 * @throws Exception with occurs during termination.

Review comment:
       ```suggestion
   	 * @throws Exception which occurs during termination.
   ```




----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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


   


----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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


   @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] tillrohrmann commented on a change in pull request #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
##########
@@ -1064,6 +1076,13 @@ public void handleError(final Exception exception) {
 	 */
 	protected abstract void initialize() throws ResourceManagerException;
 
+	/**
+	 * Terminates the framework specific components.
+	 *
+	 * @throws Throwable with occurs during termination.
+	 */
+	protected abstract void terminate() throws Throwable;

Review comment:
       Can we weaken the signature to `throws Exception`?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/ActiveResourceManager.java
##########
@@ -0,0 +1,287 @@
+/*
+ * 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.annotation.VisibleForTesting;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.akka.AkkaUtils;
+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.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+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.ResourceManagerPartitionTrackerFactory;
+import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.JobLeaderIdService;
+import org.apache.flink.runtime.resourcemanager.ResourceManager;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException;
+import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * An active implementation of {@link ResourceManager}.
+ *
+ * <p>This resource manager actively requests and releases resources from/to the external resource management frameworks.
+ * With different {@link ResourceManagerDriver} provided, this resource manager can work with various frameworks.
+ */
+public class ActiveResourceManager<WorkerType extends ResourceIDRetrievable>
+		extends ResourceManager<WorkerType> implements ResourceEventHandler<WorkerType> {
+
+	protected final Configuration flinkConfig;
+
+	private final ResourceManagerDriver<WorkerType> resourceManagerDriver;
+
+	/** All workers maintained by {@link ActiveResourceManager}. */
+	private final Map<ResourceID, WorkerType> workerNodeMap;
+
+	/** Number of requested and not registered workers per worker resource spec. */
+	private final PendingWorkerCounter pendingWorkerCounter;
+
+	/** Identifiers and worker resource spec of requested not registered workers. */
+	private final Map<ResourceID, WorkerResourceSpec> currentAttemptUnregisteredWorkers;
+
+	public ActiveResourceManager(
+			ResourceManagerDriver<WorkerType> resourceManagerDriver,
+			Configuration flinkConfig,
+			RpcService rpcService,
+			ResourceID resourceId,
+			HighAvailabilityServices highAvailabilityServices,
+			HeartbeatServices heartbeatServices,
+			SlotManager slotManager,
+			ResourceManagerPartitionTrackerFactory clusterPartitionTrackerFactory,
+			JobLeaderIdService jobLeaderIdService,
+			ClusterInformation clusterInformation,
+			FatalErrorHandler fatalErrorHandler,
+			ResourceManagerMetricGroup resourceManagerMetricGroup) {
+		super(
+				rpcService,
+				resourceId,
+				highAvailabilityServices,
+				heartbeatServices,
+				slotManager,
+				clusterPartitionTrackerFactory,
+				jobLeaderIdService,
+				clusterInformation,
+				fatalErrorHandler,
+				resourceManagerMetricGroup,
+				AkkaUtils.getTimeoutAsTime(Preconditions.checkNotNull(flinkConfig)));
+
+		this.flinkConfig = flinkConfig;
+		this.resourceManagerDriver = resourceManagerDriver;
+		this.workerNodeMap = new HashMap<>();
+		this.pendingWorkerCounter = new PendingWorkerCounter();
+		this.currentAttemptUnregisteredWorkers = new HashMap<>();
+	}
+
+	// ------------------------------------------------------------------------
+	//  ResourceManager
+	// ------------------------------------------------------------------------
+
+	@Override
+	protected void initialize() throws ResourceManagerException {
+		try {
+			resourceManagerDriver.initialize(
+					this,
+					command -> getMainThreadExecutor().execute(command)); // always execute on the current main thread executor.
+		} catch (Throwable t) {
+			throw new ResourceManagerException("Cannot initialize resource provider.", t);
+		}
+	}
+
+	@Override
+	protected void terminate() throws ResourceManagerException {
+		try {
+			resourceManagerDriver.terminate().get();
+		} catch (Throwable t) {
+			throw new ResourceManagerException("Cannot terminate resource provider.", t);
+		}
+	}
+
+	@Override
+	protected void internalDeregisterApplication(ApplicationStatus finalStatus, @Nullable String optionalDiagnostics)
+			throws ResourceManagerException {
+		try {
+			resourceManagerDriver.deregisterApplication(finalStatus, optionalDiagnostics);
+		} catch (Throwable t) {

Review comment:
       Same here.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/ActiveResourceManager.java
##########
@@ -0,0 +1,287 @@
+/*
+ * 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.annotation.VisibleForTesting;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.akka.AkkaUtils;
+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.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+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.ResourceManagerPartitionTrackerFactory;
+import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.JobLeaderIdService;
+import org.apache.flink.runtime.resourcemanager.ResourceManager;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException;
+import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * An active implementation of {@link ResourceManager}.
+ *
+ * <p>This resource manager actively requests and releases resources from/to the external resource management frameworks.
+ * With different {@link ResourceManagerDriver} provided, this resource manager can work with various frameworks.
+ */
+public class ActiveResourceManager<WorkerType extends ResourceIDRetrievable>
+		extends ResourceManager<WorkerType> implements ResourceEventHandler<WorkerType> {
+
+	protected final Configuration flinkConfig;
+
+	private final ResourceManagerDriver<WorkerType> resourceManagerDriver;
+
+	/** All workers maintained by {@link ActiveResourceManager}. */
+	private final Map<ResourceID, WorkerType> workerNodeMap;
+
+	/** Number of requested and not registered workers per worker resource spec. */
+	private final PendingWorkerCounter pendingWorkerCounter;
+
+	/** Identifiers and worker resource spec of requested not registered workers. */
+	private final Map<ResourceID, WorkerResourceSpec> currentAttemptUnregisteredWorkers;
+
+	public ActiveResourceManager(
+			ResourceManagerDriver<WorkerType> resourceManagerDriver,
+			Configuration flinkConfig,
+			RpcService rpcService,
+			ResourceID resourceId,
+			HighAvailabilityServices highAvailabilityServices,
+			HeartbeatServices heartbeatServices,
+			SlotManager slotManager,
+			ResourceManagerPartitionTrackerFactory clusterPartitionTrackerFactory,
+			JobLeaderIdService jobLeaderIdService,
+			ClusterInformation clusterInformation,
+			FatalErrorHandler fatalErrorHandler,
+			ResourceManagerMetricGroup resourceManagerMetricGroup) {
+		super(
+				rpcService,
+				resourceId,
+				highAvailabilityServices,
+				heartbeatServices,
+				slotManager,
+				clusterPartitionTrackerFactory,
+				jobLeaderIdService,
+				clusterInformation,
+				fatalErrorHandler,
+				resourceManagerMetricGroup,
+				AkkaUtils.getTimeoutAsTime(Preconditions.checkNotNull(flinkConfig)));
+
+		this.flinkConfig = flinkConfig;
+		this.resourceManagerDriver = resourceManagerDriver;
+		this.workerNodeMap = new HashMap<>();
+		this.pendingWorkerCounter = new PendingWorkerCounter();
+		this.currentAttemptUnregisteredWorkers = new HashMap<>();
+	}
+
+	// ------------------------------------------------------------------------
+	//  ResourceManager
+	// ------------------------------------------------------------------------
+
+	@Override
+	protected void initialize() throws ResourceManagerException {
+		try {
+			resourceManagerDriver.initialize(
+					this,
+					command -> getMainThreadExecutor().execute(command)); // always execute on the current main thread executor.
+		} catch (Throwable t) {
+			throw new ResourceManagerException("Cannot initialize resource provider.", t);
+		}
+	}
+
+	@Override
+	protected void terminate() throws ResourceManagerException {
+		try {
+			resourceManagerDriver.terminate().get();
+		} catch (Throwable t) {

Review comment:
       Same here with `Throwable`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/ActiveResourceManager.java
##########
@@ -0,0 +1,287 @@
+/*
+ * 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.annotation.VisibleForTesting;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.akka.AkkaUtils;
+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.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+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.ResourceManagerPartitionTrackerFactory;
+import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.JobLeaderIdService;
+import org.apache.flink.runtime.resourcemanager.ResourceManager;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException;
+import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * An active implementation of {@link ResourceManager}.
+ *
+ * <p>This resource manager actively requests and releases resources from/to the external resource management frameworks.
+ * With different {@link ResourceManagerDriver} provided, this resource manager can work with various frameworks.
+ */
+public class ActiveResourceManager<WorkerType extends ResourceIDRetrievable>
+		extends ResourceManager<WorkerType> implements ResourceEventHandler<WorkerType> {
+
+	protected final Configuration flinkConfig;
+
+	private final ResourceManagerDriver<WorkerType> resourceManagerDriver;
+
+	/** All workers maintained by {@link ActiveResourceManager}. */
+	private final Map<ResourceID, WorkerType> workerNodeMap;
+
+	/** Number of requested and not registered workers per worker resource spec. */
+	private final PendingWorkerCounter pendingWorkerCounter;
+
+	/** Identifiers and worker resource spec of requested not registered workers. */
+	private final Map<ResourceID, WorkerResourceSpec> currentAttemptUnregisteredWorkers;
+
+	public ActiveResourceManager(
+			ResourceManagerDriver<WorkerType> resourceManagerDriver,
+			Configuration flinkConfig,
+			RpcService rpcService,
+			ResourceID resourceId,
+			HighAvailabilityServices highAvailabilityServices,
+			HeartbeatServices heartbeatServices,
+			SlotManager slotManager,
+			ResourceManagerPartitionTrackerFactory clusterPartitionTrackerFactory,
+			JobLeaderIdService jobLeaderIdService,
+			ClusterInformation clusterInformation,
+			FatalErrorHandler fatalErrorHandler,
+			ResourceManagerMetricGroup resourceManagerMetricGroup) {
+		super(
+				rpcService,
+				resourceId,
+				highAvailabilityServices,
+				heartbeatServices,
+				slotManager,
+				clusterPartitionTrackerFactory,
+				jobLeaderIdService,
+				clusterInformation,
+				fatalErrorHandler,
+				resourceManagerMetricGroup,
+				AkkaUtils.getTimeoutAsTime(Preconditions.checkNotNull(flinkConfig)));
+
+		this.flinkConfig = flinkConfig;
+		this.resourceManagerDriver = resourceManagerDriver;
+		this.workerNodeMap = new HashMap<>();
+		this.pendingWorkerCounter = new PendingWorkerCounter();
+		this.currentAttemptUnregisteredWorkers = new HashMap<>();
+	}
+
+	// ------------------------------------------------------------------------
+	//  ResourceManager
+	// ------------------------------------------------------------------------
+
+	@Override
+	protected void initialize() throws ResourceManagerException {
+		try {
+			resourceManagerDriver.initialize(
+					this,
+					command -> getMainThreadExecutor().execute(command)); // always execute on the current main thread executor.

Review comment:
       Just as a note: If the `ResourceManagerDriver` tries to run something on this executor while the `ResourceManager` has no leadership, then it will simply be ignored and not executed. Is this ok with the current behaviour of the `ResourceManagerDriver`?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/ActiveResourceManager.java
##########
@@ -0,0 +1,287 @@
+/*
+ * 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.annotation.VisibleForTesting;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.akka.AkkaUtils;
+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.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+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.ResourceManagerPartitionTrackerFactory;
+import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.JobLeaderIdService;
+import org.apache.flink.runtime.resourcemanager.ResourceManager;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException;
+import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * An active implementation of {@link ResourceManager}.
+ *
+ * <p>This resource manager actively requests and releases resources from/to the external resource management frameworks.
+ * With different {@link ResourceManagerDriver} provided, this resource manager can work with various frameworks.
+ */
+public class ActiveResourceManager<WorkerType extends ResourceIDRetrievable>
+		extends ResourceManager<WorkerType> implements ResourceEventHandler<WorkerType> {
+
+	protected final Configuration flinkConfig;
+
+	private final ResourceManagerDriver<WorkerType> resourceManagerDriver;
+
+	/** All workers maintained by {@link ActiveResourceManager}. */
+	private final Map<ResourceID, WorkerType> workerNodeMap;
+
+	/** Number of requested and not registered workers per worker resource spec. */
+	private final PendingWorkerCounter pendingWorkerCounter;
+
+	/** Identifiers and worker resource spec of requested not registered workers. */
+	private final Map<ResourceID, WorkerResourceSpec> currentAttemptUnregisteredWorkers;
+
+	public ActiveResourceManager(
+			ResourceManagerDriver<WorkerType> resourceManagerDriver,
+			Configuration flinkConfig,
+			RpcService rpcService,
+			ResourceID resourceId,
+			HighAvailabilityServices highAvailabilityServices,
+			HeartbeatServices heartbeatServices,
+			SlotManager slotManager,
+			ResourceManagerPartitionTrackerFactory clusterPartitionTrackerFactory,
+			JobLeaderIdService jobLeaderIdService,
+			ClusterInformation clusterInformation,
+			FatalErrorHandler fatalErrorHandler,
+			ResourceManagerMetricGroup resourceManagerMetricGroup) {
+		super(
+				rpcService,
+				resourceId,
+				highAvailabilityServices,
+				heartbeatServices,
+				slotManager,
+				clusterPartitionTrackerFactory,
+				jobLeaderIdService,
+				clusterInformation,
+				fatalErrorHandler,
+				resourceManagerMetricGroup,
+				AkkaUtils.getTimeoutAsTime(Preconditions.checkNotNull(flinkConfig)));
+
+		this.flinkConfig = flinkConfig;
+		this.resourceManagerDriver = resourceManagerDriver;
+		this.workerNodeMap = new HashMap<>();
+		this.pendingWorkerCounter = new PendingWorkerCounter();
+		this.currentAttemptUnregisteredWorkers = new HashMap<>();
+	}
+
+	// ------------------------------------------------------------------------
+	//  ResourceManager
+	// ------------------------------------------------------------------------
+
+	@Override
+	protected void initialize() throws ResourceManagerException {
+		try {
+			resourceManagerDriver.initialize(
+					this,
+					command -> getMainThreadExecutor().execute(command)); // always execute on the current main thread executor.
+		} catch (Throwable t) {

Review comment:
       I'm wondering whether we should really catch all `Throwables` here. I think that JVM `Errors` should not be caught at this point.




----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/ResourceEventHandler.java
##########
@@ -0,0 +1,60 @@
+/*
+ * 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 java.util.Collection;
+
+/**
+ * Callback interfaces for handling resource events from external resource managers.
+ */
+public interface ResourceEventHandler<WorkerType extends ResourceIDRetrievable> {
+
+	/**
+	 * Notifies that workers of previous attempt have been recovered from the external resource manager.
+	 *
+	 * @param recoveredWorkers Collection of worker nodes, in the deployment specific type.
+	 */
+	void onPreviousAttemptWorkersRecovered(Collection<WorkerType> recoveredWorkers);
+
+	/**
+	 * Notifies that the worker has been terminated.
+	 *
+	 * <p>See also {@link ResourceManagerDriver#requestResource}.
+	 *
+	 * @param resourceId Identifier of the terminated worker.
+	 */
+	void onWorkerTerminated(ResourceID resourceId);
+
+	/**
+	 * Notifies that an error has occurred that the process cannot proceed.
+	 *
+	 * @param exception Exception that describes the error.
+	 */
+	void onError(Throwable exception);
+
+	/**
+	 * Execute given runnable in the rpc main thread.
+	 *
+	 * @param runnable Runnable to be executed.
+	 */
+	void handleInMainThread(Runnable runnable);

Review comment:
       I recalled the other reason I did not pass in the main thread executor. It has been two weeks.
   
   I did tried passing in `ComponentMainThreadExecutor` but it doesn't work. Because `FencedRpcEndpoint#fencedMainThreadExecutor` could be changed via `setFencingToken`. So the main thread executor at the time initializing will no longer be the main thread executor after RM accepting the leadership.




----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/ActiveResourceManager.java
##########
@@ -0,0 +1,287 @@
+/*
+ * 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.annotation.VisibleForTesting;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.akka.AkkaUtils;
+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.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.ResourceManagerPartitionTrackerFactory;
+import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.JobLeaderIdService;
+import org.apache.flink.runtime.resourcemanager.ResourceManager;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException;
+import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * An active implementation of {@link ResourceManager}.
+ *
+ * <p>This resource manager actively requests and releases resources from/to the external resource management frameworks.
+ * With different {@link ResourceManagerDriver} provided, this resource manager can work with various frameworks.
+ */
+public class ActiveResourceManager<WorkerType extends ResourceIDRetrievable>
+		extends ResourceManager<WorkerType> implements ResourceEventHandler<WorkerType> {
+
+	protected final Configuration flinkConfig;
+
+	private final ResourceManagerDriver<WorkerType> resourceManagerDriver;
+
+	/** All workers maintained by {@link ActiveResourceManager}. */
+	private final Map<ResourceID, WorkerType> workerNodeMap;
+
+	/** Number of requested and not registered workers per worker resource spec. */
+	private final PendingWorkerCounter pendingWorkerCounter;
+
+	/** Identifiers and worker resource spec of requested not registered workers. */
+	private final Map<ResourceID, WorkerResourceSpec> currentAttemptUnregisteredWorkers;
+
+	public ActiveResourceManager(
+			ResourceManagerDriver<WorkerType> resourceManagerDriver,
+			Configuration flinkConfig,
+			RpcService rpcService,
+			ResourceID resourceId,
+			HighAvailabilityServices highAvailabilityServices,
+			HeartbeatServices heartbeatServices,
+			SlotManager slotManager,
+			ResourceManagerPartitionTrackerFactory clusterPartitionTrackerFactory,
+			JobLeaderIdService jobLeaderIdService,
+			ClusterInformation clusterInformation,
+			FatalErrorHandler fatalErrorHandler,
+			ResourceManagerMetricGroup resourceManagerMetricGroup) {
+		super(
+				rpcService,
+				resourceId,
+				highAvailabilityServices,
+				heartbeatServices,
+				slotManager,
+				clusterPartitionTrackerFactory,
+				jobLeaderIdService,
+				clusterInformation,
+				fatalErrorHandler,
+				resourceManagerMetricGroup,
+				AkkaUtils.getTimeoutAsTime(Preconditions.checkNotNull(flinkConfig)));
+
+		this.flinkConfig = flinkConfig;
+		this.resourceManagerDriver = Preconditions.checkNotNull(resourceManagerDriver);
+		this.workerNodeMap = new HashMap<>();
+		this.pendingWorkerCounter = new PendingWorkerCounter();
+		this.currentAttemptUnregisteredWorkers = new HashMap<>();
+	}
+
+	// ------------------------------------------------------------------------
+	//  ResourceManager
+	// ------------------------------------------------------------------------
+
+	@Override
+	protected void initialize() throws ResourceManagerException {
+		try {
+			resourceManagerDriver.initialize(this);
+		} catch (Throwable t) {
+			throw new ResourceManagerException("Cannot initialize resource provider.", t);
+		}
+	}
+
+	@Override
+	protected void terminate() throws ResourceManagerException {
+		try {
+			resourceManagerDriver.terminate().get();
+		} catch (Throwable t) {
+			throw new ResourceManagerException("Cannot terminate resource provider.", t);
+		}
+	}
+
+	@Override
+	protected void internalDeregisterApplication(ApplicationStatus finalStatus, @Nullable String optionalDiagnostics)
+			throws ResourceManagerException {
+		try {
+			resourceManagerDriver.deregisterApplication(finalStatus, optionalDiagnostics);
+		} catch (Throwable t) {
+			throw new ResourceManagerException("Cannot deregister application.", t);
+		}
+	}
+
+	@Override
+	public boolean startNewWorker(WorkerResourceSpec workerResourceSpec) {
+		requestNewWorker(workerResourceSpec);
+		return true;
+	}
+
+	@Override
+	protected WorkerType workerStarted(ResourceID resourceID) {
+		return workerNodeMap.get(resourceID);
+	}
+
+	@Override
+	public boolean stopWorker(WorkerType worker) {
+		final ResourceID resourceId = worker.getResourceID();
+		resourceManagerDriver.releaseResource(worker);
+
+		log.info("Stopping worker {}.", resourceId);
+
+		clearStateForWorker(resourceId);
+
+		return true;
+	}
+
+	@Override
+	protected void onWorkerRegistered(WorkerType worker) {
+		final ResourceID resourceId = worker.getResourceID();
+		log.info("Worker {} is registered.", resourceId);
+
+		final WorkerResourceSpec workerResourceSpec = currentAttemptUnregisteredWorkers.remove(resourceId);
+		if (workerResourceSpec != null) {
+			final int count = pendingWorkerCounter.decreaseAndGet(workerResourceSpec);
+			log.info("Worker {} with resource spec {} was requested in current attempt." +
+							" Current pending count after registering: {}.",
+					resourceId,
+					workerResourceSpec,
+					count);
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  ResourceEventListener
+	// ------------------------------------------------------------------------
+
+	@Override
+	public void onPreviousAttemptWorkersRecovered(Collection<WorkerType> recoveredWorkers) {
+		log.info("Recovered {} workers from previous attempt.", recoveredWorkers.size());
+		for (WorkerType worker : recoveredWorkers) {
+			final ResourceID resourceId = worker.getResourceID();
+			workerNodeMap.put(resourceId, worker);
+			log.info("Worker {} recovered from previous attempt.", resourceId);
+		}
+	}
+
+	@Override
+	public void onWorkerTerminated(ResourceID resourceId) {
+		log.info("Worker {} is terminated.", resourceId);
+		if (clearStateForWorker(resourceId)) {
+			requestWorkerIfRequired();
+		}
+	}
+
+	@Override
+	public void onError(Throwable exception) {
+		onFatalError(exception);
+	}
+
+	@Override
+	public void handleInMainThread(Runnable runnable) {
+		runAsync(runnable);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Internal
+	// ------------------------------------------------------------------------
+
+	private void requestNewWorker(WorkerResourceSpec workerResourceSpec) {
+		final TaskExecutorProcessSpec taskExecutorProcessSpec =
+				TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, workerResourceSpec);
+		final int pendingCount = pendingWorkerCounter.increaseAndGet(workerResourceSpec);
+
+		log.info("Requesting new worker with resource spec {}, current pending count: {}.",
+				workerResourceSpec,
+				pendingCount);
+
+		resourceManagerDriver.requestResource(taskExecutorProcessSpec)
+				.whenComplete((worker, exception) -> {
+					if (exception != null) {
+						final int count = pendingWorkerCounter.decreaseAndGet(workerResourceSpec);
+						log.warn("Failed requesting worker with resource spec {}, current pending count: {}, exception: {}",
+								workerResourceSpec,
+								count,
+								exception);
+						requestWorkerIfRequired();
+					} else {
+						final ResourceID resourceId = worker.getResourceID();
+						workerNodeMap.put(resourceId, worker);
+						currentAttemptUnregisteredWorkers.put(resourceId, workerResourceSpec);
+						log.info("Requested worker {} with resource spec {}.",
+								resourceId,
+								workerResourceSpec);
+					}
+				});

Review comment:
       `FutureUtils.assertNoException` will guard against any unforeseen exceptions which might bubble up from the `requestWorkerIfRequired` method, for example. The problem with the current version is that these exception will get silently swallowed by the resulting future of `whenComplete`.
   
   In other words, I don't want to treat the exception coming out of `requestResource` as fatally but all other exceptions coming out of the `whenComplete` handler because they won't be handled properly.




----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4957",
       "triggerID" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5404",
       "triggerID" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5458363056979d6be8594af4ac469c57ee8f9b15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5409",
       "triggerID" : "5458363056979d6be8594af4ac469c57ee8f9b15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5428",
       "triggerID" : "672548350",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5428",
       "triggerID" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "672548350",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "4847c551d75f244418947031dfe8b53bcf34073e",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5449",
       "triggerID" : "4847c551d75f244418947031dfe8b53bcf34073e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "afc7db9e33b3996dd6b859b1ed1e9c418a7a0ef0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5502",
       "triggerID" : "afc7db9e33b3996dd6b859b1ed1e9c418a7a0ef0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4847c551d75f244418947031dfe8b53bcf34073e Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5449) 
   * afc7db9e33b3996dd6b859b1ed1e9c418a7a0ef0 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5502) 
   
   <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 commented on pull request #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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






----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/ResourceManagerDriver.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.ApplicationStatus;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec;
+import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable;
+import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException;
+
+import javax.annotation.Nullable;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * A {@link ResourceManagerDriver} is responsible for requesting and releasing resources from/to a particular external
+ * resource manager.
+ */
+public interface ResourceManagerDriver<WorkerType extends ResourceIDRetrievable> {
+
+	/**
+	 * Initialize the deployment specific components.
+	 *
+	 * @param resourceEventHandler Handler that handles resource events.
+	 */
+	void initialize(ResourceEventHandler<WorkerType> resourceEventHandler) throws Throwable;
+
+	/**
+	 * Terminate the deployment specific components.
+	 *
+	 * @return A future that will be completed successfully when the driver is terminated, or exceptionally if cannot be
+	 * terminated.
+	 */
+	CompletableFuture<Void> terminate();
+
+	/**
+	 * The deployment specific code to deregister the application. This should report the application's final status and
+	 * shut down the resource manager driver cleanly.
+	 *
+	 * <p>This method also needs to make sure all pending containers that are not registered yet are returned.
+	 *
+	 * @param finalStatus The application status to report.
+	 * @param optionalDiagnostics A diagnostics message or {@code null}.
+	 * @throws ResourceManagerException if the application could not be shut down.
+	 */
+	void deregisterApplication(ApplicationStatus finalStatus, @Nullable String optionalDiagnostics) throws Throwable;

Review comment:
       Maybe a `CompletableFutuer<Void> terminate(ApplicationStatus finalStatus, String diagnostics)` could be enough instead of having two methods. Or maybe `deregisterApplication` only deregisters the application but does not shut down the component.




----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/ResourceEventHandler.java
##########
@@ -0,0 +1,60 @@
+/*
+ * 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 java.util.Collection;
+
+/**
+ * Callback interfaces for handling resource events from external resource managers.
+ */
+public interface ResourceEventHandler<WorkerType extends ResourceIDRetrievable> {
+
+	/**
+	 * Notifies that workers of previous attempt have been recovered from the external resource manager.
+	 *
+	 * @param recoveredWorkers Collection of worker nodes, in the deployment specific type.
+	 */
+	void onPreviousAttemptWorkersRecovered(Collection<WorkerType> recoveredWorkers);
+
+	/**
+	 * Notifies that the worker has been terminated.
+	 *
+	 * <p>See also {@link ResourceManagerDriver#requestResource}.
+	 *
+	 * @param resourceId Identifier of the terminated worker.
+	 */
+	void onWorkerTerminated(ResourceID resourceId);
+
+	/**
+	 * Notifies that an error has occurred that the process cannot proceed.
+	 *
+	 * @param exception Exception that describes the error.
+	 */
+	void onError(Throwable exception);
+
+	/**
+	 * Execute given runnable in the rpc main thread.
+	 *
+	 * @param runnable Runnable to be executed.
+	 */
+	void handleInMainThread(Runnable runnable);

Review comment:
       We might even be able to pass in an `Executor` if we don't need any other methods than `execute`.




----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4957",
       "triggerID" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5404",
       "triggerID" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4957) 
   * 906162e1f7a7988fd1f70bae81a76d6172497c63 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5404) 
   
   <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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/jobmanager/JobManagerFlinkMemory.java
##########
@@ -50,7 +53,8 @@
 	private final MemorySize jvmHeap;
 	private final MemorySize offHeapMemory;
 
-	JobManagerFlinkMemory(MemorySize jvmHeap, MemorySize offHeapMemory) {
+	@VisibleForTesting

Review comment:
       The reason why Xintong added this annotation is that he changed the visibility of this class for some tests I believe.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/ResourceManagerDriver.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.ApplicationStatus;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec;
+import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable;
+import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException;
+
+import javax.annotation.Nullable;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * A {@link ResourceManagerDriver} is responsible for requesting and releasing resources from/to a particular external
+ * resource manager.
+ */
+public interface ResourceManagerDriver<WorkerType extends ResourceIDRetrievable> {
+
+	/**
+	 * Initialize the deployment specific components.
+	 *
+	 * @param resourceEventHandler Handler that handles resource events.
+	 */
+	void initialize(ResourceEventHandler<WorkerType> resourceEventHandler) throws Throwable;
+
+	/**
+	 * Terminate the deployment specific components.
+	 *
+	 * @return A future that will be completed successfully when the driver is terminated, or exceptionally if cannot be
+	 * terminated.
+	 */
+	CompletableFuture<Void> terminate();
+
+	/**
+	 * The deployment specific code to deregister the application. This should report the application's final status and
+	 * shut down the resource manager driver cleanly.
+	 *
+	 * <p>This method also needs to make sure all pending containers that are not registered yet are returned.
+	 *
+	 * @param finalStatus The application status to report.
+	 * @param optionalDiagnostics A diagnostics message or {@code null}.
+	 * @throws ResourceManagerException if the application could not be shut down.
+	 */
+	void deregisterApplication(ApplicationStatus finalStatus, @Nullable String optionalDiagnostics) throws Throwable;
+
+	/**
+	 * Request resource from the external resource manager.
+	 *
+	 * <p>This method request a new resource from the external resource manager, and tries to launch a task manager
+	 * inside the allocated resource, with respect to the provided taskExecutorProcessSpec The returned future will be

Review comment:
       ```suggestion
   	 * inside the allocated resource, with respect to the provided taskExecutorProcessSpec. The returned future will be
   ```

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/active/ActiveResourceManagerTest.java
##########
@@ -0,0 +1,428 @@
+/*
+ * 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.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.entrypoint.ClusterInformation;
+import org.apache.flink.runtime.instance.HardwareDescription;
+import org.apache.flink.runtime.io.network.partition.NoOpResourceManagerPartitionTracker;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.registration.RegistrationResponse;
+import org.apache.flink.runtime.resourcemanager.TaskExecutorRegistration;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager;
+import org.apache.flink.runtime.resourcemanager.slotmanager.TestingSlotManagerBuilder;
+import org.apache.flink.runtime.resourcemanager.utils.MockResourceManagerRuntimeServices;
+import org.apache.flink.runtime.rpc.TestingRpcService;
+import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.runtime.util.TestingFatalErrorHandler;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.hamcrest.Matchers.instanceOf;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.lessThan;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for {@link ActiveResourceManager}.
+ */
+public class ActiveResourceManagerTest extends TestLogger {
+
+	private static final long TIMEOUT_SEC = 5L;
+	private static final Time TIMEOUT_TIME = Time.seconds(TIMEOUT_SEC);
+
+	private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = WorkerResourceSpec.ZERO;
+
+	/**
+	 * Tests worker successfully requested, started and registered.
+	 */
+	@Test
+	public void testStartNewWorker() throws Exception {
+		new Context() {{
+			final ResourceID tmResourceId = ResourceID.generate();
+			final CompletableFuture<TaskExecutorProcessSpec> requestWorkerFromDriverFuture = new CompletableFuture<>();
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				requestWorkerFromDriverFuture.complete(taskExecutorProcessSpec);
+				return CompletableFuture.completedFuture(tmResourceId);
+			});
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec = requestWorkerFromDriverFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// worker registered, verify registration succeed
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(tmResourceId);
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+			});
+		}};
+	}
+
+	/**
+	 * Tests worker failed while requesting.
+	 */
+	@Test
+	public void testStartNewWorkerFailedRequesting() throws Exception {
+		new Context() {{
+			final ResourceID tmResourceId = ResourceID.generate();
+			final AtomicInteger requestCount = new AtomicInteger(0);
+
+			final List<CompletableFuture<ResourceID>> resourceIdFutures = new ArrayList<>();
+			resourceIdFutures.add(new CompletableFuture<>());
+			resourceIdFutures.add(new CompletableFuture<>());
+
+			final List<CompletableFuture<TaskExecutorProcessSpec>> requestWorkerFromDriverFutures = new ArrayList<>();
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				int idx = requestCount.getAndIncrement();
+				assertThat(idx, lessThan(2));
+
+				requestWorkerFromDriverFutures.get(idx).complete(taskExecutorProcessSpec);
+				return resourceIdFutures.get(idx);
+			});
+
+			slotManagerBuilder.setGetRequiredResourcesSupplier(() -> Collections.singletonMap(WORKER_RESOURCE_SPEC, 1));
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec1 = requestWorkerFromDriverFutures.get(0).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec1,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// first request failed, verify requesting another worker from driver
+				runInMainThread(() -> resourceIdFutures.get(0).completeExceptionally(new Throwable("testing error")));
+				TaskExecutorProcessSpec taskExecutorProcessSpec2 =
+						requestWorkerFromDriverFutures.get(1).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(taskExecutorProcessSpec2, is(taskExecutorProcessSpec1));
+
+				// second request allocated, verify registration succeed
+				runInMainThread(() -> resourceIdFutures.get(1).complete(tmResourceId));
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(tmResourceId);
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+			});
+		}};
+	}
+
+	/**
+	 * Tests worker terminated after requested before registered.
+	 */
+	@Test
+	public void testWorkerTerminatedBeforeRegister() throws Exception {
+		new Context() {{
+			final AtomicInteger requestCount = new AtomicInteger(0);
+
+			final List<ResourceID> tmResourceIds = new ArrayList<>();
+			tmResourceIds.add(ResourceID.generate());
+			tmResourceIds.add(ResourceID.generate());
+
+			final List<CompletableFuture<TaskExecutorProcessSpec>> requestWorkerFromDriverFutures = new ArrayList<>();
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				int idx = requestCount.getAndIncrement();
+				assertThat(idx, lessThan(2));
+
+				requestWorkerFromDriverFutures.get(idx).complete(taskExecutorProcessSpec);
+				return CompletableFuture.completedFuture(tmResourceIds.get(idx));
+			});
+
+			slotManagerBuilder.setGetRequiredResourcesSupplier(() -> Collections.singletonMap(WORKER_RESOURCE_SPEC, 1));
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec1 = requestWorkerFromDriverFutures.get(0).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec1,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// first worker failed before register, verify requesting another worker from driver
+				runInMainThread(() -> getResourceManager().onWorkerTerminated(tmResourceIds.get(0)));
+				TaskExecutorProcessSpec taskExecutorProcessSpec2 =
+						requestWorkerFromDriverFutures.get(1).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(taskExecutorProcessSpec2, is(taskExecutorProcessSpec1));
+
+				// second worker registered, verify registration succeed
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(tmResourceIds.get(1));
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+			});
+		}};
+	}
+
+	/**
+	 * Tests worker terminated after registered.
+	 */
+	@Test
+	public void testWorkerTerminatedAfterRegister() throws Exception {
+		new Context() {{
+			final AtomicInteger requestCount = new AtomicInteger(0);
+
+			final List<ResourceID> tmResourceIds = new ArrayList<>();
+			tmResourceIds.add(ResourceID.generate());
+			tmResourceIds.add(ResourceID.generate());
+
+			final List<CompletableFuture<TaskExecutorProcessSpec>> requestWorkerFromDriverFutures = new ArrayList<>();
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				int idx = requestCount.getAndIncrement();
+				assertThat(idx, lessThan(2));
+
+				requestWorkerFromDriverFutures.get(idx).complete(taskExecutorProcessSpec);
+				return CompletableFuture.completedFuture(tmResourceIds.get(idx));
+			});
+
+			slotManagerBuilder.setGetRequiredResourcesSupplier(() -> Collections.singletonMap(WORKER_RESOURCE_SPEC, 1));
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec1 = requestWorkerFromDriverFutures.get(0).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec1,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// first worker registered, verify registration succeed
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture1 = registerTaskExecutor(tmResourceIds.get(0));
+				assertThat(registerTaskExecutorFuture1.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+
+				// first worker terminated, verify requesting another worker from driver
+				runInMainThread(() -> getResourceManager().onWorkerTerminated(tmResourceIds.get(0)));
+				TaskExecutorProcessSpec taskExecutorProcessSpec2 =
+						requestWorkerFromDriverFutures.get(1).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(taskExecutorProcessSpec2, is(taskExecutorProcessSpec1));
+
+				// second worker registered, verify registration succeed
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture2 = registerTaskExecutor(tmResourceIds.get(1));
+				assertThat(registerTaskExecutorFuture2.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+			});
+		}};
+	}
+
+	/**
+	 * Tests worker terminated and is no longer required.
+	 */
+	@Test
+	public void testWorkerTerminatedNoLongerRequired() throws Exception {
+		new Context() {{
+			final ResourceID tmResourceId = ResourceID.generate();
+			final AtomicInteger requestCount = new AtomicInteger(0);
+
+			final List<CompletableFuture<TaskExecutorProcessSpec>> requestWorkerFromDriverFutures = new ArrayList<>();
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				int idx = requestCount.getAndIncrement();
+				assertThat(idx, lessThan(2));
+
+				requestWorkerFromDriverFutures.get(idx).complete(taskExecutorProcessSpec);
+				return CompletableFuture.completedFuture(tmResourceId);
+			});
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec = requestWorkerFromDriverFutures.get(0).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// worker registered, verify registration succeed
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(tmResourceId);
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+
+				// worker terminated, verify not requesting new worker
+				runInMainThread(() -> {
+					getResourceManager().onWorkerTerminated(tmResourceId);
+					// needs to return something, so that we can use `get()` to make sure the main thread processing
+					// finishes before the assertions
+					return null;
+				}).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+				assertFalse(requestWorkerFromDriverFutures.get(1).isDone());
+			});
+		}};
+	}
+
+	/**
+	 * Tests workers from previous attempt successfully recovered and registered.
+	 */
+	@Test
+	public void testRecoverWorkerFromPreviousAttempt() throws Exception {
+		new Context() {{
+			final ResourceID tmResourceId = ResourceID.generate();
+
+			runTest(() -> {
+				runInMainThread(() -> getResourceManager().onPreviousAttemptWorkersRecovered(Collections.singleton(tmResourceId)));
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(tmResourceId);
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+			});
+		}};
+	}
+
+	/**
+	 * Tests decline unknown worker registration.
+	 */
+	@Test
+	public void testRegisterUnknownWorker() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(ResourceID.generate());
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Decline.class));
+			});
+		}};
+	}
+
+	@Test
+	public void testOnError() throws Exception {
+		new Context() {{
+			final Throwable fatalError = new Throwable("Testing fatal error");
+			runTest(() -> {
+				runInMainThread(() -> getResourceManager().onError(fatalError));
+				final Throwable reportedError = getFatalErrorHandler().getErrorFuture().get(TIMEOUT_SEC, TimeUnit.SECONDS);
+				assertThat(reportedError, is(fatalError));
+			});
+		}};
+	}
+
+	class Context {
+
+		final Configuration flinkConfig = new Configuration();
+		final TestingResourceManagerDriver.Builder driverBuilder = new TestingResourceManagerDriver.Builder();
+		final TestingSlotManagerBuilder slotManagerBuilder = new TestingSlotManagerBuilder();
+
+		private ActiveResourceManager<ResourceID> resourceManager;
+		private TestingFatalErrorHandler fatalErrorHandler;
+
+		ActiveResourceManager<ResourceID> getResourceManager() {
+			return resourceManager;
+		}
+
+		TestingFatalErrorHandler getFatalErrorHandler() {
+			return fatalErrorHandler;
+		}
+
+		void runTest(RunnableWithException testMethod) throws Exception {
+			fatalErrorHandler = new TestingFatalErrorHandler();
+			resourceManager = createAndStartResourceManager(
+					flinkConfig,
+					driverBuilder.build(),
+					slotManagerBuilder.createSlotManager());
+
+			try {
+				testMethod.run();
+			} finally {
+				resourceManager.close();
+			}
+		}
+
+		private ActiveResourceManager<ResourceID> createAndStartResourceManager(
+				Configuration configuration,
+				ResourceManagerDriver<ResourceID> driver,
+				SlotManager slotManager) throws Exception {
+			final TestingRpcService rpcService = new TestingRpcService(configuration);

Review comment:
       nit: I think it would be a bit cheaper to reuse the `TestingRpcService` via `@ClassRule static final TestingRpcServiceResource testingRpcServiceResource = ....`.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/active/ActiveResourceManagerTest.java
##########
@@ -0,0 +1,428 @@
+/*
+ * 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.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.entrypoint.ClusterInformation;
+import org.apache.flink.runtime.instance.HardwareDescription;
+import org.apache.flink.runtime.io.network.partition.NoOpResourceManagerPartitionTracker;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.registration.RegistrationResponse;
+import org.apache.flink.runtime.resourcemanager.TaskExecutorRegistration;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager;
+import org.apache.flink.runtime.resourcemanager.slotmanager.TestingSlotManagerBuilder;
+import org.apache.flink.runtime.resourcemanager.utils.MockResourceManagerRuntimeServices;
+import org.apache.flink.runtime.rpc.TestingRpcService;
+import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.runtime.util.TestingFatalErrorHandler;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.hamcrest.Matchers.instanceOf;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.lessThan;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for {@link ActiveResourceManager}.
+ */
+public class ActiveResourceManagerTest extends TestLogger {
+
+	private static final long TIMEOUT_SEC = 5L;
+	private static final Time TIMEOUT_TIME = Time.seconds(TIMEOUT_SEC);
+
+	private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = WorkerResourceSpec.ZERO;
+
+	/**
+	 * Tests worker successfully requested, started and registered.
+	 */
+	@Test
+	public void testStartNewWorker() throws Exception {
+		new Context() {{
+			final ResourceID tmResourceId = ResourceID.generate();
+			final CompletableFuture<TaskExecutorProcessSpec> requestWorkerFromDriverFuture = new CompletableFuture<>();
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				requestWorkerFromDriverFuture.complete(taskExecutorProcessSpec);
+				return CompletableFuture.completedFuture(tmResourceId);
+			});
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec = requestWorkerFromDriverFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// worker registered, verify registration succeed
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(tmResourceId);
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+			});
+		}};
+	}
+
+	/**
+	 * Tests worker failed while requesting.
+	 */
+	@Test
+	public void testStartNewWorkerFailedRequesting() throws Exception {
+		new Context() {{
+			final ResourceID tmResourceId = ResourceID.generate();
+			final AtomicInteger requestCount = new AtomicInteger(0);
+
+			final List<CompletableFuture<ResourceID>> resourceIdFutures = new ArrayList<>();
+			resourceIdFutures.add(new CompletableFuture<>());
+			resourceIdFutures.add(new CompletableFuture<>());
+
+			final List<CompletableFuture<TaskExecutorProcessSpec>> requestWorkerFromDriverFutures = new ArrayList<>();
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				int idx = requestCount.getAndIncrement();
+				assertThat(idx, lessThan(2));
+
+				requestWorkerFromDriverFutures.get(idx).complete(taskExecutorProcessSpec);
+				return resourceIdFutures.get(idx);
+			});
+
+			slotManagerBuilder.setGetRequiredResourcesSupplier(() -> Collections.singletonMap(WORKER_RESOURCE_SPEC, 1));
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec1 = requestWorkerFromDriverFutures.get(0).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec1,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// first request failed, verify requesting another worker from driver
+				runInMainThread(() -> resourceIdFutures.get(0).completeExceptionally(new Throwable("testing error")));
+				TaskExecutorProcessSpec taskExecutorProcessSpec2 =
+						requestWorkerFromDriverFutures.get(1).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(taskExecutorProcessSpec2, is(taskExecutorProcessSpec1));
+
+				// second request allocated, verify registration succeed
+				runInMainThread(() -> resourceIdFutures.get(1).complete(tmResourceId));
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(tmResourceId);
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+			});
+		}};
+	}
+
+	/**
+	 * Tests worker terminated after requested before registered.
+	 */
+	@Test
+	public void testWorkerTerminatedBeforeRegister() throws Exception {
+		new Context() {{
+			final AtomicInteger requestCount = new AtomicInteger(0);
+
+			final List<ResourceID> tmResourceIds = new ArrayList<>();
+			tmResourceIds.add(ResourceID.generate());
+			tmResourceIds.add(ResourceID.generate());
+
+			final List<CompletableFuture<TaskExecutorProcessSpec>> requestWorkerFromDriverFutures = new ArrayList<>();
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				int idx = requestCount.getAndIncrement();
+				assertThat(idx, lessThan(2));
+
+				requestWorkerFromDriverFutures.get(idx).complete(taskExecutorProcessSpec);
+				return CompletableFuture.completedFuture(tmResourceIds.get(idx));
+			});
+
+			slotManagerBuilder.setGetRequiredResourcesSupplier(() -> Collections.singletonMap(WORKER_RESOURCE_SPEC, 1));
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec1 = requestWorkerFromDriverFutures.get(0).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec1,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// first worker failed before register, verify requesting another worker from driver
+				runInMainThread(() -> getResourceManager().onWorkerTerminated(tmResourceIds.get(0)));
+				TaskExecutorProcessSpec taskExecutorProcessSpec2 =
+						requestWorkerFromDriverFutures.get(1).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(taskExecutorProcessSpec2, is(taskExecutorProcessSpec1));
+
+				// second worker registered, verify registration succeed
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(tmResourceIds.get(1));
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+			});
+		}};
+	}
+
+	/**
+	 * Tests worker terminated after registered.
+	 */
+	@Test
+	public void testWorkerTerminatedAfterRegister() throws Exception {
+		new Context() {{
+			final AtomicInteger requestCount = new AtomicInteger(0);
+
+			final List<ResourceID> tmResourceIds = new ArrayList<>();
+			tmResourceIds.add(ResourceID.generate());
+			tmResourceIds.add(ResourceID.generate());
+
+			final List<CompletableFuture<TaskExecutorProcessSpec>> requestWorkerFromDriverFutures = new ArrayList<>();
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				int idx = requestCount.getAndIncrement();
+				assertThat(idx, lessThan(2));
+
+				requestWorkerFromDriverFutures.get(idx).complete(taskExecutorProcessSpec);
+				return CompletableFuture.completedFuture(tmResourceIds.get(idx));
+			});
+
+			slotManagerBuilder.setGetRequiredResourcesSupplier(() -> Collections.singletonMap(WORKER_RESOURCE_SPEC, 1));
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec1 = requestWorkerFromDriverFutures.get(0).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec1,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// first worker registered, verify registration succeed
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture1 = registerTaskExecutor(tmResourceIds.get(0));
+				assertThat(registerTaskExecutorFuture1.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+
+				// first worker terminated, verify requesting another worker from driver
+				runInMainThread(() -> getResourceManager().onWorkerTerminated(tmResourceIds.get(0)));
+				TaskExecutorProcessSpec taskExecutorProcessSpec2 =
+						requestWorkerFromDriverFutures.get(1).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(taskExecutorProcessSpec2, is(taskExecutorProcessSpec1));
+
+				// second worker registered, verify registration succeed
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture2 = registerTaskExecutor(tmResourceIds.get(1));
+				assertThat(registerTaskExecutorFuture2.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+			});
+		}};
+	}
+
+	/**
+	 * Tests worker terminated and is no longer required.
+	 */
+	@Test
+	public void testWorkerTerminatedNoLongerRequired() throws Exception {
+		new Context() {{
+			final ResourceID tmResourceId = ResourceID.generate();
+			final AtomicInteger requestCount = new AtomicInteger(0);
+
+			final List<CompletableFuture<TaskExecutorProcessSpec>> requestWorkerFromDriverFutures = new ArrayList<>();
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				int idx = requestCount.getAndIncrement();
+				assertThat(idx, lessThan(2));
+
+				requestWorkerFromDriverFutures.get(idx).complete(taskExecutorProcessSpec);
+				return CompletableFuture.completedFuture(tmResourceId);
+			});
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec = requestWorkerFromDriverFutures.get(0).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// worker registered, verify registration succeed
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(tmResourceId);
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+
+				// worker terminated, verify not requesting new worker
+				runInMainThread(() -> {
+					getResourceManager().onWorkerTerminated(tmResourceId);
+					// needs to return something, so that we can use `get()` to make sure the main thread processing
+					// finishes before the assertions
+					return null;
+				}).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+				assertFalse(requestWorkerFromDriverFutures.get(1).isDone());
+			});
+		}};
+	}
+
+	/**
+	 * Tests workers from previous attempt successfully recovered and registered.
+	 */
+	@Test
+	public void testRecoverWorkerFromPreviousAttempt() throws Exception {
+		new Context() {{
+			final ResourceID tmResourceId = ResourceID.generate();
+
+			runTest(() -> {
+				runInMainThread(() -> getResourceManager().onPreviousAttemptWorkersRecovered(Collections.singleton(tmResourceId)));
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(tmResourceId);
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+			});
+		}};
+	}
+
+	/**
+	 * Tests decline unknown worker registration.
+	 */
+	@Test
+	public void testRegisterUnknownWorker() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(ResourceID.generate());
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Decline.class));
+			});
+		}};
+	}
+
+	@Test
+	public void testOnError() throws Exception {
+		new Context() {{
+			final Throwable fatalError = new Throwable("Testing fatal error");
+			runTest(() -> {
+				runInMainThread(() -> getResourceManager().onError(fatalError));
+				final Throwable reportedError = getFatalErrorHandler().getErrorFuture().get(TIMEOUT_SEC, TimeUnit.SECONDS);
+				assertThat(reportedError, is(fatalError));
+			});
+		}};
+	}
+
+	class Context {
+
+		final Configuration flinkConfig = new Configuration();
+		final TestingResourceManagerDriver.Builder driverBuilder = new TestingResourceManagerDriver.Builder();
+		final TestingSlotManagerBuilder slotManagerBuilder = new TestingSlotManagerBuilder();
+
+		private ActiveResourceManager<ResourceID> resourceManager;
+		private TestingFatalErrorHandler fatalErrorHandler;
+
+		ActiveResourceManager<ResourceID> getResourceManager() {
+			return resourceManager;
+		}
+
+		TestingFatalErrorHandler getFatalErrorHandler() {
+			return fatalErrorHandler;
+		}
+
+		void runTest(RunnableWithException testMethod) throws Exception {
+			fatalErrorHandler = new TestingFatalErrorHandler();
+			resourceManager = createAndStartResourceManager(
+					flinkConfig,
+					driverBuilder.build(),
+					slotManagerBuilder.createSlotManager());
+
+			try {
+				testMethod.run();
+			} finally {
+				resourceManager.close();
+			}
+		}
+
+		private ActiveResourceManager<ResourceID> createAndStartResourceManager(
+				Configuration configuration,
+				ResourceManagerDriver<ResourceID> driver,
+				SlotManager slotManager) throws Exception {
+			final TestingRpcService rpcService = new TestingRpcService(configuration);
+			final MockResourceManagerRuntimeServices rmServices = new MockResourceManagerRuntimeServices(rpcService, TIMEOUT_TIME, slotManager);
+
+			final ActiveResourceManager<ResourceID> activeResourceManager = new ActiveResourceManager<>(
+					driver,
+					configuration,
+					rpcService,
+					ResourceID.generate(),
+					rmServices.highAvailabilityServices,
+					rmServices.heartbeatServices,
+					rmServices.slotManager,
+					NoOpResourceManagerPartitionTracker::get,
+					rmServices.jobLeaderIdService,
+					new ClusterInformation("localhost", 1234),
+					fatalErrorHandler,
+					UnregisteredMetricGroups.createUnregisteredResourceManagerMetricGroup());
+
+			activeResourceManager.start();
+			rmServices.grantLeadership();
+
+			return activeResourceManager;
+		}
+
+		public void runInMainThread(Runnable runnable) {
+			resourceManager.handleInMainThread(runnable);
+		}
+
+		public <T> CompletableFuture<T> runInMainThread(Callable<T> callable) {
+			return resourceManager.runInMainThread(callable, TIMEOUT_TIME);
+		}
+
+		CompletableFuture<RegistrationResponse> registerTaskExecutor(ResourceID resourceID) throws Exception {
+			final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+					.createTestingTaskExecutorGateway();
+			((TestingRpcService) resourceManager.getRpcService()).registerGateway(resourceID.toString(), taskExecutorGateway);

Review comment:
       Having a class wide `TestingRpcService` would avoid this casting here.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/ResourceEventHandler.java
##########
@@ -0,0 +1,60 @@
+/*
+ * 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 java.util.Collection;
+
+/**
+ * Callback interfaces for handling resource events from external resource managers.
+ */
+public interface ResourceEventHandler<WorkerType extends ResourceIDRetrievable> {
+
+	/**
+	 * Notifies that workers of previous attempt have been recovered from the external resource manager.
+	 *
+	 * @param recoveredWorkers Collection of worker nodes, in the deployment specific type.
+	 */
+	void onPreviousAttemptWorkersRecovered(Collection<WorkerType> recoveredWorkers);
+
+	/**
+	 * Notifies that the worker has been terminated.
+	 *
+	 * <p>See also {@link ResourceManagerDriver#requestResource}.
+	 *
+	 * @param resourceId Identifier of the terminated worker.
+	 */
+	void onWorkerTerminated(ResourceID resourceId);
+
+	/**
+	 * Notifies that an error has occurred that the process cannot proceed.
+	 *
+	 * @param exception Exception that describes the error.
+	 */
+	void onError(Throwable exception);
+
+	/**
+	 * Execute given runnable in the rpc main thread.
+	 *
+	 * @param runnable Runnable to be executed.
+	 */
+	void handleInMainThread(Runnable runnable);

Review comment:
       Would it make sense to pass into the `ResourceManagerDriver.initialize()` a `MainThreadExecutor` instead of offering this method? The advantage would be that once could directly run future call backs in the main thread and does not have to call this method from the callback. Moreover, it would separate the concerns a bit better because the event handler does not need to know about the main thread if I'm not mistaken.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/ResourceManagerDriver.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.ApplicationStatus;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec;
+import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable;
+import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException;
+
+import javax.annotation.Nullable;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * A {@link ResourceManagerDriver} is responsible for requesting and releasing resources from/to a particular external
+ * resource manager.
+ */
+public interface ResourceManagerDriver<WorkerType extends ResourceIDRetrievable> {
+
+	/**
+	 * Initialize the deployment specific components.
+	 *
+	 * @param resourceEventHandler Handler that handles resource events.
+	 */
+	void initialize(ResourceEventHandler<WorkerType> resourceEventHandler) throws Throwable;
+
+	/**
+	 * Terminate the deployment specific components.
+	 *
+	 * @return A future that will be completed successfully when the driver is terminated, or exceptionally if cannot be
+	 * terminated.
+	 */
+	CompletableFuture<Void> terminate();
+
+	/**
+	 * The deployment specific code to deregister the application. This should report the application's final status and
+	 * shut down the resource manager driver cleanly.
+	 *
+	 * <p>This method also needs to make sure all pending containers that are not registered yet are returned.
+	 *
+	 * @param finalStatus The application status to report.
+	 * @param optionalDiagnostics A diagnostics message or {@code null}.
+	 * @throws ResourceManagerException if the application could not be shut down.
+	 */
+	void deregisterApplication(ApplicationStatus finalStatus, @Nullable String optionalDiagnostics) throws Throwable;
+
+	/**
+	 * Request resource from the external resource manager.
+	 *
+	 * <p>This method request a new resource from the external resource manager, and tries to launch a task manager
+	 * inside the allocated resource, with respect to the provided taskExecutorProcessSpec The returned future will be
+	 * completed with a worker node in the deployment specific type, or exceptionally if the allocation has failed.
+	 *
+	 * <p>Note: Success completion of the returned future does not necessarily mean the success of resource allocation
+	 * and task manager launching. Allocation and launching failures can still happen after the future completion. In
+	 * such cases, {@link ResourceEventHandler#onWorkerTerminated} will be called.
+	 *
+	 * <p>The future is guaranteed to be completed in the rpc main thread, before trying to launch the task manager,
+	 * thus before the task manager registration. It is also guaranteed that
+	 * {@link ResourceEventHandler#onWorkerTerminated} will not be called on the requested worker, until the returned
+	 * future is completed successfully.

Review comment:
       Nice description of the contract.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/ResourceManagerDriver.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.ApplicationStatus;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec;
+import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable;
+import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException;
+
+import javax.annotation.Nullable;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * A {@link ResourceManagerDriver} is responsible for requesting and releasing resources from/to a particular external
+ * resource manager.
+ */
+public interface ResourceManagerDriver<WorkerType extends ResourceIDRetrievable> {
+
+	/**
+	 * Initialize the deployment specific components.
+	 *
+	 * @param resourceEventHandler Handler that handles resource events.
+	 */
+	void initialize(ResourceEventHandler<WorkerType> resourceEventHandler) throws Throwable;
+
+	/**
+	 * Terminate the deployment specific components.
+	 *
+	 * @return A future that will be completed successfully when the driver is terminated, or exceptionally if cannot be
+	 * terminated.
+	 */
+	CompletableFuture<Void> terminate();
+
+	/**
+	 * The deployment specific code to deregister the application. This should report the application's final status and
+	 * shut down the resource manager driver cleanly.
+	 *
+	 * <p>This method also needs to make sure all pending containers that are not registered yet are returned.
+	 *
+	 * @param finalStatus The application status to report.
+	 * @param optionalDiagnostics A diagnostics message or {@code null}.
+	 * @throws ResourceManagerException if the application could not be shut down.
+	 */
+	void deregisterApplication(ApplicationStatus finalStatus, @Nullable String optionalDiagnostics) throws Throwable;
+
+	/**
+	 * Request resource from the external resource manager.
+	 *
+	 * <p>This method request a new resource from the external resource manager, and tries to launch a task manager
+	 * inside the allocated resource, with respect to the provided taskExecutorProcessSpec The returned future will be
+	 * completed with a worker node in the deployment specific type, or exceptionally if the allocation has failed.
+	 *
+	 * <p>Note: Success completion of the returned future does not necessarily mean the success of resource allocation

Review comment:
       ```suggestion
   	 * <p>Note: Completion of the returned future does not necessarily mean the success of resource allocation
   ```

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/ResourceManagerDriver.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.ApplicationStatus;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec;
+import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable;
+import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException;
+
+import javax.annotation.Nullable;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * A {@link ResourceManagerDriver} is responsible for requesting and releasing resources from/to a particular external
+ * resource manager.
+ */
+public interface ResourceManagerDriver<WorkerType extends ResourceIDRetrievable> {
+
+	/**
+	 * Initialize the deployment specific components.
+	 *
+	 * @param resourceEventHandler Handler that handles resource events.
+	 */
+	void initialize(ResourceEventHandler<WorkerType> resourceEventHandler) throws Throwable;
+
+	/**
+	 * Terminate the deployment specific components.
+	 *
+	 * @return A future that will be completed successfully when the driver is terminated, or exceptionally if cannot be
+	 * terminated.
+	 */
+	CompletableFuture<Void> terminate();
+
+	/**
+	 * The deployment specific code to deregister the application. This should report the application's final status and
+	 * shut down the resource manager driver cleanly.
+	 *
+	 * <p>This method also needs to make sure all pending containers that are not registered yet are returned.
+	 *
+	 * @param finalStatus The application status to report.
+	 * @param optionalDiagnostics A diagnostics message or {@code null}.
+	 * @throws ResourceManagerException if the application could not be shut down.
+	 */
+	void deregisterApplication(ApplicationStatus finalStatus, @Nullable String optionalDiagnostics) throws Throwable;

Review comment:
       What is the relation between these two methods? From the JavaDocs, it reads as if `deregisterApplication` would also shut down the driver similar to `terminate`. Can they be unified?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/ActiveResourceManager.java
##########
@@ -0,0 +1,287 @@
+/*
+ * 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.annotation.VisibleForTesting;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.akka.AkkaUtils;
+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.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.ResourceManagerPartitionTrackerFactory;
+import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.JobLeaderIdService;
+import org.apache.flink.runtime.resourcemanager.ResourceManager;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException;
+import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * An active implementation of {@link ResourceManager}.
+ *
+ * <p>This resource manager actively requests and releases resources from/to the external resource management frameworks.
+ * With different {@link ResourceManagerDriver} provided, this resource manager can work with various frameworks.
+ */
+public class ActiveResourceManager<WorkerType extends ResourceIDRetrievable>
+		extends ResourceManager<WorkerType> implements ResourceEventHandler<WorkerType> {
+
+	protected final Configuration flinkConfig;
+
+	private final ResourceManagerDriver<WorkerType> resourceManagerDriver;
+
+	/** All workers maintained by {@link ActiveResourceManager}. */
+	private final Map<ResourceID, WorkerType> workerNodeMap;
+
+	/** Number of requested and not registered workers per worker resource spec. */
+	private final PendingWorkerCounter pendingWorkerCounter;
+
+	/** Identifiers and worker resource spec of requested not registered workers. */
+	private final Map<ResourceID, WorkerResourceSpec> currentAttemptUnregisteredWorkers;
+
+	public ActiveResourceManager(
+			ResourceManagerDriver<WorkerType> resourceManagerDriver,
+			Configuration flinkConfig,
+			RpcService rpcService,
+			ResourceID resourceId,
+			HighAvailabilityServices highAvailabilityServices,
+			HeartbeatServices heartbeatServices,
+			SlotManager slotManager,
+			ResourceManagerPartitionTrackerFactory clusterPartitionTrackerFactory,
+			JobLeaderIdService jobLeaderIdService,
+			ClusterInformation clusterInformation,
+			FatalErrorHandler fatalErrorHandler,
+			ResourceManagerMetricGroup resourceManagerMetricGroup) {
+		super(
+				rpcService,
+				resourceId,
+				highAvailabilityServices,
+				heartbeatServices,
+				slotManager,
+				clusterPartitionTrackerFactory,
+				jobLeaderIdService,
+				clusterInformation,
+				fatalErrorHandler,
+				resourceManagerMetricGroup,
+				AkkaUtils.getTimeoutAsTime(Preconditions.checkNotNull(flinkConfig)));
+
+		this.flinkConfig = flinkConfig;
+		this.resourceManagerDriver = Preconditions.checkNotNull(resourceManagerDriver);
+		this.workerNodeMap = new HashMap<>();
+		this.pendingWorkerCounter = new PendingWorkerCounter();
+		this.currentAttemptUnregisteredWorkers = new HashMap<>();
+	}
+
+	// ------------------------------------------------------------------------
+	//  ResourceManager
+	// ------------------------------------------------------------------------
+
+	@Override
+	protected void initialize() throws ResourceManagerException {
+		try {
+			resourceManagerDriver.initialize(this);
+		} catch (Throwable t) {
+			throw new ResourceManagerException("Cannot initialize resource provider.", t);
+		}
+	}
+
+	@Override
+	protected void terminate() throws ResourceManagerException {
+		try {
+			resourceManagerDriver.terminate().get();
+		} catch (Throwable t) {
+			throw new ResourceManagerException("Cannot terminate resource provider.", t);
+		}
+	}
+
+	@Override
+	protected void internalDeregisterApplication(ApplicationStatus finalStatus, @Nullable String optionalDiagnostics)
+			throws ResourceManagerException {
+		try {
+			resourceManagerDriver.deregisterApplication(finalStatus, optionalDiagnostics);
+		} catch (Throwable t) {
+			throw new ResourceManagerException("Cannot deregister application.", t);
+		}
+	}
+
+	@Override
+	public boolean startNewWorker(WorkerResourceSpec workerResourceSpec) {
+		requestNewWorker(workerResourceSpec);
+		return true;
+	}
+
+	@Override
+	protected WorkerType workerStarted(ResourceID resourceID) {
+		return workerNodeMap.get(resourceID);
+	}
+
+	@Override
+	public boolean stopWorker(WorkerType worker) {
+		final ResourceID resourceId = worker.getResourceID();
+		resourceManagerDriver.releaseResource(worker);
+
+		log.info("Stopping worker {}.", resourceId);
+
+		clearStateForWorker(resourceId);
+
+		return true;
+	}
+
+	@Override
+	protected void onWorkerRegistered(WorkerType worker) {
+		final ResourceID resourceId = worker.getResourceID();
+		log.info("Worker {} is registered.", resourceId);
+
+		final WorkerResourceSpec workerResourceSpec = currentAttemptUnregisteredWorkers.remove(resourceId);
+		if (workerResourceSpec != null) {
+			final int count = pendingWorkerCounter.decreaseAndGet(workerResourceSpec);
+			log.info("Worker {} with resource spec {} was requested in current attempt." +
+							" Current pending count after registering: {}.",
+					resourceId,
+					workerResourceSpec,
+					count);
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  ResourceEventListener
+	// ------------------------------------------------------------------------
+
+	@Override
+	public void onPreviousAttemptWorkersRecovered(Collection<WorkerType> recoveredWorkers) {
+		log.info("Recovered {} workers from previous attempt.", recoveredWorkers.size());
+		for (WorkerType worker : recoveredWorkers) {
+			final ResourceID resourceId = worker.getResourceID();
+			workerNodeMap.put(resourceId, worker);
+			log.info("Worker {} recovered from previous attempt.", resourceId);
+		}
+	}
+
+	@Override
+	public void onWorkerTerminated(ResourceID resourceId) {
+		log.info("Worker {} is terminated.", resourceId);
+		if (clearStateForWorker(resourceId)) {
+			requestWorkerIfRequired();
+		}
+	}
+
+	@Override
+	public void onError(Throwable exception) {
+		onFatalError(exception);
+	}
+
+	@Override
+	public void handleInMainThread(Runnable runnable) {
+		runAsync(runnable);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Internal
+	// ------------------------------------------------------------------------
+
+	private void requestNewWorker(WorkerResourceSpec workerResourceSpec) {
+		final TaskExecutorProcessSpec taskExecutorProcessSpec =
+				TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, workerResourceSpec);
+		final int pendingCount = pendingWorkerCounter.increaseAndGet(workerResourceSpec);
+
+		log.info("Requesting new worker with resource spec {}, current pending count: {}.",
+				workerResourceSpec,
+				pendingCount);
+
+		resourceManagerDriver.requestResource(taskExecutorProcessSpec)
+				.whenComplete((worker, exception) -> {
+					if (exception != null) {
+						final int count = pendingWorkerCounter.decreaseAndGet(workerResourceSpec);
+						log.warn("Failed requesting worker with resource spec {}, current pending count: {}, exception: {}",
+								workerResourceSpec,
+								count,
+								exception);
+						requestWorkerIfRequired();
+					} else {
+						final ResourceID resourceId = worker.getResourceID();
+						workerNodeMap.put(resourceId, worker);
+						currentAttemptUnregisteredWorkers.put(resourceId, workerResourceSpec);
+						log.info("Requested worker {} with resource spec {}.",
+								resourceId,
+								workerResourceSpec);
+					}
+				});

Review comment:
       It would be good to guard against unwanted exceptions via `FutureUtils.assertNoException(...)`. In this case `whenComplete` would have to become `handle` which returns `null`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/CommonProcessMemorySpec.java
##########
@@ -94,4 +96,16 @@ public MemorySize getTotalFlinkMemorySize() {
 	public MemorySize getTotalProcessMemorySize() {
 		return flinkMemory.getTotalFlinkMemorySize().add(getJvmMetaspaceSize()).add(getJvmOverheadSize());
 	}
+
+	@Override
+	public boolean equals(Object obj) {

Review comment:
       For what do we need the `equals` method on `CommonProcessMemorySpec`?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/active/TestingResourceManagerDriver.java
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.ApplicationStatus;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.BiConsumerWithException;
+import org.apache.flink.util.function.ConsumerWithException;
+
+import javax.annotation.Nullable;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Supplier;
+
+/**
+ * Testing implementation of {@link ResourceManagerDriver}.
+ */
+public class TestingResourceManagerDriver implements ResourceManagerDriver<ResourceID> {
+
+	private final ConsumerWithException<ResourceEventHandler<ResourceID>, Throwable> initializeConsumer;
+	private final Supplier<CompletableFuture<Void>> terminateSupplier;
+	private final BiConsumerWithException<ApplicationStatus, String, Throwable> deregisterApplicationConsumer;
+	private final Function<TaskExecutorProcessSpec, CompletableFuture<ResourceID>> requestResourceFunction;
+	private final Consumer<ResourceID> releaseResourceConsumer;
+
+	private TestingResourceManagerDriver(
+			final ConsumerWithException<ResourceEventHandler<ResourceID>, Throwable> initializeConsumer,
+			final Supplier<CompletableFuture<Void>> terminateSupplier,
+			final BiConsumerWithException<ApplicationStatus, String, Throwable> deregisterApplicationConsumer,
+			final Function<TaskExecutorProcessSpec, CompletableFuture<ResourceID>> requestResourceFunction,
+			final Consumer<ResourceID> releaseResourceConsumer) {
+		this.initializeConsumer = Preconditions.checkNotNull(initializeConsumer);
+		this.terminateSupplier = Preconditions.checkNotNull(terminateSupplier);
+		this.deregisterApplicationConsumer = Preconditions.checkNotNull(deregisterApplicationConsumer);
+		this.requestResourceFunction = Preconditions.checkNotNull(requestResourceFunction);
+		this.releaseResourceConsumer = Preconditions.checkNotNull(releaseResourceConsumer);
+	}
+
+	@Override
+	public void initialize(ResourceEventHandler<ResourceID> resourceEventHandler) throws Throwable {
+		initializeConsumer.accept(resourceEventHandler);
+	}
+
+	@Override
+	public CompletableFuture<Void> terminate() {
+		return terminateSupplier.get();
+	}
+
+	@Override
+	public void deregisterApplication(ApplicationStatus finalStatus, @Nullable String optionalDiagnostics) throws Throwable {
+		deregisterApplicationConsumer.accept(finalStatus, optionalDiagnostics);
+	}
+
+	@Override
+	public CompletableFuture<ResourceID> requestResource(TaskExecutorProcessSpec taskExecutorProcessSpec) {
+		return requestResourceFunction.apply(taskExecutorProcessSpec);
+	}
+
+	@Override
+	public void releaseResource(ResourceID worker) {
+		releaseResourceConsumer.accept(worker);
+	}
+
+	public static class Builder {
+		private ConsumerWithException<ResourceEventHandler<ResourceID>, Throwable> initializeConsumer =
+				(ignore) -> {};
+
+		private Supplier<CompletableFuture<Void>> terminateSupplier =
+				() -> CompletableFuture.completedFuture(null);
+
+		private BiConsumerWithException<ApplicationStatus, String, Throwable> deregisterApplicationConsumer =
+				(ignore1, ignore20) -> {};

Review comment:
       ```suggestion
   				(ignore1, ignore2) -> {};
   ```

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/active/ActiveResourceManagerTest.java
##########
@@ -0,0 +1,428 @@
+/*
+ * 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.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.entrypoint.ClusterInformation;
+import org.apache.flink.runtime.instance.HardwareDescription;
+import org.apache.flink.runtime.io.network.partition.NoOpResourceManagerPartitionTracker;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.registration.RegistrationResponse;
+import org.apache.flink.runtime.resourcemanager.TaskExecutorRegistration;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager;
+import org.apache.flink.runtime.resourcemanager.slotmanager.TestingSlotManagerBuilder;
+import org.apache.flink.runtime.resourcemanager.utils.MockResourceManagerRuntimeServices;
+import org.apache.flink.runtime.rpc.TestingRpcService;
+import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.runtime.util.TestingFatalErrorHandler;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.hamcrest.Matchers.instanceOf;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.lessThan;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for {@link ActiveResourceManager}.
+ */
+public class ActiveResourceManagerTest extends TestLogger {
+
+	private static final long TIMEOUT_SEC = 5L;
+	private static final Time TIMEOUT_TIME = Time.seconds(TIMEOUT_SEC);
+
+	private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = WorkerResourceSpec.ZERO;
+
+	/**
+	 * Tests worker successfully requested, started and registered.
+	 */
+	@Test
+	public void testStartNewWorker() throws Exception {
+		new Context() {{
+			final ResourceID tmResourceId = ResourceID.generate();
+			final CompletableFuture<TaskExecutorProcessSpec> requestWorkerFromDriverFuture = new CompletableFuture<>();
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				requestWorkerFromDriverFuture.complete(taskExecutorProcessSpec);
+				return CompletableFuture.completedFuture(tmResourceId);
+			});
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec = requestWorkerFromDriverFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// worker registered, verify registration succeed

Review comment:
       ```suggestion
   				// worker registered, verify registration succeeded
   ```

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/ActiveResourceManager.java
##########
@@ -0,0 +1,287 @@
+/*
+ * 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.annotation.VisibleForTesting;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.akka.AkkaUtils;
+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.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.ResourceManagerPartitionTrackerFactory;
+import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.JobLeaderIdService;
+import org.apache.flink.runtime.resourcemanager.ResourceManager;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException;
+import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * An active implementation of {@link ResourceManager}.
+ *
+ * <p>This resource manager actively requests and releases resources from/to the external resource management frameworks.
+ * With different {@link ResourceManagerDriver} provided, this resource manager can work with various frameworks.
+ */
+public class ActiveResourceManager<WorkerType extends ResourceIDRetrievable>
+		extends ResourceManager<WorkerType> implements ResourceEventHandler<WorkerType> {
+
+	protected final Configuration flinkConfig;
+
+	private final ResourceManagerDriver<WorkerType> resourceManagerDriver;
+
+	/** All workers maintained by {@link ActiveResourceManager}. */
+	private final Map<ResourceID, WorkerType> workerNodeMap;
+
+	/** Number of requested and not registered workers per worker resource spec. */
+	private final PendingWorkerCounter pendingWorkerCounter;
+
+	/** Identifiers and worker resource spec of requested not registered workers. */
+	private final Map<ResourceID, WorkerResourceSpec> currentAttemptUnregisteredWorkers;
+
+	public ActiveResourceManager(
+			ResourceManagerDriver<WorkerType> resourceManagerDriver,
+			Configuration flinkConfig,
+			RpcService rpcService,
+			ResourceID resourceId,
+			HighAvailabilityServices highAvailabilityServices,
+			HeartbeatServices heartbeatServices,
+			SlotManager slotManager,
+			ResourceManagerPartitionTrackerFactory clusterPartitionTrackerFactory,
+			JobLeaderIdService jobLeaderIdService,
+			ClusterInformation clusterInformation,
+			FatalErrorHandler fatalErrorHandler,
+			ResourceManagerMetricGroup resourceManagerMetricGroup) {
+		super(
+				rpcService,
+				resourceId,
+				highAvailabilityServices,
+				heartbeatServices,
+				slotManager,
+				clusterPartitionTrackerFactory,
+				jobLeaderIdService,
+				clusterInformation,
+				fatalErrorHandler,
+				resourceManagerMetricGroup,
+				AkkaUtils.getTimeoutAsTime(Preconditions.checkNotNull(flinkConfig)));
+
+		this.flinkConfig = flinkConfig;
+		this.resourceManagerDriver = Preconditions.checkNotNull(resourceManagerDriver);

Review comment:
       null checks seem a bit inconsistent here.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/active/ActiveResourceManagerTest.java
##########
@@ -0,0 +1,428 @@
+/*
+ * 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.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.entrypoint.ClusterInformation;
+import org.apache.flink.runtime.instance.HardwareDescription;
+import org.apache.flink.runtime.io.network.partition.NoOpResourceManagerPartitionTracker;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.registration.RegistrationResponse;
+import org.apache.flink.runtime.resourcemanager.TaskExecutorRegistration;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager;
+import org.apache.flink.runtime.resourcemanager.slotmanager.TestingSlotManagerBuilder;
+import org.apache.flink.runtime.resourcemanager.utils.MockResourceManagerRuntimeServices;
+import org.apache.flink.runtime.rpc.TestingRpcService;
+import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.runtime.util.TestingFatalErrorHandler;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.hamcrest.Matchers.instanceOf;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.lessThan;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for {@link ActiveResourceManager}.
+ */
+public class ActiveResourceManagerTest extends TestLogger {
+
+	private static final long TIMEOUT_SEC = 5L;
+	private static final Time TIMEOUT_TIME = Time.seconds(TIMEOUT_SEC);
+
+	private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = WorkerResourceSpec.ZERO;
+
+	/**
+	 * Tests worker successfully requested, started and registered.
+	 */
+	@Test
+	public void testStartNewWorker() throws Exception {
+		new Context() {{
+			final ResourceID tmResourceId = ResourceID.generate();
+			final CompletableFuture<TaskExecutorProcessSpec> requestWorkerFromDriverFuture = new CompletableFuture<>();
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				requestWorkerFromDriverFuture.complete(taskExecutorProcessSpec);
+				return CompletableFuture.completedFuture(tmResourceId);
+			});
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec = requestWorkerFromDriverFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// worker registered, verify registration succeed
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(tmResourceId);
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+			});
+		}};
+	}
+
+	/**
+	 * Tests worker failed while requesting.
+	 */
+	@Test
+	public void testStartNewWorkerFailedRequesting() throws Exception {
+		new Context() {{
+			final ResourceID tmResourceId = ResourceID.generate();
+			final AtomicInteger requestCount = new AtomicInteger(0);
+
+			final List<CompletableFuture<ResourceID>> resourceIdFutures = new ArrayList<>();
+			resourceIdFutures.add(new CompletableFuture<>());
+			resourceIdFutures.add(new CompletableFuture<>());
+
+			final List<CompletableFuture<TaskExecutorProcessSpec>> requestWorkerFromDriverFutures = new ArrayList<>();
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				int idx = requestCount.getAndIncrement();
+				assertThat(idx, lessThan(2));
+
+				requestWorkerFromDriverFutures.get(idx).complete(taskExecutorProcessSpec);
+				return resourceIdFutures.get(idx);
+			});
+
+			slotManagerBuilder.setGetRequiredResourcesSupplier(() -> Collections.singletonMap(WORKER_RESOURCE_SPEC, 1));
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec1 = requestWorkerFromDriverFutures.get(0).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec1,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// first request failed, verify requesting another worker from driver
+				runInMainThread(() -> resourceIdFutures.get(0).completeExceptionally(new Throwable("testing error")));
+				TaskExecutorProcessSpec taskExecutorProcessSpec2 =
+						requestWorkerFromDriverFutures.get(1).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(taskExecutorProcessSpec2, is(taskExecutorProcessSpec1));
+
+				// second request allocated, verify registration succeed
+				runInMainThread(() -> resourceIdFutures.get(1).complete(tmResourceId));
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(tmResourceId);
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+			});
+		}};
+	}
+
+	/**
+	 * Tests worker terminated after requested before registered.
+	 */
+	@Test
+	public void testWorkerTerminatedBeforeRegister() throws Exception {
+		new Context() {{
+			final AtomicInteger requestCount = new AtomicInteger(0);
+
+			final List<ResourceID> tmResourceIds = new ArrayList<>();
+			tmResourceIds.add(ResourceID.generate());
+			tmResourceIds.add(ResourceID.generate());
+
+			final List<CompletableFuture<TaskExecutorProcessSpec>> requestWorkerFromDriverFutures = new ArrayList<>();
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				int idx = requestCount.getAndIncrement();
+				assertThat(idx, lessThan(2));
+
+				requestWorkerFromDriverFutures.get(idx).complete(taskExecutorProcessSpec);
+				return CompletableFuture.completedFuture(tmResourceIds.get(idx));
+			});
+
+			slotManagerBuilder.setGetRequiredResourcesSupplier(() -> Collections.singletonMap(WORKER_RESOURCE_SPEC, 1));
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec1 = requestWorkerFromDriverFutures.get(0).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec1,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// first worker failed before register, verify requesting another worker from driver
+				runInMainThread(() -> getResourceManager().onWorkerTerminated(tmResourceIds.get(0)));
+				TaskExecutorProcessSpec taskExecutorProcessSpec2 =
+						requestWorkerFromDriverFutures.get(1).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(taskExecutorProcessSpec2, is(taskExecutorProcessSpec1));
+
+				// second worker registered, verify registration succeed
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(tmResourceIds.get(1));
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+			});
+		}};
+	}
+
+	/**
+	 * Tests worker terminated after registered.
+	 */
+	@Test
+	public void testWorkerTerminatedAfterRegister() throws Exception {
+		new Context() {{
+			final AtomicInteger requestCount = new AtomicInteger(0);
+
+			final List<ResourceID> tmResourceIds = new ArrayList<>();
+			tmResourceIds.add(ResourceID.generate());
+			tmResourceIds.add(ResourceID.generate());
+
+			final List<CompletableFuture<TaskExecutorProcessSpec>> requestWorkerFromDriverFutures = new ArrayList<>();
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				int idx = requestCount.getAndIncrement();
+				assertThat(idx, lessThan(2));
+
+				requestWorkerFromDriverFutures.get(idx).complete(taskExecutorProcessSpec);
+				return CompletableFuture.completedFuture(tmResourceIds.get(idx));
+			});
+
+			slotManagerBuilder.setGetRequiredResourcesSupplier(() -> Collections.singletonMap(WORKER_RESOURCE_SPEC, 1));
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec1 = requestWorkerFromDriverFutures.get(0).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec1,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// first worker registered, verify registration succeed
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture1 = registerTaskExecutor(tmResourceIds.get(0));
+				assertThat(registerTaskExecutorFuture1.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+
+				// first worker terminated, verify requesting another worker from driver
+				runInMainThread(() -> getResourceManager().onWorkerTerminated(tmResourceIds.get(0)));
+				TaskExecutorProcessSpec taskExecutorProcessSpec2 =
+						requestWorkerFromDriverFutures.get(1).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(taskExecutorProcessSpec2, is(taskExecutorProcessSpec1));
+
+				// second worker registered, verify registration succeed
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture2 = registerTaskExecutor(tmResourceIds.get(1));
+				assertThat(registerTaskExecutorFuture2.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+			});
+		}};
+	}
+
+	/**
+	 * Tests worker terminated and is no longer required.
+	 */
+	@Test
+	public void testWorkerTerminatedNoLongerRequired() throws Exception {
+		new Context() {{
+			final ResourceID tmResourceId = ResourceID.generate();
+			final AtomicInteger requestCount = new AtomicInteger(0);
+
+			final List<CompletableFuture<TaskExecutorProcessSpec>> requestWorkerFromDriverFutures = new ArrayList<>();
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				int idx = requestCount.getAndIncrement();
+				assertThat(idx, lessThan(2));
+
+				requestWorkerFromDriverFutures.get(idx).complete(taskExecutorProcessSpec);
+				return CompletableFuture.completedFuture(tmResourceId);
+			});
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec = requestWorkerFromDriverFutures.get(0).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// worker registered, verify registration succeed
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(tmResourceId);
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+
+				// worker terminated, verify not requesting new worker
+				runInMainThread(() -> {
+					getResourceManager().onWorkerTerminated(tmResourceId);
+					// needs to return something, so that we can use `get()` to make sure the main thread processing
+					// finishes before the assertions
+					return null;
+				}).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+				assertFalse(requestWorkerFromDriverFutures.get(1).isDone());
+			});
+		}};
+	}
+
+	/**
+	 * Tests workers from previous attempt successfully recovered and registered.
+	 */
+	@Test
+	public void testRecoverWorkerFromPreviousAttempt() throws Exception {
+		new Context() {{
+			final ResourceID tmResourceId = ResourceID.generate();
+
+			runTest(() -> {
+				runInMainThread(() -> getResourceManager().onPreviousAttemptWorkersRecovered(Collections.singleton(tmResourceId)));
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(tmResourceId);
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+			});
+		}};
+	}
+
+	/**
+	 * Tests decline unknown worker registration.
+	 */
+	@Test
+	public void testRegisterUnknownWorker() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(ResourceID.generate());
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Decline.class));
+			});
+		}};
+	}
+
+	@Test
+	public void testOnError() throws Exception {
+		new Context() {{
+			final Throwable fatalError = new Throwable("Testing fatal error");
+			runTest(() -> {
+				runInMainThread(() -> getResourceManager().onError(fatalError));
+				final Throwable reportedError = getFatalErrorHandler().getErrorFuture().get(TIMEOUT_SEC, TimeUnit.SECONDS);
+				assertThat(reportedError, is(fatalError));
+			});
+		}};
+	}
+
+	class Context {
+
+		final Configuration flinkConfig = new Configuration();
+		final TestingResourceManagerDriver.Builder driverBuilder = new TestingResourceManagerDriver.Builder();
+		final TestingSlotManagerBuilder slotManagerBuilder = new TestingSlotManagerBuilder();
+
+		private ActiveResourceManager<ResourceID> resourceManager;
+		private TestingFatalErrorHandler fatalErrorHandler;
+
+		ActiveResourceManager<ResourceID> getResourceManager() {
+			return resourceManager;
+		}
+
+		TestingFatalErrorHandler getFatalErrorHandler() {
+			return fatalErrorHandler;
+		}
+
+		void runTest(RunnableWithException testMethod) throws Exception {
+			fatalErrorHandler = new TestingFatalErrorHandler();
+			resourceManager = createAndStartResourceManager(
+					flinkConfig,
+					driverBuilder.build(),
+					slotManagerBuilder.createSlotManager());
+
+			try {
+				testMethod.run();
+			} finally {
+				resourceManager.close();
+			}
+		}
+
+		private ActiveResourceManager<ResourceID> createAndStartResourceManager(
+				Configuration configuration,
+				ResourceManagerDriver<ResourceID> driver,
+				SlotManager slotManager) throws Exception {
+			final TestingRpcService rpcService = new TestingRpcService(configuration);
+			final MockResourceManagerRuntimeServices rmServices = new MockResourceManagerRuntimeServices(rpcService, TIMEOUT_TIME, slotManager);
+
+			final ActiveResourceManager<ResourceID> activeResourceManager = new ActiveResourceManager<>(
+					driver,
+					configuration,
+					rpcService,
+					ResourceID.generate(),
+					rmServices.highAvailabilityServices,
+					rmServices.heartbeatServices,
+					rmServices.slotManager,
+					NoOpResourceManagerPartitionTracker::get,
+					rmServices.jobLeaderIdService,
+					new ClusterInformation("localhost", 1234),
+					fatalErrorHandler,
+					UnregisteredMetricGroups.createUnregisteredResourceManagerMetricGroup());
+
+			activeResourceManager.start();
+			rmServices.grantLeadership();
+
+			return activeResourceManager;
+		}
+
+		public void runInMainThread(Runnable runnable) {
+			resourceManager.handleInMainThread(runnable);
+		}
+
+		public <T> CompletableFuture<T> runInMainThread(Callable<T> callable) {
+			return resourceManager.runInMainThread(callable, TIMEOUT_TIME);
+		}
+
+		CompletableFuture<RegistrationResponse> registerTaskExecutor(ResourceID resourceID) throws Exception {
+			final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+					.createTestingTaskExecutorGateway();
+			((TestingRpcService) resourceManager.getRpcService()).registerGateway(resourceID.toString(), taskExecutorGateway);
+
+			final TaskExecutorRegistration taskExecutorRegistration = new TaskExecutorRegistration(
+					resourceID.toString(),
+					resourceID,
+					1234,
+					new HardwareDescription(1, 2L, 3L, 4L),
+					ResourceProfile.ZERO,
+					ResourceProfile.ZERO);
+
+			return runInMainThread(() -> resourceManager.registerTaskExecutor(taskExecutorRegistration, TIMEOUT_TIME))
+					.get(TIMEOUT_SEC, TimeUnit.SECONDS);

Review comment:
       We could avoid running something in the main thread by using the `ResourceManagerGateway.registerTaskExecutor` 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 a change in pull request #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/ActiveResourceManager.java
##########
@@ -0,0 +1,287 @@
+/*
+ * 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.annotation.VisibleForTesting;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.akka.AkkaUtils;
+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.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.ResourceManagerPartitionTrackerFactory;
+import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.JobLeaderIdService;
+import org.apache.flink.runtime.resourcemanager.ResourceManager;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException;
+import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * An active implementation of {@link ResourceManager}.
+ *
+ * <p>This resource manager actively requests and releases resources from/to the external resource management frameworks.
+ * With different {@link ResourceManagerDriver} provided, this resource manager can work with various frameworks.
+ */
+public class ActiveResourceManager<WorkerType extends ResourceIDRetrievable>
+		extends ResourceManager<WorkerType> implements ResourceEventHandler<WorkerType> {
+
+	protected final Configuration flinkConfig;
+
+	private final ResourceManagerDriver<WorkerType> resourceManagerDriver;
+
+	/** All workers maintained by {@link ActiveResourceManager}. */
+	private final Map<ResourceID, WorkerType> workerNodeMap;
+
+	/** Number of requested and not registered workers per worker resource spec. */
+	private final PendingWorkerCounter pendingWorkerCounter;
+
+	/** Identifiers and worker resource spec of requested not registered workers. */
+	private final Map<ResourceID, WorkerResourceSpec> currentAttemptUnregisteredWorkers;
+
+	public ActiveResourceManager(
+			ResourceManagerDriver<WorkerType> resourceManagerDriver,
+			Configuration flinkConfig,
+			RpcService rpcService,
+			ResourceID resourceId,
+			HighAvailabilityServices highAvailabilityServices,
+			HeartbeatServices heartbeatServices,
+			SlotManager slotManager,
+			ResourceManagerPartitionTrackerFactory clusterPartitionTrackerFactory,
+			JobLeaderIdService jobLeaderIdService,
+			ClusterInformation clusterInformation,
+			FatalErrorHandler fatalErrorHandler,
+			ResourceManagerMetricGroup resourceManagerMetricGroup) {
+		super(
+				rpcService,
+				resourceId,
+				highAvailabilityServices,
+				heartbeatServices,
+				slotManager,
+				clusterPartitionTrackerFactory,
+				jobLeaderIdService,
+				clusterInformation,
+				fatalErrorHandler,
+				resourceManagerMetricGroup,
+				AkkaUtils.getTimeoutAsTime(Preconditions.checkNotNull(flinkConfig)));
+
+		this.flinkConfig = flinkConfig;
+		this.resourceManagerDriver = Preconditions.checkNotNull(resourceManagerDriver);
+		this.workerNodeMap = new HashMap<>();
+		this.pendingWorkerCounter = new PendingWorkerCounter();
+		this.currentAttemptUnregisteredWorkers = new HashMap<>();
+	}
+
+	// ------------------------------------------------------------------------
+	//  ResourceManager
+	// ------------------------------------------------------------------------
+
+	@Override
+	protected void initialize() throws ResourceManagerException {
+		try {
+			resourceManagerDriver.initialize(this);
+		} catch (Throwable t) {
+			throw new ResourceManagerException("Cannot initialize resource provider.", t);
+		}
+	}
+
+	@Override
+	protected void terminate() throws ResourceManagerException {
+		try {
+			resourceManagerDriver.terminate().get();
+		} catch (Throwable t) {
+			throw new ResourceManagerException("Cannot terminate resource provider.", t);
+		}
+	}
+
+	@Override
+	protected void internalDeregisterApplication(ApplicationStatus finalStatus, @Nullable String optionalDiagnostics)
+			throws ResourceManagerException {
+		try {
+			resourceManagerDriver.deregisterApplication(finalStatus, optionalDiagnostics);
+		} catch (Throwable t) {
+			throw new ResourceManagerException("Cannot deregister application.", t);
+		}
+	}
+
+	@Override
+	public boolean startNewWorker(WorkerResourceSpec workerResourceSpec) {
+		requestNewWorker(workerResourceSpec);
+		return true;
+	}
+
+	@Override
+	protected WorkerType workerStarted(ResourceID resourceID) {
+		return workerNodeMap.get(resourceID);
+	}
+
+	@Override
+	public boolean stopWorker(WorkerType worker) {
+		final ResourceID resourceId = worker.getResourceID();
+		resourceManagerDriver.releaseResource(worker);
+
+		log.info("Stopping worker {}.", resourceId);
+
+		clearStateForWorker(resourceId);
+
+		return true;
+	}
+
+	@Override
+	protected void onWorkerRegistered(WorkerType worker) {
+		final ResourceID resourceId = worker.getResourceID();
+		log.info("Worker {} is registered.", resourceId);
+
+		final WorkerResourceSpec workerResourceSpec = currentAttemptUnregisteredWorkers.remove(resourceId);
+		if (workerResourceSpec != null) {
+			final int count = pendingWorkerCounter.decreaseAndGet(workerResourceSpec);
+			log.info("Worker {} with resource spec {} was requested in current attempt." +
+							" Current pending count after registering: {}.",
+					resourceId,
+					workerResourceSpec,
+					count);
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  ResourceEventListener
+	// ------------------------------------------------------------------------
+
+	@Override
+	public void onPreviousAttemptWorkersRecovered(Collection<WorkerType> recoveredWorkers) {
+		log.info("Recovered {} workers from previous attempt.", recoveredWorkers.size());
+		for (WorkerType worker : recoveredWorkers) {
+			final ResourceID resourceId = worker.getResourceID();
+			workerNodeMap.put(resourceId, worker);
+			log.info("Worker {} recovered from previous attempt.", resourceId);
+		}
+	}
+
+	@Override
+	public void onWorkerTerminated(ResourceID resourceId) {
+		log.info("Worker {} is terminated.", resourceId);
+		if (clearStateForWorker(resourceId)) {
+			requestWorkerIfRequired();
+		}
+	}
+
+	@Override
+	public void onError(Throwable exception) {
+		onFatalError(exception);
+	}
+
+	@Override
+	public void handleInMainThread(Runnable runnable) {
+		runAsync(runnable);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Internal
+	// ------------------------------------------------------------------------
+
+	private void requestNewWorker(WorkerResourceSpec workerResourceSpec) {
+		final TaskExecutorProcessSpec taskExecutorProcessSpec =
+				TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, workerResourceSpec);
+		final int pendingCount = pendingWorkerCounter.increaseAndGet(workerResourceSpec);
+
+		log.info("Requesting new worker with resource spec {}, current pending count: {}.",
+				workerResourceSpec,
+				pendingCount);
+
+		resourceManagerDriver.requestResource(taskExecutorProcessSpec)
+				.whenComplete((worker, exception) -> {
+					if (exception != null) {
+						final int count = pendingWorkerCounter.decreaseAndGet(workerResourceSpec);
+						log.warn("Failed requesting worker with resource spec {}, current pending count: {}, exception: {}",
+								workerResourceSpec,
+								count,
+								exception);
+						requestWorkerIfRequired();
+					} else {
+						final ResourceID resourceId = worker.getResourceID();
+						workerNodeMap.put(resourceId, worker);
+						currentAttemptUnregisteredWorkers.put(resourceId, workerResourceSpec);
+						log.info("Requested worker {} with resource spec {}.",
+								resourceId,
+								workerResourceSpec);
+					}
+				});

Review comment:
       Ah, I see. Thanks for the explanation.




----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/jobmanager/JobManagerFlinkMemory.java
##########
@@ -69,4 +73,23 @@ public MemorySize getJvmDirectMemorySize() {
 	public MemorySize getTotalFlinkMemorySize() {
 		return jvmHeap.add(offHeapMemory);
 	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (obj == this) {
+			return true;
+		} else if (obj instanceof JobManagerFlinkMemory) {
+			JobManagerFlinkMemory that = (JobManagerFlinkMemory) obj;
+			return Objects.equals(this.jvmHeap, that.jvmHeap) &&
+					Objects.equals(this. offHeapMemory, that.offHeapMemory);

Review comment:
       Here's also a formatting problem having the space between `this.` and `offHeapMemory`. Alternatively, `this.` could be removed entirely to be on par with the `hashCode()` implementation.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/JvmMetaspaceAndOverhead.java
##########
@@ -49,4 +50,23 @@ public MemorySize getMetaspace() {
 	public MemorySize getOverhead() {
 		return overhead;
 	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (obj == this) {
+			return true;
+		} else if (obj instanceof JvmMetaspaceAndOverhead ) {
+			JvmMetaspaceAndOverhead that = (JvmMetaspaceAndOverhead) obj;
+			return Objects.equals(this.metaspace, that.metaspace) &&
+					Objects.equals(this. overhead, that.overhead);

Review comment:
       Here as well: space between `this.` and `overhead` shouldn't be there.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/CommonProcessMemorySpec.java
##########
@@ -94,4 +96,23 @@ public MemorySize getTotalFlinkMemorySize() {
 	public MemorySize getTotalProcessMemorySize() {
 		return flinkMemory.getTotalFlinkMemorySize().add(getJvmMetaspaceSize()).add(getJvmOverheadSize());
 	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (obj == this) {
+			return true;
+		} else if (obj instanceof CommonProcessMemorySpec<?> ) {
+			CommonProcessMemorySpec<?> that = (CommonProcessMemorySpec<?>) obj;
+			return Objects.equals(this.flinkMemory, that.flinkMemory) &&
+					Objects.equals(this. jvmMetaspaceAndOverhead, that.jvmMetaspaceAndOverhead);

Review comment:
       Here as well: Space between `this.` and `jvmMetaspaceAndOverhead`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
##########
@@ -1064,6 +1076,13 @@ public void handleError(final Exception exception) {
 	 */
 	protected abstract void initialize() throws ResourceManagerException;
 
+	/**
+	 * Terminates the framework specific components.
+	 *
+	 * @throws Exception with occurs during termination.
+	 */
+	protected abstract void terminate() throws Exception;

Review comment:
       Just as a side note: As discussed with @tillrohrmann making terminate being non-blocking would be follow-up feature we might consider later on.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/taskmanager/TaskExecutorFlinkMemory.java
##########
@@ -128,4 +130,31 @@ public MemorySize getJvmDirectMemorySize() {
 	public MemorySize getTotalFlinkMemorySize() {
 		return frameworkHeap.add(frameworkOffHeap).add(taskHeap).add(taskOffHeap).add(network).add(managed);
 	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (obj == this) {
+			return true;
+		} else if (obj instanceof TaskExecutorFlinkMemory) {
+			TaskExecutorFlinkMemory that = (TaskExecutorFlinkMemory) obj;
+			return Objects.equals(this.frameworkHeap, that.frameworkHeap) &&
+					Objects.equals(this. frameworkOffHeap, that.frameworkOffHeap) &&
+					Objects.equals(this. taskHeap, that.taskHeap)&&
+					Objects.equals(this. taskOffHeap, that.taskOffHeap)&&
+					Objects.equals(this. network, that.network)&&
+					Objects.equals(this. managed, that.managed);

Review comment:
       Interesting: I would have thought that this is causing a compilation error. But it looks like the compiler can deal with spaces between `this.` and the member. I learned something new. :-) Anyway, the formatting should be fixed here removing the spaces between `this.` and the member variable and adding a space before the `&&` operators.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/active/TestingResourceManagerDriver.java
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.ApplicationStatus;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.BiConsumerWithException;
+
+import javax.annotation.Nullable;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Supplier;
+
+/**
+ * Testing implementation of {@link ResourceManagerDriver}.
+ */
+public class TestingResourceManagerDriver implements ResourceManagerDriver<ResourceID> {
+
+	private final BiConsumerWithException<ResourceEventHandler<ResourceID>, Executor, Exception> initializeConsumer;
+	private final Supplier<CompletableFuture<Void>> terminateSupplier;
+	private final BiConsumerWithException<ApplicationStatus, String, Exception> deregisterApplicationConsumer;
+	private final Function<TaskExecutorProcessSpec, CompletableFuture<ResourceID>> requestResourceFunction;
+	private final Consumer<ResourceID> releaseResourceConsumer;
+
+	private TestingResourceManagerDriver(
+			final BiConsumerWithException<ResourceEventHandler<ResourceID>, Executor, Exception> initializeConsumer,
+			final Supplier<CompletableFuture<Void>> terminateSupplier,
+			final BiConsumerWithException<ApplicationStatus, String, Exception> deregisterApplicationConsumer,
+			final Function<TaskExecutorProcessSpec, CompletableFuture<ResourceID>> requestResourceFunction,
+			final Consumer<ResourceID> releaseResourceConsumer) {
+		this.initializeConsumer = Preconditions.checkNotNull(initializeConsumer);
+		this.terminateSupplier = Preconditions.checkNotNull(terminateSupplier);
+		this.deregisterApplicationConsumer = Preconditions.checkNotNull(deregisterApplicationConsumer);
+		this.requestResourceFunction = Preconditions.checkNotNull(requestResourceFunction);
+		this.releaseResourceConsumer = Preconditions.checkNotNull(releaseResourceConsumer);
+	}
+
+	@Override
+	public void initialize(ResourceEventHandler<ResourceID> resourceEventHandler, Executor mainThreadExecutor) throws Exception {
+		initializeConsumer.accept(resourceEventHandler, mainThreadExecutor);
+	}
+
+	@Override
+	public CompletableFuture<Void> terminate() {
+		return terminateSupplier.get();
+	}
+
+	@Override
+	public void deregisterApplication(ApplicationStatus finalStatus, @Nullable String optionalDiagnostics) throws Exception {
+		deregisterApplicationConsumer.accept(finalStatus, optionalDiagnostics);
+	}
+
+	@Override
+	public CompletableFuture<ResourceID> requestResource(TaskExecutorProcessSpec taskExecutorProcessSpec) {
+		return requestResourceFunction.apply(taskExecutorProcessSpec);
+	}
+
+	@Override
+	public void releaseResource(ResourceID worker) {
+		releaseResourceConsumer.accept(worker);
+	}
+
+	public static class Builder {
+		private BiConsumerWithException<ResourceEventHandler<ResourceID>, Executor, Exception> initializeConsumer =
+				(ignore1, ignore2) -> {};
+
+		private Supplier<CompletableFuture<Void>> terminateSupplier =
+				() -> CompletableFuture.completedFuture(null);
+
+		private BiConsumerWithException<ApplicationStatus, String, Exception> deregisterApplicationConsumer =
+				(ignore1, ignore2) -> {};
+
+		private Function<TaskExecutorProcessSpec, CompletableFuture<ResourceID>> requestResourceFunction =
+				(ignore) -> CompletableFuture.completedFuture(ResourceID.generate());
+
+		private Consumer<ResourceID> releaseResourceConsumer =
+				(ignore) -> {};
+
+		public Builder setInitializeConsumer(BiConsumerWithException<ResourceEventHandler<ResourceID>, Executor, Exception> initializeConsumer) {

Review comment:
       CheckStyle complains about missing JavaDoc here.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/ResourceManagerDriver.java
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.ApplicationStatus;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec;
+import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable;
+import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException;
+
+import javax.annotation.Nullable;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/**
+ * A {@link ResourceManagerDriver} is responsible for requesting and releasing resources from/to a particular external
+ * resource manager.
+ */
+public interface ResourceManagerDriver<WorkerType extends ResourceIDRetrievable> {
+
+	/**
+	 * Initialize the deployment specific components.
+	 *
+	 * @param resourceEventHandler Handler that handles resource events.
+	 * @param mainThreadExecutor Rpc main thread executor.
+	 */
+	void initialize(ResourceEventHandler<WorkerType> resourceEventHandler, Executor mainThreadExecutor) throws Exception;
+
+	/**
+	 * Terminate the deployment specific components.
+	 *
+	 * @return A future that will be completed successfully when the driver is terminated, or exceptionally if cannot be

Review comment:
       ```suggestion
   	 * @return A future that will be completed successfully when the driver is terminated, or exceptionally if it cannot be
   ```




----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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


   Thanks @tillrohrmann. I've addressed the comments.


----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/active/ActiveResourceManagerTest.java
##########
@@ -0,0 +1,428 @@
+/*
+ * 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.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.entrypoint.ClusterInformation;
+import org.apache.flink.runtime.instance.HardwareDescription;
+import org.apache.flink.runtime.io.network.partition.NoOpResourceManagerPartitionTracker;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.registration.RegistrationResponse;
+import org.apache.flink.runtime.resourcemanager.TaskExecutorRegistration;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager;
+import org.apache.flink.runtime.resourcemanager.slotmanager.TestingSlotManagerBuilder;
+import org.apache.flink.runtime.resourcemanager.utils.MockResourceManagerRuntimeServices;
+import org.apache.flink.runtime.rpc.TestingRpcService;
+import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.runtime.util.TestingFatalErrorHandler;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.hamcrest.Matchers.instanceOf;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.lessThan;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for {@link ActiveResourceManager}.
+ */
+public class ActiveResourceManagerTest extends TestLogger {
+
+	private static final long TIMEOUT_SEC = 5L;
+	private static final Time TIMEOUT_TIME = Time.seconds(TIMEOUT_SEC);
+
+	private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = WorkerResourceSpec.ZERO;
+
+	/**
+	 * Tests worker successfully requested, started and registered.
+	 */
+	@Test
+	public void testStartNewWorker() throws Exception {
+		new Context() {{
+			final ResourceID tmResourceId = ResourceID.generate();
+			final CompletableFuture<TaskExecutorProcessSpec> requestWorkerFromDriverFuture = new CompletableFuture<>();
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				requestWorkerFromDriverFuture.complete(taskExecutorProcessSpec);
+				return CompletableFuture.completedFuture(tmResourceId);
+			});
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec = requestWorkerFromDriverFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// worker registered, verify registration succeed
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(tmResourceId);
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+			});
+		}};
+	}
+
+	/**
+	 * Tests worker failed while requesting.
+	 */
+	@Test
+	public void testStartNewWorkerFailedRequesting() throws Exception {
+		new Context() {{
+			final ResourceID tmResourceId = ResourceID.generate();
+			final AtomicInteger requestCount = new AtomicInteger(0);
+
+			final List<CompletableFuture<ResourceID>> resourceIdFutures = new ArrayList<>();
+			resourceIdFutures.add(new CompletableFuture<>());
+			resourceIdFutures.add(new CompletableFuture<>());
+
+			final List<CompletableFuture<TaskExecutorProcessSpec>> requestWorkerFromDriverFutures = new ArrayList<>();
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				int idx = requestCount.getAndIncrement();
+				assertThat(idx, lessThan(2));
+
+				requestWorkerFromDriverFutures.get(idx).complete(taskExecutorProcessSpec);
+				return resourceIdFutures.get(idx);
+			});
+
+			slotManagerBuilder.setGetRequiredResourcesSupplier(() -> Collections.singletonMap(WORKER_RESOURCE_SPEC, 1));
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec1 = requestWorkerFromDriverFutures.get(0).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec1,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// first request failed, verify requesting another worker from driver
+				runInMainThread(() -> resourceIdFutures.get(0).completeExceptionally(new Throwable("testing error")));
+				TaskExecutorProcessSpec taskExecutorProcessSpec2 =
+						requestWorkerFromDriverFutures.get(1).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(taskExecutorProcessSpec2, is(taskExecutorProcessSpec1));
+
+				// second request allocated, verify registration succeed
+				runInMainThread(() -> resourceIdFutures.get(1).complete(tmResourceId));
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(tmResourceId);
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+			});
+		}};
+	}
+
+	/**
+	 * Tests worker terminated after requested before registered.
+	 */
+	@Test
+	public void testWorkerTerminatedBeforeRegister() throws Exception {
+		new Context() {{
+			final AtomicInteger requestCount = new AtomicInteger(0);
+
+			final List<ResourceID> tmResourceIds = new ArrayList<>();
+			tmResourceIds.add(ResourceID.generate());
+			tmResourceIds.add(ResourceID.generate());
+
+			final List<CompletableFuture<TaskExecutorProcessSpec>> requestWorkerFromDriverFutures = new ArrayList<>();
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				int idx = requestCount.getAndIncrement();
+				assertThat(idx, lessThan(2));
+
+				requestWorkerFromDriverFutures.get(idx).complete(taskExecutorProcessSpec);
+				return CompletableFuture.completedFuture(tmResourceIds.get(idx));
+			});
+
+			slotManagerBuilder.setGetRequiredResourcesSupplier(() -> Collections.singletonMap(WORKER_RESOURCE_SPEC, 1));
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec1 = requestWorkerFromDriverFutures.get(0).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec1,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// first worker failed before register, verify requesting another worker from driver
+				runInMainThread(() -> getResourceManager().onWorkerTerminated(tmResourceIds.get(0)));
+				TaskExecutorProcessSpec taskExecutorProcessSpec2 =
+						requestWorkerFromDriverFutures.get(1).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(taskExecutorProcessSpec2, is(taskExecutorProcessSpec1));
+
+				// second worker registered, verify registration succeed
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(tmResourceIds.get(1));
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+			});
+		}};
+	}
+
+	/**
+	 * Tests worker terminated after registered.
+	 */
+	@Test
+	public void testWorkerTerminatedAfterRegister() throws Exception {
+		new Context() {{
+			final AtomicInteger requestCount = new AtomicInteger(0);
+
+			final List<ResourceID> tmResourceIds = new ArrayList<>();
+			tmResourceIds.add(ResourceID.generate());
+			tmResourceIds.add(ResourceID.generate());
+
+			final List<CompletableFuture<TaskExecutorProcessSpec>> requestWorkerFromDriverFutures = new ArrayList<>();
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				int idx = requestCount.getAndIncrement();
+				assertThat(idx, lessThan(2));
+
+				requestWorkerFromDriverFutures.get(idx).complete(taskExecutorProcessSpec);
+				return CompletableFuture.completedFuture(tmResourceIds.get(idx));
+			});
+
+			slotManagerBuilder.setGetRequiredResourcesSupplier(() -> Collections.singletonMap(WORKER_RESOURCE_SPEC, 1));
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec1 = requestWorkerFromDriverFutures.get(0).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec1,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// first worker registered, verify registration succeed
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture1 = registerTaskExecutor(tmResourceIds.get(0));
+				assertThat(registerTaskExecutorFuture1.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+
+				// first worker terminated, verify requesting another worker from driver
+				runInMainThread(() -> getResourceManager().onWorkerTerminated(tmResourceIds.get(0)));
+				TaskExecutorProcessSpec taskExecutorProcessSpec2 =
+						requestWorkerFromDriverFutures.get(1).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(taskExecutorProcessSpec2, is(taskExecutorProcessSpec1));
+
+				// second worker registered, verify registration succeed
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture2 = registerTaskExecutor(tmResourceIds.get(1));
+				assertThat(registerTaskExecutorFuture2.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+			});
+		}};
+	}
+
+	/**
+	 * Tests worker terminated and is no longer required.
+	 */
+	@Test
+	public void testWorkerTerminatedNoLongerRequired() throws Exception {
+		new Context() {{
+			final ResourceID tmResourceId = ResourceID.generate();
+			final AtomicInteger requestCount = new AtomicInteger(0);
+
+			final List<CompletableFuture<TaskExecutorProcessSpec>> requestWorkerFromDriverFutures = new ArrayList<>();
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				int idx = requestCount.getAndIncrement();
+				assertThat(idx, lessThan(2));
+
+				requestWorkerFromDriverFutures.get(idx).complete(taskExecutorProcessSpec);
+				return CompletableFuture.completedFuture(tmResourceId);
+			});
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec = requestWorkerFromDriverFutures.get(0).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// worker registered, verify registration succeed
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(tmResourceId);
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+
+				// worker terminated, verify not requesting new worker
+				runInMainThread(() -> {
+					getResourceManager().onWorkerTerminated(tmResourceId);
+					// needs to return something, so that we can use `get()` to make sure the main thread processing
+					// finishes before the assertions
+					return null;
+				}).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+				assertFalse(requestWorkerFromDriverFutures.get(1).isDone());
+			});
+		}};
+	}
+
+	/**
+	 * Tests workers from previous attempt successfully recovered and registered.
+	 */
+	@Test
+	public void testRecoverWorkerFromPreviousAttempt() throws Exception {
+		new Context() {{
+			final ResourceID tmResourceId = ResourceID.generate();
+
+			runTest(() -> {
+				runInMainThread(() -> getResourceManager().onPreviousAttemptWorkersRecovered(Collections.singleton(tmResourceId)));
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(tmResourceId);
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+			});
+		}};
+	}
+
+	/**
+	 * Tests decline unknown worker registration.
+	 */
+	@Test
+	public void testRegisterUnknownWorker() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(ResourceID.generate());
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Decline.class));
+			});
+		}};
+	}
+
+	@Test
+	public void testOnError() throws Exception {
+		new Context() {{
+			final Throwable fatalError = new Throwable("Testing fatal error");
+			runTest(() -> {
+				runInMainThread(() -> getResourceManager().onError(fatalError));
+				final Throwable reportedError = getFatalErrorHandler().getErrorFuture().get(TIMEOUT_SEC, TimeUnit.SECONDS);
+				assertThat(reportedError, is(fatalError));
+			});
+		}};
+	}
+
+	class Context {
+
+		final Configuration flinkConfig = new Configuration();
+		final TestingResourceManagerDriver.Builder driverBuilder = new TestingResourceManagerDriver.Builder();
+		final TestingSlotManagerBuilder slotManagerBuilder = new TestingSlotManagerBuilder();
+
+		private ActiveResourceManager<ResourceID> resourceManager;
+		private TestingFatalErrorHandler fatalErrorHandler;
+
+		ActiveResourceManager<ResourceID> getResourceManager() {
+			return resourceManager;
+		}
+
+		TestingFatalErrorHandler getFatalErrorHandler() {
+			return fatalErrorHandler;
+		}
+
+		void runTest(RunnableWithException testMethod) throws Exception {
+			fatalErrorHandler = new TestingFatalErrorHandler();
+			resourceManager = createAndStartResourceManager(
+					flinkConfig,
+					driverBuilder.build(),
+					slotManagerBuilder.createSlotManager());
+
+			try {
+				testMethod.run();
+			} finally {
+				resourceManager.close();
+			}
+		}
+
+		private ActiveResourceManager<ResourceID> createAndStartResourceManager(
+				Configuration configuration,
+				ResourceManagerDriver<ResourceID> driver,
+				SlotManager slotManager) throws Exception {
+			final TestingRpcService rpcService = new TestingRpcService(configuration);

Review comment:
       Ok, make sense. I could also add a notice about 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] xintongsong commented on a change in pull request #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/ResourceEventHandler.java
##########
@@ -0,0 +1,60 @@
+/*
+ * 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 java.util.Collection;
+
+/**
+ * Callback interfaces for handling resource events from external resource managers.
+ */
+public interface ResourceEventHandler<WorkerType extends ResourceIDRetrievable> {
+
+	/**
+	 * Notifies that workers of previous attempt have been recovered from the external resource manager.
+	 *
+	 * @param recoveredWorkers Collection of worker nodes, in the deployment specific type.
+	 */
+	void onPreviousAttemptWorkersRecovered(Collection<WorkerType> recoveredWorkers);
+
+	/**
+	 * Notifies that the worker has been terminated.
+	 *
+	 * <p>See also {@link ResourceManagerDriver#requestResource}.
+	 *
+	 * @param resourceId Identifier of the terminated worker.
+	 */
+	void onWorkerTerminated(ResourceID resourceId);
+
+	/**
+	 * Notifies that an error has occurred that the process cannot proceed.
+	 *
+	 * @param exception Exception that describes the error.
+	 */
+	void onError(Throwable exception);
+
+	/**
+	 * Execute given runnable in the rpc main thread.
+	 *
+	 * @param runnable Runnable to be executed.
+	 */
+	void handleInMainThread(Runnable runnable);

Review comment:
       There's probably an easier way to keep the main thread executor on driver side up to date.
   
   We can introduce the following implementation of `Executor` as an inner class of `ActiveResourceManager`, and pass it into `ResourceManagerDriver.initialize()`.
   
   ```
   /**
    * Always execute on the current main thread executor.
    */
   private class GatewayExecutor implements Executor {
   	@Override
   	public void execute(Runnable command) {
   		getMainThreadExecutor().execute(command);
   	}
   }
   ```
   
   This should be equivalent to currently calling `ResourceEventHandler. handleInMainThread()`, while provide better separation of concerns. WDYT?




----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/ResourceManagerDriver.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.ApplicationStatus;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec;
+import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable;
+import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException;
+
+import javax.annotation.Nullable;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * A {@link ResourceManagerDriver} is responsible for requesting and releasing resources from/to a particular external
+ * resource manager.
+ */
+public interface ResourceManagerDriver<WorkerType extends ResourceIDRetrievable> {
+
+	/**
+	 * Initialize the deployment specific components.
+	 *
+	 * @param resourceEventHandler Handler that handles resource events.
+	 */
+	void initialize(ResourceEventHandler<WorkerType> resourceEventHandler) throws Throwable;
+
+	/**
+	 * Terminate the deployment specific components.
+	 *
+	 * @return A future that will be completed successfully when the driver is terminated, or exceptionally if cannot be
+	 * terminated.
+	 */
+	CompletableFuture<Void> terminate();
+
+	/**
+	 * The deployment specific code to deregister the application. This should report the application's final status and
+	 * shut down the resource manager driver cleanly.
+	 *
+	 * <p>This method also needs to make sure all pending containers that are not registered yet are returned.
+	 *
+	 * @param finalStatus The application status to report.
+	 * @param optionalDiagnostics A diagnostics message or {@code null}.
+	 * @throws ResourceManagerException if the application could not be shut down.
+	 */
+	void deregisterApplication(ApplicationStatus finalStatus, @Nullable String optionalDiagnostics) throws Throwable;

Review comment:
       > Or maybe deregisterApplication only deregisters the application but does not shut down the component.
   
   I think that's indeed what `deregisterApplication` does. Basically, after deregistering, Kubernetes / Yarn will terminate the job manager process.
   
   I copied the JavaDoc from the original `ResourceManager#internalDeregisterApplication`. Maybe we should remove " and shut down the resource manager driver cleanly".




----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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



##########
File path: flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java
##########
@@ -349,6 +349,18 @@ public static void tryRethrowException(@Nullable Exception e) throws Exception {
 		}
 	}
 
+	/**
+	 * Tries to throw the given throwable if not null.
+	 *
+	 * @param t throwable to throw if not null
+	 * @throws Throwable
+	 */
+	public static void tryRethrowThrowable(@Nullable Throwable t) throws Throwable {

Review comment:
       We could combine `tryRethrowThrowable(Throwable)` and `tryRethrowException(Exception)`:
   ```suggestion
   	public static <T extends Throwable> void tryRethrowThrowable(@Nullable T t) throws T {
   	 	if (t != null) {
   			throw t;
   		}
   	}
   ```

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/JvmMetaspaceAndOverhead.java
##########
@@ -49,4 +50,16 @@ public MemorySize getMetaspace() {
 	public MemorySize getOverhead() {
 		return overhead;
 	}
+
+	@Override
+	public boolean equals(Object obj) {

Review comment:
       Providing a customized `equals(Object)` usually means that one should also provide a corresponding `hashCode()` implementation. Is the `hashCode()` method missing on purpose?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/jobmanager/JobManagerFlinkMemory.java
##########
@@ -50,7 +53,8 @@
 	private final MemorySize jvmHeap;
 	private final MemorySize offHeapMemory;
 
-	JobManagerFlinkMemory(MemorySize jvmHeap, MemorySize offHeapMemory) {
+	@VisibleForTesting

Review comment:
       Why do we use the `@VisibleForTesting` annotation here? It looks like the constructor is also used in the context of production code.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/jobmanager/JobManagerFlinkMemory.java
##########
@@ -69,4 +73,16 @@ public MemorySize getJvmDirectMemorySize() {
 	public MemorySize getTotalFlinkMemorySize() {
 		return jvmHeap.add(offHeapMemory);
 	}
+
+	@Override
+	public boolean equals(Object obj) {

Review comment:
       Providing a customized `equals(Object)` usually means that one should also provide a corresponding `hashCode()` implementation. Is the `hashCode()` method missing on purpose?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/CommonProcessMemorySpec.java
##########
@@ -94,4 +96,16 @@ public MemorySize getTotalFlinkMemorySize() {
 	public MemorySize getTotalProcessMemorySize() {
 		return flinkMemory.getTotalFlinkMemorySize().add(getJvmMetaspaceSize()).add(getJvmOverheadSize());
 	}
+
+	@Override
+	public boolean equals(Object obj) {

Review comment:
       Providing a customized `equals(Object)` usually means that one should also provide a corresponding `hashCode()` implementation. Is the `hashCode()` method missing on purpose?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/taskmanager/TaskExecutorFlinkMemory.java
##########
@@ -128,4 +130,20 @@ public MemorySize getJvmDirectMemorySize() {
 	public MemorySize getTotalFlinkMemorySize() {
 		return frameworkHeap.add(frameworkOffHeap).add(taskHeap).add(taskOffHeap).add(network).add(managed);
 	}
+
+	@Override
+	public boolean equals(Object obj) {

Review comment:
       Providing a customized `equals(Object)` usually means that one should also provide a corresponding `hashCode()` implementation. Is the `hashCode()` method missing on purpose?




----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/jobmanager/JobManagerFlinkMemory.java
##########
@@ -50,7 +53,8 @@
 	private final MemorySize jvmHeap;
 	private final MemorySize offHeapMemory;
 
-	JobManagerFlinkMemory(MemorySize jvmHeap, MemorySize offHeapMemory) {
+	@VisibleForTesting

Review comment:
       Exactly.
   The constructor was package privacy. I made it public so it can be called in the constructor of `JobManagerProcessSpec`. The later is for test purpose only.




----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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


   Thank you both again. I'll merge this once travis gives green light.


----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/ResourceEventHandler.java
##########
@@ -0,0 +1,60 @@
+/*
+ * 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 java.util.Collection;
+
+/**
+ * Callback interfaces for handling resource events from external resource managers.
+ */
+public interface ResourceEventHandler<WorkerType extends ResourceIDRetrievable> {
+
+	/**
+	 * Notifies that workers of previous attempt have been recovered from the external resource manager.
+	 *
+	 * @param recoveredWorkers Collection of worker nodes, in the deployment specific type.
+	 */
+	void onPreviousAttemptWorkersRecovered(Collection<WorkerType> recoveredWorkers);
+
+	/**
+	 * Notifies that the worker has been terminated.
+	 *
+	 * <p>See also {@link ResourceManagerDriver#requestResource}.
+	 *
+	 * @param resourceId Identifier of the terminated worker.
+	 */
+	void onWorkerTerminated(ResourceID resourceId);
+
+	/**
+	 * Notifies that an error has occurred that the process cannot proceed.
+	 *
+	 * @param exception Exception that describes the error.
+	 */
+	void onError(Throwable exception);
+
+	/**
+	 * Execute given runnable in the rpc main thread.
+	 *
+	 * @param runnable Runnable to be executed.
+	 */
+	void handleInMainThread(Runnable runnable);

Review comment:
       There's probably an easier way to keep the main thread executor on driver side up to date.
   
   We can introduce the following implementation of `Executor` as an inner class of `ActiveResourceManager`, and pass it into `ResourceManagerDriver.initialize()`.
   
   ```
   /**
    * Always execute on the current main thread executor.
    */
   private class GatewayExecutor implements Executor {
   	@Override
   	public void execute(Runnable command) {
   		getMainThreadExecutor().execute(command);
   	}
   }
   ```
   
   This should be equivalent to currently calling `ResourceEventHandler.handleInMainThread()`, while achieving better separation of concerns. WDYT?




----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/active/ActiveResourceManagerTest.java
##########
@@ -0,0 +1,428 @@
+/*
+ * 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.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.entrypoint.ClusterInformation;
+import org.apache.flink.runtime.instance.HardwareDescription;
+import org.apache.flink.runtime.io.network.partition.NoOpResourceManagerPartitionTracker;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.registration.RegistrationResponse;
+import org.apache.flink.runtime.resourcemanager.TaskExecutorRegistration;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager;
+import org.apache.flink.runtime.resourcemanager.slotmanager.TestingSlotManagerBuilder;
+import org.apache.flink.runtime.resourcemanager.utils.MockResourceManagerRuntimeServices;
+import org.apache.flink.runtime.rpc.TestingRpcService;
+import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.runtime.util.TestingFatalErrorHandler;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.hamcrest.Matchers.instanceOf;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.lessThan;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for {@link ActiveResourceManager}.
+ */
+public class ActiveResourceManagerTest extends TestLogger {
+
+	private static final long TIMEOUT_SEC = 5L;
+	private static final Time TIMEOUT_TIME = Time.seconds(TIMEOUT_SEC);
+
+	private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = WorkerResourceSpec.ZERO;
+
+	/**
+	 * Tests worker successfully requested, started and registered.
+	 */
+	@Test
+	public void testStartNewWorker() throws Exception {
+		new Context() {{
+			final ResourceID tmResourceId = ResourceID.generate();
+			final CompletableFuture<TaskExecutorProcessSpec> requestWorkerFromDriverFuture = new CompletableFuture<>();
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				requestWorkerFromDriverFuture.complete(taskExecutorProcessSpec);
+				return CompletableFuture.completedFuture(tmResourceId);
+			});
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec = requestWorkerFromDriverFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// worker registered, verify registration succeed
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(tmResourceId);
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+			});
+		}};
+	}
+
+	/**
+	 * Tests worker failed while requesting.
+	 */
+	@Test
+	public void testStartNewWorkerFailedRequesting() throws Exception {
+		new Context() {{
+			final ResourceID tmResourceId = ResourceID.generate();
+			final AtomicInteger requestCount = new AtomicInteger(0);
+
+			final List<CompletableFuture<ResourceID>> resourceIdFutures = new ArrayList<>();
+			resourceIdFutures.add(new CompletableFuture<>());
+			resourceIdFutures.add(new CompletableFuture<>());
+
+			final List<CompletableFuture<TaskExecutorProcessSpec>> requestWorkerFromDriverFutures = new ArrayList<>();
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				int idx = requestCount.getAndIncrement();
+				assertThat(idx, lessThan(2));
+
+				requestWorkerFromDriverFutures.get(idx).complete(taskExecutorProcessSpec);
+				return resourceIdFutures.get(idx);
+			});
+
+			slotManagerBuilder.setGetRequiredResourcesSupplier(() -> Collections.singletonMap(WORKER_RESOURCE_SPEC, 1));
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec1 = requestWorkerFromDriverFutures.get(0).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec1,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// first request failed, verify requesting another worker from driver
+				runInMainThread(() -> resourceIdFutures.get(0).completeExceptionally(new Throwable("testing error")));
+				TaskExecutorProcessSpec taskExecutorProcessSpec2 =
+						requestWorkerFromDriverFutures.get(1).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(taskExecutorProcessSpec2, is(taskExecutorProcessSpec1));
+
+				// second request allocated, verify registration succeed
+				runInMainThread(() -> resourceIdFutures.get(1).complete(tmResourceId));
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(tmResourceId);
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+			});
+		}};
+	}
+
+	/**
+	 * Tests worker terminated after requested before registered.
+	 */
+	@Test
+	public void testWorkerTerminatedBeforeRegister() throws Exception {
+		new Context() {{
+			final AtomicInteger requestCount = new AtomicInteger(0);
+
+			final List<ResourceID> tmResourceIds = new ArrayList<>();
+			tmResourceIds.add(ResourceID.generate());
+			tmResourceIds.add(ResourceID.generate());
+
+			final List<CompletableFuture<TaskExecutorProcessSpec>> requestWorkerFromDriverFutures = new ArrayList<>();
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				int idx = requestCount.getAndIncrement();
+				assertThat(idx, lessThan(2));
+
+				requestWorkerFromDriverFutures.get(idx).complete(taskExecutorProcessSpec);
+				return CompletableFuture.completedFuture(tmResourceIds.get(idx));
+			});
+
+			slotManagerBuilder.setGetRequiredResourcesSupplier(() -> Collections.singletonMap(WORKER_RESOURCE_SPEC, 1));
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec1 = requestWorkerFromDriverFutures.get(0).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec1,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// first worker failed before register, verify requesting another worker from driver
+				runInMainThread(() -> getResourceManager().onWorkerTerminated(tmResourceIds.get(0)));
+				TaskExecutorProcessSpec taskExecutorProcessSpec2 =
+						requestWorkerFromDriverFutures.get(1).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(taskExecutorProcessSpec2, is(taskExecutorProcessSpec1));
+
+				// second worker registered, verify registration succeed
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(tmResourceIds.get(1));
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+			});
+		}};
+	}
+
+	/**
+	 * Tests worker terminated after registered.
+	 */
+	@Test
+	public void testWorkerTerminatedAfterRegister() throws Exception {
+		new Context() {{
+			final AtomicInteger requestCount = new AtomicInteger(0);
+
+			final List<ResourceID> tmResourceIds = new ArrayList<>();
+			tmResourceIds.add(ResourceID.generate());
+			tmResourceIds.add(ResourceID.generate());
+
+			final List<CompletableFuture<TaskExecutorProcessSpec>> requestWorkerFromDriverFutures = new ArrayList<>();
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				int idx = requestCount.getAndIncrement();
+				assertThat(idx, lessThan(2));
+
+				requestWorkerFromDriverFutures.get(idx).complete(taskExecutorProcessSpec);
+				return CompletableFuture.completedFuture(tmResourceIds.get(idx));
+			});
+
+			slotManagerBuilder.setGetRequiredResourcesSupplier(() -> Collections.singletonMap(WORKER_RESOURCE_SPEC, 1));
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec1 = requestWorkerFromDriverFutures.get(0).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec1,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// first worker registered, verify registration succeed
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture1 = registerTaskExecutor(tmResourceIds.get(0));
+				assertThat(registerTaskExecutorFuture1.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+
+				// first worker terminated, verify requesting another worker from driver
+				runInMainThread(() -> getResourceManager().onWorkerTerminated(tmResourceIds.get(0)));
+				TaskExecutorProcessSpec taskExecutorProcessSpec2 =
+						requestWorkerFromDriverFutures.get(1).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(taskExecutorProcessSpec2, is(taskExecutorProcessSpec1));
+
+				// second worker registered, verify registration succeed
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture2 = registerTaskExecutor(tmResourceIds.get(1));
+				assertThat(registerTaskExecutorFuture2.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+			});
+		}};
+	}
+
+	/**
+	 * Tests worker terminated and is no longer required.
+	 */
+	@Test
+	public void testWorkerTerminatedNoLongerRequired() throws Exception {
+		new Context() {{
+			final ResourceID tmResourceId = ResourceID.generate();
+			final AtomicInteger requestCount = new AtomicInteger(0);
+
+			final List<CompletableFuture<TaskExecutorProcessSpec>> requestWorkerFromDriverFutures = new ArrayList<>();
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				int idx = requestCount.getAndIncrement();
+				assertThat(idx, lessThan(2));
+
+				requestWorkerFromDriverFutures.get(idx).complete(taskExecutorProcessSpec);
+				return CompletableFuture.completedFuture(tmResourceId);
+			});
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec = requestWorkerFromDriverFutures.get(0).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// worker registered, verify registration succeed
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(tmResourceId);
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+
+				// worker terminated, verify not requesting new worker
+				runInMainThread(() -> {
+					getResourceManager().onWorkerTerminated(tmResourceId);
+					// needs to return something, so that we can use `get()` to make sure the main thread processing
+					// finishes before the assertions
+					return null;
+				}).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+				assertFalse(requestWorkerFromDriverFutures.get(1).isDone());
+			});
+		}};
+	}
+
+	/**
+	 * Tests workers from previous attempt successfully recovered and registered.
+	 */
+	@Test
+	public void testRecoverWorkerFromPreviousAttempt() throws Exception {
+		new Context() {{
+			final ResourceID tmResourceId = ResourceID.generate();
+
+			runTest(() -> {
+				runInMainThread(() -> getResourceManager().onPreviousAttemptWorkersRecovered(Collections.singleton(tmResourceId)));
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(tmResourceId);
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+			});
+		}};
+	}
+
+	/**
+	 * Tests decline unknown worker registration.
+	 */
+	@Test
+	public void testRegisterUnknownWorker() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(ResourceID.generate());
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Decline.class));
+			});
+		}};
+	}
+
+	@Test
+	public void testOnError() throws Exception {
+		new Context() {{
+			final Throwable fatalError = new Throwable("Testing fatal error");
+			runTest(() -> {
+				runInMainThread(() -> getResourceManager().onError(fatalError));
+				final Throwable reportedError = getFatalErrorHandler().getErrorFuture().get(TIMEOUT_SEC, TimeUnit.SECONDS);
+				assertThat(reportedError, is(fatalError));
+			});
+		}};
+	}
+
+	class Context {
+
+		final Configuration flinkConfig = new Configuration();
+		final TestingResourceManagerDriver.Builder driverBuilder = new TestingResourceManagerDriver.Builder();
+		final TestingSlotManagerBuilder slotManagerBuilder = new TestingSlotManagerBuilder();
+
+		private ActiveResourceManager<ResourceID> resourceManager;
+		private TestingFatalErrorHandler fatalErrorHandler;
+
+		ActiveResourceManager<ResourceID> getResourceManager() {
+			return resourceManager;
+		}
+
+		TestingFatalErrorHandler getFatalErrorHandler() {
+			return fatalErrorHandler;
+		}
+
+		void runTest(RunnableWithException testMethod) throws Exception {
+			fatalErrorHandler = new TestingFatalErrorHandler();
+			resourceManager = createAndStartResourceManager(
+					flinkConfig,
+					driverBuilder.build(),
+					slotManagerBuilder.createSlotManager());
+
+			try {
+				testMethod.run();
+			} finally {
+				resourceManager.close();
+			}
+		}
+
+		private ActiveResourceManager<ResourceID> createAndStartResourceManager(
+				Configuration configuration,
+				ResourceManagerDriver<ResourceID> driver,
+				SlotManager slotManager) throws Exception {
+			final TestingRpcService rpcService = new TestingRpcService(configuration);

Review comment:
       Hmm, do any of the tests configure something `RpcService` related? If a new test should require this, then one would probably create a separate test class since all other tests can work with the same `RpcService`.




----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4957",
       "triggerID" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5404",
       "triggerID" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5458363056979d6be8594af4ac469c57ee8f9b15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5409",
       "triggerID" : "5458363056979d6be8594af4ac469c57ee8f9b15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "672548350",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5428",
       "triggerID" : "672548350",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5428",
       "triggerID" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   *  Unknown: [CANCELED](TBD) 
   * f1bfc953a17f24a74ac3da5db2d9dd790f90de8c Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5428) 
   
   <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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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


   Thanks for the review, @XComp & @tillrohrmann.
   I have rebased onto master branch, and addressed comments in the fixup commits.
   There's also a revert commit, since we no longer need `ConsumerWithException`. This is kept for the purpose of highlighting new changes, and should be removed on merging together with the commit it reverts.


----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/active/ActiveResourceManagerTest.java
##########
@@ -0,0 +1,428 @@
+/*
+ * 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.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.entrypoint.ClusterInformation;
+import org.apache.flink.runtime.instance.HardwareDescription;
+import org.apache.flink.runtime.io.network.partition.NoOpResourceManagerPartitionTracker;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.registration.RegistrationResponse;
+import org.apache.flink.runtime.resourcemanager.TaskExecutorRegistration;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager;
+import org.apache.flink.runtime.resourcemanager.slotmanager.TestingSlotManagerBuilder;
+import org.apache.flink.runtime.resourcemanager.utils.MockResourceManagerRuntimeServices;
+import org.apache.flink.runtime.rpc.TestingRpcService;
+import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.runtime.util.TestingFatalErrorHandler;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.hamcrest.Matchers.instanceOf;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.lessThan;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for {@link ActiveResourceManager}.
+ */
+public class ActiveResourceManagerTest extends TestLogger {
+
+	private static final long TIMEOUT_SEC = 5L;
+	private static final Time TIMEOUT_TIME = Time.seconds(TIMEOUT_SEC);
+
+	private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = WorkerResourceSpec.ZERO;
+
+	/**
+	 * Tests worker successfully requested, started and registered.
+	 */
+	@Test
+	public void testStartNewWorker() throws Exception {
+		new Context() {{
+			final ResourceID tmResourceId = ResourceID.generate();
+			final CompletableFuture<TaskExecutorProcessSpec> requestWorkerFromDriverFuture = new CompletableFuture<>();
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				requestWorkerFromDriverFuture.complete(taskExecutorProcessSpec);
+				return CompletableFuture.completedFuture(tmResourceId);
+			});
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec = requestWorkerFromDriverFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// worker registered, verify registration succeed
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(tmResourceId);
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+			});
+		}};
+	}
+
+	/**
+	 * Tests worker failed while requesting.
+	 */
+	@Test
+	public void testStartNewWorkerFailedRequesting() throws Exception {
+		new Context() {{
+			final ResourceID tmResourceId = ResourceID.generate();
+			final AtomicInteger requestCount = new AtomicInteger(0);
+
+			final List<CompletableFuture<ResourceID>> resourceIdFutures = new ArrayList<>();
+			resourceIdFutures.add(new CompletableFuture<>());
+			resourceIdFutures.add(new CompletableFuture<>());
+
+			final List<CompletableFuture<TaskExecutorProcessSpec>> requestWorkerFromDriverFutures = new ArrayList<>();
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				int idx = requestCount.getAndIncrement();
+				assertThat(idx, lessThan(2));
+
+				requestWorkerFromDriverFutures.get(idx).complete(taskExecutorProcessSpec);
+				return resourceIdFutures.get(idx);
+			});
+
+			slotManagerBuilder.setGetRequiredResourcesSupplier(() -> Collections.singletonMap(WORKER_RESOURCE_SPEC, 1));
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec1 = requestWorkerFromDriverFutures.get(0).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec1,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// first request failed, verify requesting another worker from driver
+				runInMainThread(() -> resourceIdFutures.get(0).completeExceptionally(new Throwable("testing error")));
+				TaskExecutorProcessSpec taskExecutorProcessSpec2 =
+						requestWorkerFromDriverFutures.get(1).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(taskExecutorProcessSpec2, is(taskExecutorProcessSpec1));
+
+				// second request allocated, verify registration succeed
+				runInMainThread(() -> resourceIdFutures.get(1).complete(tmResourceId));
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(tmResourceId);
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+			});
+		}};
+	}
+
+	/**
+	 * Tests worker terminated after requested before registered.
+	 */
+	@Test
+	public void testWorkerTerminatedBeforeRegister() throws Exception {
+		new Context() {{
+			final AtomicInteger requestCount = new AtomicInteger(0);
+
+			final List<ResourceID> tmResourceIds = new ArrayList<>();
+			tmResourceIds.add(ResourceID.generate());
+			tmResourceIds.add(ResourceID.generate());
+
+			final List<CompletableFuture<TaskExecutorProcessSpec>> requestWorkerFromDriverFutures = new ArrayList<>();
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				int idx = requestCount.getAndIncrement();
+				assertThat(idx, lessThan(2));
+
+				requestWorkerFromDriverFutures.get(idx).complete(taskExecutorProcessSpec);
+				return CompletableFuture.completedFuture(tmResourceIds.get(idx));
+			});
+
+			slotManagerBuilder.setGetRequiredResourcesSupplier(() -> Collections.singletonMap(WORKER_RESOURCE_SPEC, 1));
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec1 = requestWorkerFromDriverFutures.get(0).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec1,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// first worker failed before register, verify requesting another worker from driver
+				runInMainThread(() -> getResourceManager().onWorkerTerminated(tmResourceIds.get(0)));
+				TaskExecutorProcessSpec taskExecutorProcessSpec2 =
+						requestWorkerFromDriverFutures.get(1).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(taskExecutorProcessSpec2, is(taskExecutorProcessSpec1));
+
+				// second worker registered, verify registration succeed
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(tmResourceIds.get(1));
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+			});
+		}};
+	}
+
+	/**
+	 * Tests worker terminated after registered.
+	 */
+	@Test
+	public void testWorkerTerminatedAfterRegister() throws Exception {
+		new Context() {{
+			final AtomicInteger requestCount = new AtomicInteger(0);
+
+			final List<ResourceID> tmResourceIds = new ArrayList<>();
+			tmResourceIds.add(ResourceID.generate());
+			tmResourceIds.add(ResourceID.generate());
+
+			final List<CompletableFuture<TaskExecutorProcessSpec>> requestWorkerFromDriverFutures = new ArrayList<>();
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				int idx = requestCount.getAndIncrement();
+				assertThat(idx, lessThan(2));
+
+				requestWorkerFromDriverFutures.get(idx).complete(taskExecutorProcessSpec);
+				return CompletableFuture.completedFuture(tmResourceIds.get(idx));
+			});
+
+			slotManagerBuilder.setGetRequiredResourcesSupplier(() -> Collections.singletonMap(WORKER_RESOURCE_SPEC, 1));
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec1 = requestWorkerFromDriverFutures.get(0).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec1,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// first worker registered, verify registration succeed
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture1 = registerTaskExecutor(tmResourceIds.get(0));
+				assertThat(registerTaskExecutorFuture1.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+
+				// first worker terminated, verify requesting another worker from driver
+				runInMainThread(() -> getResourceManager().onWorkerTerminated(tmResourceIds.get(0)));
+				TaskExecutorProcessSpec taskExecutorProcessSpec2 =
+						requestWorkerFromDriverFutures.get(1).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(taskExecutorProcessSpec2, is(taskExecutorProcessSpec1));
+
+				// second worker registered, verify registration succeed
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture2 = registerTaskExecutor(tmResourceIds.get(1));
+				assertThat(registerTaskExecutorFuture2.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+			});
+		}};
+	}
+
+	/**
+	 * Tests worker terminated and is no longer required.
+	 */
+	@Test
+	public void testWorkerTerminatedNoLongerRequired() throws Exception {
+		new Context() {{
+			final ResourceID tmResourceId = ResourceID.generate();
+			final AtomicInteger requestCount = new AtomicInteger(0);
+
+			final List<CompletableFuture<TaskExecutorProcessSpec>> requestWorkerFromDriverFutures = new ArrayList<>();
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				int idx = requestCount.getAndIncrement();
+				assertThat(idx, lessThan(2));
+
+				requestWorkerFromDriverFutures.get(idx).complete(taskExecutorProcessSpec);
+				return CompletableFuture.completedFuture(tmResourceId);
+			});
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec = requestWorkerFromDriverFutures.get(0).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// worker registered, verify registration succeed
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(tmResourceId);
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+
+				// worker terminated, verify not requesting new worker
+				runInMainThread(() -> {
+					getResourceManager().onWorkerTerminated(tmResourceId);
+					// needs to return something, so that we can use `get()` to make sure the main thread processing
+					// finishes before the assertions
+					return null;
+				}).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+				assertFalse(requestWorkerFromDriverFutures.get(1).isDone());
+			});
+		}};
+	}
+
+	/**
+	 * Tests workers from previous attempt successfully recovered and registered.
+	 */
+	@Test
+	public void testRecoverWorkerFromPreviousAttempt() throws Exception {
+		new Context() {{
+			final ResourceID tmResourceId = ResourceID.generate();
+
+			runTest(() -> {
+				runInMainThread(() -> getResourceManager().onPreviousAttemptWorkersRecovered(Collections.singleton(tmResourceId)));
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(tmResourceId);
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+			});
+		}};
+	}
+
+	/**
+	 * Tests decline unknown worker registration.
+	 */
+	@Test
+	public void testRegisterUnknownWorker() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(ResourceID.generate());
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Decline.class));
+			});
+		}};
+	}
+
+	@Test
+	public void testOnError() throws Exception {
+		new Context() {{
+			final Throwable fatalError = new Throwable("Testing fatal error");
+			runTest(() -> {
+				runInMainThread(() -> getResourceManager().onError(fatalError));
+				final Throwable reportedError = getFatalErrorHandler().getErrorFuture().get(TIMEOUT_SEC, TimeUnit.SECONDS);
+				assertThat(reportedError, is(fatalError));
+			});
+		}};
+	}
+
+	class Context {
+
+		final Configuration flinkConfig = new Configuration();
+		final TestingResourceManagerDriver.Builder driverBuilder = new TestingResourceManagerDriver.Builder();
+		final TestingSlotManagerBuilder slotManagerBuilder = new TestingSlotManagerBuilder();
+
+		private ActiveResourceManager<ResourceID> resourceManager;
+		private TestingFatalErrorHandler fatalErrorHandler;
+
+		ActiveResourceManager<ResourceID> getResourceManager() {
+			return resourceManager;
+		}
+
+		TestingFatalErrorHandler getFatalErrorHandler() {
+			return fatalErrorHandler;
+		}
+
+		void runTest(RunnableWithException testMethod) throws Exception {
+			fatalErrorHandler = new TestingFatalErrorHandler();
+			resourceManager = createAndStartResourceManager(
+					flinkConfig,
+					driverBuilder.build(),
+					slotManagerBuilder.createSlotManager());
+
+			try {
+				testMethod.run();
+			} finally {
+				resourceManager.close();
+			}
+		}
+
+		private ActiveResourceManager<ResourceID> createAndStartResourceManager(
+				Configuration configuration,
+				ResourceManagerDriver<ResourceID> driver,
+				SlotManager slotManager) throws Exception {
+			final TestingRpcService rpcService = new TestingRpcService(configuration);

Review comment:
       The `configuration` used for creating `TestingRpcService` might be different for test cases.
   It's probably not a problem, since the chance that `configuration` contains something affects the `TestingRpcService` is small. But I would still like to make it save.




----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4957",
       "triggerID" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5404",
       "triggerID" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5458363056979d6be8594af4ac469c57ee8f9b15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5409",
       "triggerID" : "5458363056979d6be8594af4ac469c57ee8f9b15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5428",
       "triggerID" : "672548350",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5428",
       "triggerID" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "672548350",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "4847c551d75f244418947031dfe8b53bcf34073e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5449",
       "triggerID" : "4847c551d75f244418947031dfe8b53bcf34073e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "afc7db9e33b3996dd6b859b1ed1e9c418a7a0ef0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5502",
       "triggerID" : "afc7db9e33b3996dd6b859b1ed1e9c418a7a0ef0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * afc7db9e33b3996dd6b859b1ed1e9c418a7a0ef0 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5502) 
   
   <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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/CommonProcessMemorySpec.java
##########
@@ -94,4 +96,16 @@ public MemorySize getTotalFlinkMemorySize() {
 	public MemorySize getTotalProcessMemorySize() {
 		return flinkMemory.getTotalFlinkMemorySize().add(getJvmMetaspaceSize()).add(getJvmOverheadSize());
 	}
+
+	@Override
+	public boolean equals(Object obj) {

Review comment:
       If it is for the tests only, then one could also think about providing a Hamcrest matcher for this task. On the other side, this class is a value class with a well-defined equality relation.




----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/ResourceEventHandler.java
##########
@@ -0,0 +1,60 @@
+/*
+ * 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 java.util.Collection;
+
+/**
+ * Callback interfaces for handling resource events from external resource managers.
+ */
+public interface ResourceEventHandler<WorkerType extends ResourceIDRetrievable> {
+
+	/**
+	 * Notifies that workers of previous attempt have been recovered from the external resource manager.
+	 *
+	 * @param recoveredWorkers Collection of worker nodes, in the deployment specific type.
+	 */
+	void onPreviousAttemptWorkersRecovered(Collection<WorkerType> recoveredWorkers);
+
+	/**
+	 * Notifies that the worker has been terminated.
+	 *
+	 * <p>See also {@link ResourceManagerDriver#requestResource}.
+	 *
+	 * @param resourceId Identifier of the terminated worker.
+	 */
+	void onWorkerTerminated(ResourceID resourceId);
+
+	/**
+	 * Notifies that an error has occurred that the process cannot proceed.
+	 *
+	 * @param exception Exception that describes the error.
+	 */
+	void onError(Throwable exception);
+
+	/**
+	 * Execute given runnable in the rpc main thread.
+	 *
+	 * @param runnable Runnable to be executed.
+	 */
+	void handleInMainThread(Runnable runnable);

Review comment:
       In this case, one could update the `ResourceManagerDriver` whenever the `ResourceManager` obtains the leadership.
   
   On a side note, I believe that we should actually not reuse the `ResourceManager` across leader sessions and instead create a new one if the RM gets the leadership (similar to what we do with the `Dispatcher` nowadays).




----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/active/ActiveResourceManagerTest.java
##########
@@ -0,0 +1,428 @@
+/*
+ * 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.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.entrypoint.ClusterInformation;
+import org.apache.flink.runtime.instance.HardwareDescription;
+import org.apache.flink.runtime.io.network.partition.NoOpResourceManagerPartitionTracker;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.registration.RegistrationResponse;
+import org.apache.flink.runtime.resourcemanager.TaskExecutorRegistration;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager;
+import org.apache.flink.runtime.resourcemanager.slotmanager.TestingSlotManagerBuilder;
+import org.apache.flink.runtime.resourcemanager.utils.MockResourceManagerRuntimeServices;
+import org.apache.flink.runtime.rpc.TestingRpcService;
+import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.runtime.util.TestingFatalErrorHandler;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.hamcrest.Matchers.instanceOf;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.lessThan;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for {@link ActiveResourceManager}.
+ */
+public class ActiveResourceManagerTest extends TestLogger {
+
+	private static final long TIMEOUT_SEC = 5L;
+	private static final Time TIMEOUT_TIME = Time.seconds(TIMEOUT_SEC);
+
+	private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = WorkerResourceSpec.ZERO;
+
+	/**
+	 * Tests worker successfully requested, started and registered.
+	 */
+	@Test
+	public void testStartNewWorker() throws Exception {
+		new Context() {{
+			final ResourceID tmResourceId = ResourceID.generate();
+			final CompletableFuture<TaskExecutorProcessSpec> requestWorkerFromDriverFuture = new CompletableFuture<>();
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				requestWorkerFromDriverFuture.complete(taskExecutorProcessSpec);
+				return CompletableFuture.completedFuture(tmResourceId);
+			});
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec = requestWorkerFromDriverFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// worker registered, verify registration succeed
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(tmResourceId);
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+			});
+		}};
+	}
+
+	/**
+	 * Tests worker failed while requesting.
+	 */
+	@Test
+	public void testStartNewWorkerFailedRequesting() throws Exception {
+		new Context() {{
+			final ResourceID tmResourceId = ResourceID.generate();
+			final AtomicInteger requestCount = new AtomicInteger(0);
+
+			final List<CompletableFuture<ResourceID>> resourceIdFutures = new ArrayList<>();
+			resourceIdFutures.add(new CompletableFuture<>());
+			resourceIdFutures.add(new CompletableFuture<>());
+
+			final List<CompletableFuture<TaskExecutorProcessSpec>> requestWorkerFromDriverFutures = new ArrayList<>();
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				int idx = requestCount.getAndIncrement();
+				assertThat(idx, lessThan(2));
+
+				requestWorkerFromDriverFutures.get(idx).complete(taskExecutorProcessSpec);
+				return resourceIdFutures.get(idx);
+			});
+
+			slotManagerBuilder.setGetRequiredResourcesSupplier(() -> Collections.singletonMap(WORKER_RESOURCE_SPEC, 1));
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec1 = requestWorkerFromDriverFutures.get(0).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec1,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// first request failed, verify requesting another worker from driver
+				runInMainThread(() -> resourceIdFutures.get(0).completeExceptionally(new Throwable("testing error")));
+				TaskExecutorProcessSpec taskExecutorProcessSpec2 =
+						requestWorkerFromDriverFutures.get(1).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(taskExecutorProcessSpec2, is(taskExecutorProcessSpec1));
+
+				// second request allocated, verify registration succeed
+				runInMainThread(() -> resourceIdFutures.get(1).complete(tmResourceId));
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(tmResourceId);
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+			});
+		}};
+	}
+
+	/**
+	 * Tests worker terminated after requested before registered.
+	 */
+	@Test
+	public void testWorkerTerminatedBeforeRegister() throws Exception {
+		new Context() {{
+			final AtomicInteger requestCount = new AtomicInteger(0);
+
+			final List<ResourceID> tmResourceIds = new ArrayList<>();
+			tmResourceIds.add(ResourceID.generate());
+			tmResourceIds.add(ResourceID.generate());
+
+			final List<CompletableFuture<TaskExecutorProcessSpec>> requestWorkerFromDriverFutures = new ArrayList<>();
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				int idx = requestCount.getAndIncrement();
+				assertThat(idx, lessThan(2));
+
+				requestWorkerFromDriverFutures.get(idx).complete(taskExecutorProcessSpec);
+				return CompletableFuture.completedFuture(tmResourceIds.get(idx));
+			});
+
+			slotManagerBuilder.setGetRequiredResourcesSupplier(() -> Collections.singletonMap(WORKER_RESOURCE_SPEC, 1));
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec1 = requestWorkerFromDriverFutures.get(0).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec1,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// first worker failed before register, verify requesting another worker from driver
+				runInMainThread(() -> getResourceManager().onWorkerTerminated(tmResourceIds.get(0)));
+				TaskExecutorProcessSpec taskExecutorProcessSpec2 =
+						requestWorkerFromDriverFutures.get(1).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(taskExecutorProcessSpec2, is(taskExecutorProcessSpec1));
+
+				// second worker registered, verify registration succeed
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(tmResourceIds.get(1));
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+			});
+		}};
+	}
+
+	/**
+	 * Tests worker terminated after registered.
+	 */
+	@Test
+	public void testWorkerTerminatedAfterRegister() throws Exception {
+		new Context() {{
+			final AtomicInteger requestCount = new AtomicInteger(0);
+
+			final List<ResourceID> tmResourceIds = new ArrayList<>();
+			tmResourceIds.add(ResourceID.generate());
+			tmResourceIds.add(ResourceID.generate());
+
+			final List<CompletableFuture<TaskExecutorProcessSpec>> requestWorkerFromDriverFutures = new ArrayList<>();
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				int idx = requestCount.getAndIncrement();
+				assertThat(idx, lessThan(2));
+
+				requestWorkerFromDriverFutures.get(idx).complete(taskExecutorProcessSpec);
+				return CompletableFuture.completedFuture(tmResourceIds.get(idx));
+			});
+
+			slotManagerBuilder.setGetRequiredResourcesSupplier(() -> Collections.singletonMap(WORKER_RESOURCE_SPEC, 1));
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec1 = requestWorkerFromDriverFutures.get(0).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec1,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// first worker registered, verify registration succeed
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture1 = registerTaskExecutor(tmResourceIds.get(0));
+				assertThat(registerTaskExecutorFuture1.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+
+				// first worker terminated, verify requesting another worker from driver
+				runInMainThread(() -> getResourceManager().onWorkerTerminated(tmResourceIds.get(0)));
+				TaskExecutorProcessSpec taskExecutorProcessSpec2 =
+						requestWorkerFromDriverFutures.get(1).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(taskExecutorProcessSpec2, is(taskExecutorProcessSpec1));
+
+				// second worker registered, verify registration succeed
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture2 = registerTaskExecutor(tmResourceIds.get(1));
+				assertThat(registerTaskExecutorFuture2.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+			});
+		}};
+	}
+
+	/**
+	 * Tests worker terminated and is no longer required.
+	 */
+	@Test
+	public void testWorkerTerminatedNoLongerRequired() throws Exception {
+		new Context() {{
+			final ResourceID tmResourceId = ResourceID.generate();
+			final AtomicInteger requestCount = new AtomicInteger(0);
+
+			final List<CompletableFuture<TaskExecutorProcessSpec>> requestWorkerFromDriverFutures = new ArrayList<>();
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+			requestWorkerFromDriverFutures.add(new CompletableFuture<>());
+
+			driverBuilder.setRequestResourceFunction(taskExecutorProcessSpec -> {
+				int idx = requestCount.getAndIncrement();
+				assertThat(idx, lessThan(2));
+
+				requestWorkerFromDriverFutures.get(idx).complete(taskExecutorProcessSpec);
+				return CompletableFuture.completedFuture(tmResourceId);
+			});
+
+			runTest(() -> {
+				// received worker request, verify requesting from driver
+				CompletableFuture<Boolean> startNewWorkerFuture = runInMainThread(() ->
+						getResourceManager().startNewWorker(WORKER_RESOURCE_SPEC));
+				TaskExecutorProcessSpec taskExecutorProcessSpec = requestWorkerFromDriverFutures.get(0).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+
+				assertThat(startNewWorkerFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), is(true));
+				assertThat(taskExecutorProcessSpec,
+						is(TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, WORKER_RESOURCE_SPEC)));
+
+				// worker registered, verify registration succeed
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(tmResourceId);
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+
+				// worker terminated, verify not requesting new worker
+				runInMainThread(() -> {
+					getResourceManager().onWorkerTerminated(tmResourceId);
+					// needs to return something, so that we can use `get()` to make sure the main thread processing
+					// finishes before the assertions
+					return null;
+				}).get(TIMEOUT_SEC, TimeUnit.SECONDS);
+				assertFalse(requestWorkerFromDriverFutures.get(1).isDone());
+			});
+		}};
+	}
+
+	/**
+	 * Tests workers from previous attempt successfully recovered and registered.
+	 */
+	@Test
+	public void testRecoverWorkerFromPreviousAttempt() throws Exception {
+		new Context() {{
+			final ResourceID tmResourceId = ResourceID.generate();
+
+			runTest(() -> {
+				runInMainThread(() -> getResourceManager().onPreviousAttemptWorkersRecovered(Collections.singleton(tmResourceId)));
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(tmResourceId);
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Success.class));
+			});
+		}};
+	}
+
+	/**
+	 * Tests decline unknown worker registration.
+	 */
+	@Test
+	public void testRegisterUnknownWorker() throws Exception {
+		new Context() {{
+			runTest(() -> {
+				CompletableFuture<RegistrationResponse> registerTaskExecutorFuture = registerTaskExecutor(ResourceID.generate());
+				assertThat(registerTaskExecutorFuture.get(TIMEOUT_SEC, TimeUnit.SECONDS), instanceOf(RegistrationResponse.Decline.class));
+			});
+		}};
+	}
+
+	@Test
+	public void testOnError() throws Exception {
+		new Context() {{
+			final Throwable fatalError = new Throwable("Testing fatal error");
+			runTest(() -> {
+				runInMainThread(() -> getResourceManager().onError(fatalError));
+				final Throwable reportedError = getFatalErrorHandler().getErrorFuture().get(TIMEOUT_SEC, TimeUnit.SECONDS);
+				assertThat(reportedError, is(fatalError));
+			});
+		}};
+	}
+
+	class Context {
+
+		final Configuration flinkConfig = new Configuration();
+		final TestingResourceManagerDriver.Builder driverBuilder = new TestingResourceManagerDriver.Builder();
+		final TestingSlotManagerBuilder slotManagerBuilder = new TestingSlotManagerBuilder();
+
+		private ActiveResourceManager<ResourceID> resourceManager;
+		private TestingFatalErrorHandler fatalErrorHandler;
+
+		ActiveResourceManager<ResourceID> getResourceManager() {
+			return resourceManager;
+		}
+
+		TestingFatalErrorHandler getFatalErrorHandler() {
+			return fatalErrorHandler;
+		}
+
+		void runTest(RunnableWithException testMethod) throws Exception {
+			fatalErrorHandler = new TestingFatalErrorHandler();
+			resourceManager = createAndStartResourceManager(
+					flinkConfig,
+					driverBuilder.build(),
+					slotManagerBuilder.createSlotManager());
+
+			try {
+				testMethod.run();
+			} finally {
+				resourceManager.close();
+			}
+		}
+
+		private ActiveResourceManager<ResourceID> createAndStartResourceManager(
+				Configuration configuration,
+				ResourceManagerDriver<ResourceID> driver,
+				SlotManager slotManager) throws Exception {
+			final TestingRpcService rpcService = new TestingRpcService(configuration);
+			final MockResourceManagerRuntimeServices rmServices = new MockResourceManagerRuntimeServices(rpcService, TIMEOUT_TIME, slotManager);
+
+			final ActiveResourceManager<ResourceID> activeResourceManager = new ActiveResourceManager<>(
+					driver,
+					configuration,
+					rpcService,
+					ResourceID.generate(),
+					rmServices.highAvailabilityServices,
+					rmServices.heartbeatServices,
+					rmServices.slotManager,
+					NoOpResourceManagerPartitionTracker::get,
+					rmServices.jobLeaderIdService,
+					new ClusterInformation("localhost", 1234),
+					fatalErrorHandler,
+					UnregisteredMetricGroups.createUnregisteredResourceManagerMetricGroup());
+
+			activeResourceManager.start();
+			rmServices.grantLeadership();
+
+			return activeResourceManager;
+		}
+
+		public void runInMainThread(Runnable runnable) {
+			resourceManager.handleInMainThread(runnable);
+		}
+
+		public <T> CompletableFuture<T> runInMainThread(Callable<T> callable) {
+			return resourceManager.runInMainThread(callable, TIMEOUT_TIME);
+		}
+
+		CompletableFuture<RegistrationResponse> registerTaskExecutor(ResourceID resourceID) throws Exception {
+			final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+					.createTestingTaskExecutorGateway();
+			((TestingRpcService) resourceManager.getRpcService()).registerGateway(resourceID.toString(), taskExecutorGateway);

Review comment:
       Please see my other comment.




----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4957",
       "triggerID" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5404",
       "triggerID" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5458363056979d6be8594af4ac469c57ee8f9b15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5409",
       "triggerID" : "5458363056979d6be8594af4ac469c57ee8f9b15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5428",
       "triggerID" : "672548350",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5428",
       "triggerID" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "672548350",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "4847c551d75f244418947031dfe8b53bcf34073e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5449",
       "triggerID" : "4847c551d75f244418947031dfe8b53bcf34073e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "afc7db9e33b3996dd6b859b1ed1e9c418a7a0ef0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5502",
       "triggerID" : "afc7db9e33b3996dd6b859b1ed1e9c418a7a0ef0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "afc7db9e33b3996dd6b859b1ed1e9c418a7a0ef0",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5505",
       "triggerID" : "673474288",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "afc7db9e33b3996dd6b859b1ed1e9c418a7a0ef0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5502",
       "triggerID" : "673474288",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * afc7db9e33b3996dd6b859b1ed1e9c418a7a0ef0 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5505) Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5502) 
   
   <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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/ResourceEventHandler.java
##########
@@ -0,0 +1,60 @@
+/*
+ * 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 java.util.Collection;
+
+/**
+ * Callback interfaces for handling resource events from external resource managers.
+ */
+public interface ResourceEventHandler<WorkerType extends ResourceIDRetrievable> {
+
+	/**
+	 * Notifies that workers of previous attempt have been recovered from the external resource manager.
+	 *
+	 * @param recoveredWorkers Collection of worker nodes, in the deployment specific type.
+	 */
+	void onPreviousAttemptWorkersRecovered(Collection<WorkerType> recoveredWorkers);
+
+	/**
+	 * Notifies that the worker has been terminated.
+	 *
+	 * <p>See also {@link ResourceManagerDriver#requestResource}.
+	 *
+	 * @param resourceId Identifier of the terminated worker.
+	 */
+	void onWorkerTerminated(ResourceID resourceId);
+
+	/**
+	 * Notifies that an error has occurred that the process cannot proceed.
+	 *
+	 * @param exception Exception that describes the error.
+	 */
+	void onError(Throwable exception);
+
+	/**
+	 * Execute given runnable in the rpc main thread.
+	 *
+	 * @param runnable Runnable to be executed.
+	 */
+	void handleInMainThread(Runnable runnable);

Review comment:
       If we update the `ResourceManagerDriver` whenever the `ResourceManager` obtains the leadership, we are hurting another separation of concerns, because `ResourceManager` will need to understand when the main thread executor will be changed, which is currently hidden inside `FencedRpcEndpoint`.
   
   I had a feeling that it could be fragile to maintain the consistency between the two main thread executors.




----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4957",
       "triggerID" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5404",
       "triggerID" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5458363056979d6be8594af4ac469c57ee8f9b15",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5409",
       "triggerID" : "5458363056979d6be8594af4ac469c57ee8f9b15",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5458363056979d6be8594af4ac469c57ee8f9b15 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5409) 
   
   <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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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


   Thanks @tillrohrmann.
   @XComp, is there any further comments?


----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/CommonProcessMemorySpec.java
##########
@@ -94,4 +96,16 @@ public MemorySize getTotalFlinkMemorySize() {
 	public MemorySize getTotalProcessMemorySize() {
 		return flinkMemory.getTotalFlinkMemorySize().add(getJvmMetaspaceSize()).add(getJvmOverheadSize());
 	}
+
+	@Override
+	public boolean equals(Object obj) {

Review comment:
       Yes, I added `equals` to the memory spec classes because I need them in `ActiveResourceManagerTest`.
   
   > this class is a value class with a well-defined equality relation
   Agree, that's way I'm in favor of adding `equals` rather than custom Hamcrest machers.
   
   I'll add the `hashCode` for these classes.




----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4957",
       "triggerID" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5404",
       "triggerID" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5458363056979d6be8594af4ac469c57ee8f9b15",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5458363056979d6be8594af4ac469c57ee8f9b15",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4957) 
   * 906162e1f7a7988fd1f70bae81a76d6172497c63 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5404) 
   * 5458363056979d6be8594af4ac469c57ee8f9b15 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] tillrohrmann commented on a change in pull request #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/ResourceEventHandler.java
##########
@@ -0,0 +1,60 @@
+/*
+ * 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 java.util.Collection;
+
+/**
+ * Callback interfaces for handling resource events from external resource managers.
+ */
+public interface ResourceEventHandler<WorkerType extends ResourceIDRetrievable> {
+
+	/**
+	 * Notifies that workers of previous attempt have been recovered from the external resource manager.
+	 *
+	 * @param recoveredWorkers Collection of worker nodes, in the deployment specific type.
+	 */
+	void onPreviousAttemptWorkersRecovered(Collection<WorkerType> recoveredWorkers);
+
+	/**
+	 * Notifies that the worker has been terminated.
+	 *
+	 * <p>See also {@link ResourceManagerDriver#requestResource}.
+	 *
+	 * @param resourceId Identifier of the terminated worker.
+	 */
+	void onWorkerTerminated(ResourceID resourceId);
+
+	/**
+	 * Notifies that an error has occurred that the process cannot proceed.
+	 *
+	 * @param exception Exception that describes the error.
+	 */
+	void onError(Throwable exception);
+
+	/**
+	 * Execute given runnable in the rpc main thread.
+	 *
+	 * @param runnable Runnable to be executed.
+	 */
+	void handleInMainThread(Runnable runnable);

Review comment:
       I think we could pass in the `ComponentMainThreadExecutor` which is a public interface.




----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/ResourceEventHandler.java
##########
@@ -0,0 +1,60 @@
+/*
+ * 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 java.util.Collection;
+
+/**
+ * Callback interfaces for handling resource events from external resource managers.
+ */
+public interface ResourceEventHandler<WorkerType extends ResourceIDRetrievable> {
+
+	/**
+	 * Notifies that workers of previous attempt have been recovered from the external resource manager.
+	 *
+	 * @param recoveredWorkers Collection of worker nodes, in the deployment specific type.
+	 */
+	void onPreviousAttemptWorkersRecovered(Collection<WorkerType> recoveredWorkers);
+
+	/**
+	 * Notifies that the worker has been terminated.
+	 *
+	 * <p>See also {@link ResourceManagerDriver#requestResource}.
+	 *
+	 * @param resourceId Identifier of the terminated worker.
+	 */
+	void onWorkerTerminated(ResourceID resourceId);
+
+	/**
+	 * Notifies that an error has occurred that the process cannot proceed.
+	 *
+	 * @param exception Exception that describes the error.
+	 */
+	void onError(Throwable exception);
+
+	/**
+	 * Execute given runnable in the rpc main thread.
+	 *
+	 * @param runnable Runnable to be executed.
+	 */
+	void handleInMainThread(Runnable runnable);

Review comment:
       I think this is already the case: See `ResourceManager.grantLeadership`, `ResourceManager.tryAcceptLeadership` and `ResourceManager.startServicesOnLeadership`.




----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4957",
       "triggerID" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5404",
       "triggerID" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5458363056979d6be8594af4ac469c57ee8f9b15",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5409",
       "triggerID" : "5458363056979d6be8594af4ac469c57ee8f9b15",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5428",
       "triggerID" : "672548350",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5428",
       "triggerID" : "f1bfc953a17f24a74ac3da5db2d9dd790f90de8c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "672548350",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "4847c551d75f244418947031dfe8b53bcf34073e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5449",
       "triggerID" : "4847c551d75f244418947031dfe8b53bcf34073e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f1bfc953a17f24a74ac3da5db2d9dd790f90de8c Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5428) 
   * 4847c551d75f244418947031dfe8b53bcf34073e Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5449) 
   
   <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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/ResourceEventHandler.java
##########
@@ -0,0 +1,60 @@
+/*
+ * 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 java.util.Collection;
+
+/**
+ * Callback interfaces for handling resource events from external resource managers.
+ */
+public interface ResourceEventHandler<WorkerType extends ResourceIDRetrievable> {
+
+	/**
+	 * Notifies that workers of previous attempt have been recovered from the external resource manager.
+	 *
+	 * @param recoveredWorkers Collection of worker nodes, in the deployment specific type.
+	 */
+	void onPreviousAttemptWorkersRecovered(Collection<WorkerType> recoveredWorkers);
+
+	/**
+	 * Notifies that the worker has been terminated.
+	 *
+	 * <p>See also {@link ResourceManagerDriver#requestResource}.
+	 *
+	 * @param resourceId Identifier of the terminated worker.
+	 */
+	void onWorkerTerminated(ResourceID resourceId);
+
+	/**
+	 * Notifies that an error has occurred that the process cannot proceed.
+	 *
+	 * @param exception Exception that describes the error.
+	 */
+	void onError(Throwable exception);
+
+	/**
+	 * Execute given runnable in the rpc main thread.
+	 *
+	 * @param runnable Runnable to be executed.
+	 */
+	void handleInMainThread(Runnable runnable);

Review comment:
       And this could be simplified with lambda.
   ```
   resourceManagerDriver.initialize(this, command -> getMainThreadExecutor().execute(command));
   ```




----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4957",
       "triggerID" : "ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "906162e1f7a7988fd1f70bae81a76d6172497c63",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ce28faeeaddfdcdc72ce40dca0c2fd8a7f74c4ef Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4957) 
   * 906162e1f7a7988fd1f70bae81a76d6172497c63 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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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


   @tillrohrmann @wangyang0918 @KarmaGYZ
   Would you please help review 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] xintongsong commented on a change in pull request #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/active/TestingResourceManagerDriver.java
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.ApplicationStatus;
+import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.BiConsumerWithException;
+
+import javax.annotation.Nullable;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Supplier;
+
+/**
+ * Testing implementation of {@link ResourceManagerDriver}.
+ */
+public class TestingResourceManagerDriver implements ResourceManagerDriver<ResourceID> {
+
+	private final BiConsumerWithException<ResourceEventHandler<ResourceID>, Executor, Exception> initializeConsumer;
+	private final Supplier<CompletableFuture<Void>> terminateSupplier;
+	private final BiConsumerWithException<ApplicationStatus, String, Exception> deregisterApplicationConsumer;
+	private final Function<TaskExecutorProcessSpec, CompletableFuture<ResourceID>> requestResourceFunction;
+	private final Consumer<ResourceID> releaseResourceConsumer;
+
+	private TestingResourceManagerDriver(
+			final BiConsumerWithException<ResourceEventHandler<ResourceID>, Executor, Exception> initializeConsumer,
+			final Supplier<CompletableFuture<Void>> terminateSupplier,
+			final BiConsumerWithException<ApplicationStatus, String, Exception> deregisterApplicationConsumer,
+			final Function<TaskExecutorProcessSpec, CompletableFuture<ResourceID>> requestResourceFunction,
+			final Consumer<ResourceID> releaseResourceConsumer) {
+		this.initializeConsumer = Preconditions.checkNotNull(initializeConsumer);
+		this.terminateSupplier = Preconditions.checkNotNull(terminateSupplier);
+		this.deregisterApplicationConsumer = Preconditions.checkNotNull(deregisterApplicationConsumer);
+		this.requestResourceFunction = Preconditions.checkNotNull(requestResourceFunction);
+		this.releaseResourceConsumer = Preconditions.checkNotNull(releaseResourceConsumer);
+	}
+
+	@Override
+	public void initialize(ResourceEventHandler<ResourceID> resourceEventHandler, Executor mainThreadExecutor) throws Exception {
+		initializeConsumer.accept(resourceEventHandler, mainThreadExecutor);
+	}
+
+	@Override
+	public CompletableFuture<Void> terminate() {
+		return terminateSupplier.get();
+	}
+
+	@Override
+	public void deregisterApplication(ApplicationStatus finalStatus, @Nullable String optionalDiagnostics) throws Exception {
+		deregisterApplicationConsumer.accept(finalStatus, optionalDiagnostics);
+	}
+
+	@Override
+	public CompletableFuture<ResourceID> requestResource(TaskExecutorProcessSpec taskExecutorProcessSpec) {
+		return requestResourceFunction.apply(taskExecutorProcessSpec);
+	}
+
+	@Override
+	public void releaseResource(ResourceID worker) {
+		releaseResourceConsumer.accept(worker);
+	}
+
+	public static class Builder {
+		private BiConsumerWithException<ResourceEventHandler<ResourceID>, Executor, Exception> initializeConsumer =
+				(ignore1, ignore2) -> {};
+
+		private Supplier<CompletableFuture<Void>> terminateSupplier =
+				() -> CompletableFuture.completedFuture(null);
+
+		private BiConsumerWithException<ApplicationStatus, String, Exception> deregisterApplicationConsumer =
+				(ignore1, ignore2) -> {};
+
+		private Function<TaskExecutorProcessSpec, CompletableFuture<ResourceID>> requestResourceFunction =
+				(ignore) -> CompletableFuture.completedFuture(ResourceID.generate());
+
+		private Consumer<ResourceID> releaseResourceConsumer =
+				(ignore) -> {};
+
+		public Builder setInitializeConsumer(BiConsumerWithException<ResourceEventHandler<ResourceID>, Executor, Exception> initializeConsumer) {

Review comment:
       Well, in this case I’d rather leave it as it is. The community [code style guideline](https://flink.apache.org/contributing/code-style-and-quality-common.html) suggests the following.
   > JavaDocs should not state meaningless information (just to satisfy the Checkstyle checker).




----------------------------------------------------------------
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 #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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


   @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] xintongsong commented on a change in pull request #13004: [FLINK-18719][runtime] Define interfaces for new active RMs and implement

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/ResourceEventHandler.java
##########
@@ -0,0 +1,60 @@
+/*
+ * 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 java.util.Collection;
+
+/**
+ * Callback interfaces for handling resource events from external resource managers.
+ */
+public interface ResourceEventHandler<WorkerType extends ResourceIDRetrievable> {
+
+	/**
+	 * Notifies that workers of previous attempt have been recovered from the external resource manager.
+	 *
+	 * @param recoveredWorkers Collection of worker nodes, in the deployment specific type.
+	 */
+	void onPreviousAttemptWorkersRecovered(Collection<WorkerType> recoveredWorkers);
+
+	/**
+	 * Notifies that the worker has been terminated.
+	 *
+	 * <p>See also {@link ResourceManagerDriver#requestResource}.
+	 *
+	 * @param resourceId Identifier of the terminated worker.
+	 */
+	void onWorkerTerminated(ResourceID resourceId);
+
+	/**
+	 * Notifies that an error has occurred that the process cannot proceed.
+	 *
+	 * @param exception Exception that describes the error.
+	 */
+	void onError(Throwable exception);
+
+	/**
+	 * Execute given runnable in the rpc main thread.
+	 *
+	 * @param runnable Runnable to be executed.
+	 */
+	void handleInMainThread(Runnable runnable);

Review comment:
       I was also hesitating between the current approach and the one you described. The problem is that, `MainThreadExecutor` is a protected inner class of `RpcEndpoint`. I'm not entirely sure whether we want to break that visibility limit for this purpose. WDYT?




----------------------------------------------------------------
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