[ 
https://issues.apache.org/jira/browse/FLINK-4927?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15704581#comment-15704581
 ] 

ASF GitHub Bot commented on FLINK-4927:
---------------------------------------

Github user shuai-xu commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2808#discussion_r89955907
  
    --- Diff: 
flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java ---
    @@ -0,0 +1,551 @@
    +/*
    + * 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.yarn;
    +
    +import org.apache.flink.api.java.tuple.Tuple2;
    +import org.apache.flink.configuration.ConfigConstants;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.runtime.clusterframework.ApplicationStatus;
    +import org.apache.flink.runtime.clusterframework.BootstrapTools;
    +import 
org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters;
    +import org.apache.flink.runtime.clusterframework.types.ResourceID;
    +import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
    +import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
    +import org.apache.flink.runtime.metrics.MetricRegistry;
    +import org.apache.flink.runtime.resourcemanager.JobLeaderIdService;
    +import org.apache.flink.runtime.resourcemanager.ResourceManager;
    +import 
org.apache.flink.runtime.resourcemanager.ResourceManagerConfiguration;
    +import 
org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException;
    +import 
org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerFactory;
    +import org.apache.flink.runtime.rpc.FatalErrorHandler;
    +import org.apache.flink.runtime.rpc.RpcService;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hadoop.io.DataOutputBuffer;
    +import org.apache.hadoop.security.Credentials;
    +import org.apache.hadoop.security.UserGroupInformation;
    +import org.apache.hadoop.yarn.api.ApplicationConstants;
    +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
    +import org.apache.hadoop.yarn.api.records.Priority;
    +import org.apache.hadoop.yarn.api.records.Resource;
    +import org.apache.hadoop.yarn.api.records.ContainerStatus;
    +import org.apache.hadoop.yarn.api.records.Container;
    +import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
    +import org.apache.hadoop.yarn.api.records.NodeReport;
    +import org.apache.hadoop.yarn.api.records.LocalResource;
    +import org.apache.hadoop.yarn.client.api.AMRMClient;
    +import org.apache.hadoop.yarn.client.api.NMClient;
    +import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
    +import org.apache.hadoop.yarn.conf.YarnConfiguration;
    +import org.apache.hadoop.yarn.util.Records;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import scala.concurrent.duration.FiniteDuration;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.nio.ByteBuffer;
    +import java.util.Map;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Collections;
    +import java.util.UUID;
    +import java.util.concurrent.TimeUnit;
    +
    +import static org.apache.flink.yarn.YarnConfigKeys.ENV_FLINK_CLASSPATH;
    +
    +/**
    + * The yarn implementation of the resource manager. Used when the system 
is started
    + * via the resource framework YARN.
    + */
    +public class YarnResourceManager extends ResourceManager<ResourceID> 
implements AMRMClientAsync.CallbackHandler {
    +   protected final Logger LOG = LoggerFactory.getLogger(getClass());
    +
    +   /** The process environment variables */
    +   private final Map<String, String> ENV;
    +
    +   /** The heartbeat interval while the resource master is waiting for 
containers */
    +   private static final int FAST_YARN_HEARTBEAT_INTERVAL_MS = 500;
    +
    +   /** The default heartbeat interval during regular operation */
    +   private static final int DEFAULT_YARN_HEARTBEAT_INTERVAL_MS = 5000;
    +
    +   /** The maximum time that TaskExecutors may be waiting to register at 
the ResourceManager before they quit */
    +   private static final FiniteDuration TASKEXECUTOR_REGISTRATION_TIMEOUT = 
new FiniteDuration(5, TimeUnit.MINUTES);
    +
    +   /** Environment variable name of the final container id used by the 
YarnResourceManager.
    +    * Container ID generation may vary across Hadoop versions. */
    +   final static String ENV_FLINK_CONTAINER_ID = "_FLINK_CONTAINER_ID";
    +   
    +   /** Environment variable name of the hostname used by the Yarn.
    +    * TaskExecutor use this host name to start port. */
    +   final static String ENV_FLINK_NODE_ID = "_FLINK_NODE_ID";
    +
    +   /** Default heartbeat interval between this resource manager and the 
YARN ResourceManager */
    +   private final int yarnHeartbeatIntervalMillis;
    +
    +   private final Configuration flinkConfig;
    +
    +   private final YarnConfiguration yarnConfig;
    +
    +   /** Client to communicate with the Resource Manager (YARN's master) */
    +   private AMRMClientAsync<AMRMClient.ContainerRequest> 
resourceManagerClient;
    +
    +   /** Client to communicate with the Node manager and launch TaskExecutor 
processes */
    +   private NMClient nodeManagerClient;
    +
    +   /** The number of containers requested, but not yet granted */
    +   private int numPendingContainerRequests;
    +
    +   public YarnResourceManager(
    +                   Configuration flinkConfig,
    +                   Map<String, String> env,
    +                   RpcService rpcService,
    +                   ResourceManagerConfiguration 
resourceManagerConfiguration,
    +                   HighAvailabilityServices highAvailabilityServices,
    +                   SlotManagerFactory slotManagerFactory,
    +                   MetricRegistry metricRegistry,
    +                   JobLeaderIdService jobLeaderIdService,
    +                   FatalErrorHandler fatalErrorHandler) {
    +           super(
    +                   rpcService,
    +                   resourceManagerConfiguration,
    +                   highAvailabilityServices,
    +                   slotManagerFactory,
    +                   metricRegistry,
    +                   jobLeaderIdService,
    +                   fatalErrorHandler);
    +           this.flinkConfig  = flinkConfig;
    +           this.yarnConfig = new YarnConfiguration();
    +           this.ENV = env;
    +           final int yarnHeartbeatIntervalMS = flinkConfig.getInteger(
    +                           ConfigConstants.YARN_HEARTBEAT_DELAY_SECONDS, 
DEFAULT_YARN_HEARTBEAT_INTERVAL_MS / 1000) * 1000;
    +
    +           final long yarnExpiryIntervalMS = yarnConfig.getLong(
    +                           YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS,
    +                           
YarnConfiguration.DEFAULT_RM_AM_EXPIRY_INTERVAL_MS);
    +
    +           if (yarnHeartbeatIntervalMS >= yarnExpiryIntervalMS) {
    +                   log.warn("The heartbeat interval of the Flink 
Application master ({}) is greater " +
    +                                   "than YARN's expiry interval ({}). The 
application is likely to be killed by YARN.",
    +                                   yarnHeartbeatIntervalMS, 
yarnExpiryIntervalMS);
    +           }
    +           yarnHeartbeatIntervalMillis = yarnHeartbeatIntervalMS;
    +           numPendingContainerRequests = 0;
    +   }
    +
    +   @Override
    +   protected void initialize() throws ResourceManagerException {
    +           resourceManagerClient = 
AMRMClientAsync.createAMRMClientAsync(yarnHeartbeatIntervalMillis, this);
    +           resourceManagerClient.init(yarnConfig);
    +           resourceManagerClient.start();
    +           try {
    +                   //TODO: change akka address to tcp host and port, the 
getAddress() interface should return a standard tcp address
    +                   Tuple2<String, Integer> hostPort = 
parseHostPort(getAddress());
    +                   
resourceManagerClient.registerApplicationMaster(hostPort.f0, hostPort.f1, 
getAddress());
    +           } catch (Exception e) {
    +                   LOG.info("registerApplicationMaster fail", e);
    +           }
    +
    +           // create the client to communicate with the node managers
    +           nodeManagerClient = NMClient.createNMClient();
    +           nodeManagerClient.init(yarnConfig);
    +           nodeManagerClient.start();
    +           nodeManagerClient.cleanupRunningContainersOnStop(true);
    +   }
    +
    +   @Override
    +   public void shutDown() throws Exception {
    +           // shut down all components
    +           if (resourceManagerClient != null) {
    +                   try {
    +                           resourceManagerClient.stop();
    +                   } catch (Throwable t) {
    +                           LOG.error("Could not cleanly shut down the 
Asynchronous Resource Manager Client", t);
    +                   }
    +           }
    +           if (nodeManagerClient != null) {
    +                   try {
    +                           nodeManagerClient.stop();
    +                   } catch (Throwable t) {
    +                           LOG.error("Could not cleanly shut down the Node 
Manager Client", t);
    +                   }
    +           }
    +           super.shutDown();
    +   }
    +
    +   @Override
    +   protected void shutDownApplication(ApplicationStatus finalStatus, 
String optionalDiagnostics) {
    +
    +           // first, de-register from YARN
    +           FinalApplicationStatus yarnStatus = getYarnStatus(finalStatus);
    +           LOG.info("Unregistering application from the YARN Resource 
Manager");
    +           try {
    +                   
resourceManagerClient.unregisterApplicationMaster(yarnStatus, 
optionalDiagnostics, "");
    +           } catch (Throwable t) {
    +                   LOG.error("Could not unregister the application 
master.", t);
    +           }
    +   }
    +
    +   @Override
    +   public void startNewWorker(ResourceProfile resourceProfile) {
    +           // Priority for worker containers - priorities are 
intra-application
    +           //TODO: set priority according to the resource allocated
    +           Priority priority = Priority.newInstance(0);
    +           int mem = resourceProfile.getMemoryInMB() <= Integer.MAX_VALUE 
? (int)resourceProfile.getMemoryInMB() : Integer.MAX_VALUE;
    +           if (mem < 0) {
    +                   mem = 1024;
    +           }
    +           int vcore = resourceProfile.getCpuCores() < 1 ? 1 : 
(int)resourceProfile.getCpuCores() + 1;
    +           Resource capability = Resource.newInstance(mem , vcore);
    +           requestYarnContainer(capability, priority);
    +   }
    +
    +   @Override
    +   protected ResourceID workerStarted(ResourceID resourceID) {
    +           return resourceID;
    +   }
    +
    +   // AMRMClientAsync CallbackHandler methods
    +   @Override
    +   public float getProgress() {
    +           // Temporarily need not record the total size of asked and 
allocated containers
    +           return 1;
    +   }
    +
    +   @Override
    +   public void onContainersCompleted(List<ContainerStatus> list) {
    +           for (ContainerStatus container : list) {
    +                   if (container.getExitStatus() < 0) {
    +                           notifyWorkerFailed(new 
ResourceID(container.getContainerId().toString()), container.getDiagnostics());
    +                           // TODO: notice job master slot fail
    +                   }
    +           }
    +   }
    +
    +   @Override
    +   public void onContainersAllocated(List<Container> containers) {
    +           for (Container container : containers) {
    +                   numPendingContainerRequests = Math.max(0, 
numPendingContainerRequests - 1);
    +                   LOG.info("Received new container: {} - Remaining 
pending container requests: {}",
    +                                   container.getId(), 
numPendingContainerRequests);
    +                   try {
    +                           /** Context information used to start a 
TaskExecutor Java process */
    +                           ContainerLaunchContext 
taskExecutorLaunchContext =
    +                                           
createTaskExecutorLaunchContext(container.getResource(), 
container.getId().toString(), container.getNodeId().getHost());
    +                           nodeManagerClient.startContainer(container, 
taskExecutorLaunchContext);
    +                   }
    +                   catch (Throwable t) {
    +                           // failed to launch the container, will release 
the failed one and ask for a new one
    +                           LOG.error("Could not start TaskManager in 
container " + container, t);
    +                           
resourceManagerClient.releaseAssignedContainer(container.getId());
    +                           requestYarnContainer(container.getResource(), 
container.getPriority());
    +                   }
    +           }
    +           if (numPendingContainerRequests <= 0) {
    +                   
resourceManagerClient.setHeartbeatInterval(yarnHeartbeatIntervalMillis);
    +           }
    +   }
    +
    +   @Override
    +   public void onShutdownRequest() {
    +           // Nothing to do
    +   }
    +
    +   @Override
    +   public void onNodesUpdated(List<NodeReport> list) {
    +           // We are not interested in node updates
    --- End diff --
    
    The yarn master will send nodes who's status is changed by this call. We 
can get enough by the notices of containers, so need not process this call.


> Implement FLI-6 YARN Resource Manager
> -------------------------------------
>
>                 Key: FLINK-4927
>                 URL: https://issues.apache.org/jira/browse/FLINK-4927
>             Project: Flink
>          Issue Type: Sub-task
>          Components: YARN
>         Environment: {{flip-6}} feature branch
>            Reporter: Stephan Ewen
>            Assignee: shuai.xu
>
> The Flink YARN Resource Manager communicates with YARN's Resource Manager to 
> acquire and release containers.
> It is also responsible to notify the JobManager eagerly about container 
> failures.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to