KarmaGYZ commented on a change in pull request #13311:
URL: https://github.com/apache/flink/pull/13311#discussion_r488363795



##########
File path: 
flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManagerDriver.java
##########
@@ -435,53 +308,43 @@ private void onContainersOfResourceAllocated(Resource 
resource, List<Container>
                        numAccepted, numExcess, numPending, resource);
        }
 
-       @VisibleForTesting
-       static ResourceID getContainerResourceId(Container container) {
-               return new ResourceID(container.getId().toString(), 
container.getNodeId().toString());
+       private int getNumRequestedNotAllocatedWorkers() {
+               return 
requestResourceFutures.values().stream().mapToInt(Queue::size).sum();
+       }
+
+       private int 
getNumRequestedNotAllocatedWorkersFor(TaskExecutorProcessSpec 
taskExecutorProcessSpec) {
+               return 
requestResourceFutures.getOrDefault(taskExecutorProcessSpec, new 
LinkedList<>()).size();
+       }
+
+       private void removeContainerRequest(AMRMClient.ContainerRequest 
pendingContainerRequest) {
+               log.info("Removing container request {}.", 
pendingContainerRequest);
+               
resourceManagerClient.removeContainerRequest(pendingContainerRequest);
+       }
+
+       private void returnExcessContainer(Container excessContainer) {
+               log.info("Returning excess container {}.", 
excessContainer.getId());
+               
resourceManagerClient.releaseAssignedContainer(excessContainer.getId());
        }
 
-       private void startTaskExecutorInContainer(Container container, 
WorkerResourceSpec workerResourceSpec, ResourceID resourceId) {
-               workerNodeMap.put(resourceId, new YarnWorkerNode(container, 
resourceId));
+       private void startTaskExecutorInContainer(Container container, 
TaskExecutorProcessSpec taskExecutorProcessSpec, ResourceID resourceId, 
CompletableFuture<YarnWorkerNode> requestResourceFuture) {
+               final YarnWorkerNode yarnWorkerNode = new 
YarnWorkerNode(container, resourceId);
 
                try {
                        // Context information used to start a TaskExecutor 
Java process
                        ContainerLaunchContext taskExecutorLaunchContext = 
createTaskExecutorLaunchContext(
                                resourceId,
                                container.getNodeId().getHost(),
-                               
TaskExecutorProcessUtils.processSpecFromWorkerResourceSpec(flinkConfig, 
workerResourceSpec));
+                               taskExecutorProcessSpec);
 
                        nodeManagerClient.startContainerAsync(container, 
taskExecutorLaunchContext);
+                       requestResourceFuture.complete(yarnWorkerNode);
                } catch (Throwable t) {
-                       
releaseFailedContainerAndRequestNewContainerIfRequired(container.getId(), t);
+                       requestResourceFuture.completeExceptionally(t);
                }
        }
 
-       private void 
releaseFailedContainerAndRequestNewContainerIfRequired(ContainerId containerId, 
Throwable throwable) {
-               validateRunsInMainThread();
-
-               log.error("Could not start TaskManager in container {}.", 
containerId, throwable);
-
-               final ResourceID resourceId = new 
ResourceID(containerId.toString());
-               // release the failed container
-               workerNodeMap.remove(resourceId);
-               resourceManagerClient.releaseAssignedContainer(containerId);
-               notifyAllocatedWorkerStopped(resourceId);
-               // and ask for a new one
-               requestYarnContainerIfRequired();
-       }
-
-       private void returnExcessContainer(Container excessContainer) {
-               log.info("Returning excess container {}.", 
excessContainer.getId());
-               
resourceManagerClient.releaseAssignedContainer(excessContainer.getId());
-       }
-
-       private void removeContainerRequest(AMRMClient.ContainerRequest 
pendingContainerRequest, WorkerResourceSpec workerResourceSpec) {
-               log.info("Removing container request {}.", 
pendingContainerRequest);
-               
resourceManagerClient.removeContainerRequest(pendingContainerRequest);
-       }
-
        private Collection<AMRMClient.ContainerRequest> 
getPendingRequestsAndCheckConsistency(Resource resource, int expectedNum) {
-               final Collection<Resource> equivalentResources = 
workerSpecContainerResourceAdapter.getEquivalentContainerResource(resource, 
matchingStrategy);
+               final Collection<Resource> equivalentResources = 
taskExecutorProcessSpecContainerResourceAdapter.getEquivalentContainerResource(resource,
 matchingStrategy);
                final List<? extends Collection<AMRMClient.ContainerRequest>> 
matchingRequests =

Review comment:
       I think we could get a bit readability here. Thanks for the remark.




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