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:
[email protected]


Reply via email to