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

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

Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2808#discussion_r89367356
  
    --- 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);
    --- End diff --
    
    Can we initialize the `AMRMClientAsync` and the `NMClient` outside of the 
`YarnResourceManager` and then pass them to it via the constructor? That way we 
could reuse them e.g. for the `Dispatcher` component to allocate containers for 
a new `JobMaster`. Furthermore, it removes initialization code from the 
`YarnResourceManager`.


> 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