diff --git hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java index 349a019..c588bbc 100644 --- hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java +++ hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java @@ -41,7 +41,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.YarnClusterMetrics; -import org.apache.hadoop.yarn.client.YarnClientImpl; +import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.util.ProtoUtils; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java index 64b4d6f..6b7768b 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java @@ -67,9 +67,9 @@ import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceRequest; -import org.apache.hadoop.yarn.client.AMRMClient.ContainerRequest; -import org.apache.hadoop.yarn.client.AMRMClientAsync; -import org.apache.hadoop.yarn.client.NMClientAsync; +import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest; +import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync; +import org.apache.hadoop.yarn.client.api.async.NMClientAsync; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.util.ConverterUtils; @@ -436,17 +436,18 @@ private void printUsage(Options opts) { * @throws YarnException * @throws IOException */ - @SuppressWarnings({ "rawtypes", "unchecked" }) + @SuppressWarnings({ "unchecked" }) public boolean run() throws YarnException, IOException { LOG.info("Starting ApplicationMaster"); AMRMClientAsync.CallbackHandler allocListener = new RMCallbackHandler(); - resourceManager = new AMRMClientAsync(appAttemptID, 1000, allocListener); + resourceManager = + AMRMClientAsync.createAMRMClientAsync(appAttemptID, 1000, allocListener); resourceManager.init(conf); resourceManager.start(); containerListener = new NMCallbackHandler(); - nmClientAsync = new NMClientAsync(containerListener); + nmClientAsync = NMClientAsync.createNMClientAsync(containerListener); nmClientAsync.init(conf); nmClientAsync.start(); @@ -682,7 +683,7 @@ public void onContainerStarted(ContainerId containerId, } Container container = containers.get(containerId); if (container != null) { - nmClientAsync.getContainerStatus(containerId, container.getNodeId(), + nmClientAsync.getContainerStatusAsync(containerId, container.getNodeId(), container.getContainerToken()); } } @@ -802,7 +803,7 @@ public void run() { ctx.setCommands(commands); containerListener.addContainer(container.getId(), container); - nmClientAsync.startContainer(container, ctx); + nmClientAsync.startContainerAsync(container, ctx); } } diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java index 392b07c..7f2379b 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java @@ -60,7 +60,7 @@ import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.api.records.YarnClusterMetrics; -import org.apache.hadoop.yarn.client.YarnClient; +import org.apache.hadoop.yarn.client.api.YarnClient; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.util.ConverterUtils; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/src/main/java/org/apache/hadoop/yarn/applications/unmanagedamlauncher/UnmanagedAMLauncher.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/src/main/java/org/apache/hadoop/yarn/applications/unmanagedamlauncher/UnmanagedAMLauncher.java index 5a7892d..a2f1019 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/src/main/java/org/apache/hadoop/yarn/applications/unmanagedamlauncher/UnmanagedAMLauncher.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/src/main/java/org/apache/hadoop/yarn/applications/unmanagedamlauncher/UnmanagedAMLauncher.java @@ -48,8 +48,8 @@ import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.YarnApplicationState; -import org.apache.hadoop.yarn.client.YarnClient; -import org.apache.hadoop.yarn.client.YarnClientImpl; +import org.apache.hadoop.yarn.client.api.YarnClient; +import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.util.Records; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClient.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClient.java deleted file mode 100644 index 7700d6c..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClient.java +++ /dev/null @@ -1,261 +0,0 @@ -/** -* 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.hadoop.yarn.client; - -import java.io.IOException; -import java.util.Collection; -import java.util.List; -import java.util.concurrent.ConcurrentMap; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceAudience.Private; -import org.apache.hadoop.classification.InterfaceAudience.Public; -import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.service.AbstractService; -import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; -import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; -import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.ContainerId; -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.Token; -import org.apache.hadoop.yarn.exceptions.YarnException; - -import com.google.common.collect.ImmutableList; - -@InterfaceAudience.Public -@InterfaceStability.Unstable -public abstract class AMRMClient extends - AbstractService { - - /** - * Create a new instance of AMRMClient. - * For usage: - *
-   * {@code
-   * AMRMClient.createAMRMClientContainerRequest(appAttemptId)
-   * }
- * @param appAttemptId the appAttemptId associated with the AMRMClient - * @return the newly create AMRMClient instance. - */ - @Public - public static AMRMClient createAMRMClient( - ApplicationAttemptId appAttemptId) { - AMRMClient client = new AMRMClientImpl(appAttemptId); - return client; - } - - @Private - protected AMRMClient(String name) { - super(name); - } - - /** - * Object to represent container request for resources. Scheduler - * documentation should be consulted for the specifics of how the parameters - * are honored. - * All getters return immutable values. - * - * @param capability - * The {@link Resource} to be requested for each container. - * @param nodes - * Any hosts to request that the containers are placed on. - * @param racks - * Any racks to request that the containers are placed on. The racks - * corresponding to any hosts requested will be automatically added to - * this list. - * @param priority - * The priority at which to request the containers. Higher priorities have - * lower numerical values. - * @param containerCount - * The number of containers to request. - */ - public static class ContainerRequest { - final Resource capability; - final List nodes; - final List racks; - final Priority priority; - final int containerCount; - - public ContainerRequest(Resource capability, String[] nodes, - String[] racks, Priority priority, int containerCount) { - this.capability = capability; - this.nodes = (nodes != null ? ImmutableList.copyOf(nodes) : null); - this.racks = (racks != null ? ImmutableList.copyOf(racks) : null); - this.priority = priority; - this.containerCount = containerCount; - } - - public Resource getCapability() { - return capability; - } - - public List getNodes() { - return nodes; - } - - public List getRacks() { - return racks; - } - - public Priority getPriority() { - return priority; - } - - public int getContainerCount() { - return containerCount; - } - - public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append("Capability[").append(capability).append("]"); - sb.append("Priority[").append(priority).append("]"); - sb.append("ContainerCount[").append(containerCount).append("]"); - return sb.toString(); - } - } - - /** - * This creates a ContainerRequest for 1 container and the - * AMRMClient stores this request internally. getMatchingRequests - * can be used to retrieve these requests from AMRMClient. These requests may - * be matched with an allocated container to determine which request to assign - * the container to. removeContainerRequest must be called using - * the same assigned StoredContainerRequest object so that - * AMRMClient can remove it from its internal store. - */ - public static class StoredContainerRequest extends ContainerRequest { - public StoredContainerRequest(Resource capability, String[] nodes, - String[] racks, Priority priority) { - super(capability, nodes, racks, priority, 1); - } - } - - /** - * Register the application master. This must be called before any - * other interaction - * @param appHostName Name of the host on which master is running - * @param appHostPort Port master is listening on - * @param appTrackingUrl URL at which the master info can be seen - * @return RegisterApplicationMasterResponse - * @throws YarnException - * @throws IOException - */ - public abstract RegisterApplicationMasterResponse - registerApplicationMaster(String appHostName, - int appHostPort, - String appTrackingUrl) - throws YarnException, IOException; - - /** - * Request additional containers and receive new container allocations. - * Requests made via addContainerRequest are sent to the - * ResourceManager. New containers assigned to the master are - * retrieved. Status of completed containers and node health updates are - * also retrieved. - * This also doubles up as a heartbeat to the ResourceManager and must be - * made periodically. - * The call may not always return any new allocations of containers. - * App should not make concurrent allocate requests. May cause request loss. - * @param progressIndicator Indicates progress made by the master - * @return the response of the allocate request - * @throws YarnException - * @throws IOException - */ - public abstract AllocateResponse allocate(float progressIndicator) - throws YarnException, IOException; - - /** - * Unregister the application master. This must be called in the end. - * @param appStatus Success/Failure status of the master - * @param appMessage Diagnostics message on failure - * @param appTrackingUrl New URL to get master info - * @throws YarnException - * @throws IOException - */ - public abstract void unregisterApplicationMaster(FinalApplicationStatus appStatus, - String appMessage, - String appTrackingUrl) - throws YarnException, IOException; - - /** - * Request containers for resources before calling allocate - * @param req Resource request - */ - public abstract void addContainerRequest(T req); - - /** - * Remove previous container request. The previous container request may have - * already been sent to the ResourceManager. So even after the remove request - * the app must be prepared to receive an allocation for the previous request - * even after the remove request - * @param req Resource request - */ - public abstract void removeContainerRequest(T req); - - /** - * Release containers assigned by the Resource Manager. If the app cannot use - * the container or wants to give up the container then it can release them. - * The app needs to make new requests for the released resource capability if - * it still needs it. eg. it released non-local resources - * @param containerId - */ - public abstract void releaseAssignedContainer(ContainerId containerId); - - /** - * Get the currently available resources in the cluster. - * A valid value is available after a call to allocate has been made - * @return Currently available resources - */ - public abstract Resource getClusterAvailableResources(); - - /** - * Get the current number of nodes in the cluster. - * A valid values is available after a call to allocate has been made - * @return Current number of nodes in the cluster - */ - public abstract int getClusterNodeCount(); - - /** - * Get outstanding StoredContainerRequests matching the given - * parameters. These StoredContainerRequests should have been added via - * addContainerRequest earlier in the lifecycle. For performance, - * the AMRMClient may return its internal collection directly without creating - * a copy. Users should not perform mutable operations on the return value. - * Each collection in the list contains requests with identical - * Resource size that fit in the given capability. In a - * collection, requests will be returned in the same order as they were added. - * @return Collection of request matching the parameters - */ - public abstract List> getMatchingRequests( - Priority priority, - String resourceName, - Resource capability); - - /** - * It returns the NMToken received on allocate call. It will not communicate - * with RM to get NMTokens. On allocate call whenever we receive new token - * along with container AMRMClient will cache this NMToken per node manager. - * This map returned should be shared with any application which is - * communicating with NodeManager (ex. NMClient) using NMTokens. If a new - * NMToken is received for the same node manager then it will be replaced. - */ - public abstract ConcurrentMap getNMTokens(); -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClientAsync.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClientAsync.java deleted file mode 100644 index 09918dd..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClientAsync.java +++ /dev/null @@ -1,423 +0,0 @@ -/** -* 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.hadoop.yarn.client; - -import java.io.IOException; -import java.util.Collection; -import java.util.List; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.atomic.AtomicInteger; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.classification.InterfaceAudience.Private; -import org.apache.hadoop.classification.InterfaceStability.Evolving; -import org.apache.hadoop.classification.InterfaceStability.Unstable; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.service.AbstractService; -import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; -import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; -import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.ContainerStatus; -import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; -import org.apache.hadoop.yarn.api.records.NodeReport; -import org.apache.hadoop.yarn.api.records.Priority; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.api.records.Token; -import org.apache.hadoop.yarn.client.AMRMClient.ContainerRequest; -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; - -import com.google.common.annotations.VisibleForTesting; - -/** - * AMRMClientAsync handles communication with the ResourceManager - * and provides asynchronous updates on events such as container allocations and - * completions. It contains a thread that sends periodic heartbeats to the - * ResourceManager. - * - * It should be used by implementing a CallbackHandler: - *
- * {@code
- * class MyCallbackHandler implements AMRMClientAsync.CallbackHandler {
- *   public void onContainersAllocated(List containers) {
- *     [run tasks on the containers]
- *   }
- *   
- *   public void onContainersCompleted(List statuses) {
- *     [update progress, check whether app is done]
- *   }
- *   
- *   public void onNodesUpdated(List updated) {}
- *   
- *   public void onReboot() {}
- * }
- * }
- * 
- * - * The client's lifecycle should be managed similarly to the following: - * - *
- * {@code
- * AMRMClientAsync asyncClient = new AMRMClientAsync(appAttId, 1000, new MyCallbackhandler());
- * asyncClient.init(conf);
- * asyncClient.start();
- * RegisterApplicationMasterResponse response = asyncClient
- *    .registerApplicationMaster(appMasterHostname, appMasterRpcPort,
- *       appMasterTrackingUrl);
- * asyncClient.addContainerRequest(containerRequest);
- * [... wait for application to complete]
- * asyncClient.unregisterApplicationMaster(status, appMsg, trackingUrl);
- * asyncClient.stop();
- * }
- * 
- */ -@Unstable -@Evolving -public class AMRMClientAsync extends AbstractService { - - private static final Log LOG = LogFactory.getLog(AMRMClientAsync.class); - - private final AMRMClient client; - private final AtomicInteger heartbeatIntervalMs = new AtomicInteger(); - private final HeartbeatThread heartbeatThread; - private final CallbackHandlerThread handlerThread; - private final CallbackHandler handler; - - private final BlockingQueue responseQueue; - - private final Object unregisterHeartbeatLock = new Object(); - - private volatile boolean keepRunning; - private volatile float progress; - - private volatile Exception savedException; - - public AMRMClientAsync(ApplicationAttemptId id, int intervalMs, - CallbackHandler callbackHandler) { - this(new AMRMClientImpl(id), intervalMs, callbackHandler); - } - - @Private - @VisibleForTesting - protected AMRMClientAsync(AMRMClient client, int intervalMs, - CallbackHandler callbackHandler) { - super(AMRMClientAsync.class.getName()); - this.client = client; - this.heartbeatIntervalMs.set(intervalMs); - handler = callbackHandler; - heartbeatThread = new HeartbeatThread(); - handlerThread = new CallbackHandlerThread(); - responseQueue = new LinkedBlockingQueue(); - keepRunning = true; - savedException = null; - } - - @Override - protected void serviceInit(Configuration conf) throws Exception { - super.serviceInit(conf); - client.init(conf); - } - - @Override - protected void serviceStart() throws Exception { - handlerThread.start(); - client.start(); - super.serviceStart(); - } - - /** - * Tells the heartbeat and handler threads to stop and waits for them to - * terminate. Calling this method from the callback handler thread would cause - * deadlock, and thus should be avoided. - */ - @Override - protected void serviceStop() throws Exception { - if (Thread.currentThread() == handlerThread) { - throw new YarnRuntimeException("Cannot call stop from callback handler thread!"); - } - keepRunning = false; - try { - heartbeatThread.join(); - } catch (InterruptedException ex) { - LOG.error("Error joining with heartbeat thread", ex); - } - client.stop(); - try { - handlerThread.interrupt(); - handlerThread.join(); - } catch (InterruptedException ex) { - LOG.error("Error joining with hander thread", ex); - } - super.serviceStop(); - } - - public void setHeartbeatInterval(int interval) { - heartbeatIntervalMs.set(interval); - } - - public List> getMatchingRequests( - Priority priority, - String resourceName, - Resource capability) { - return client.getMatchingRequests(priority, resourceName, capability); - } - - /** - * Registers this application master with the resource manager. On successful - * registration, starts the heartbeating thread. - * @throws YarnException - * @throws IOException - */ - public RegisterApplicationMasterResponse registerApplicationMaster( - String appHostName, int appHostPort, String appTrackingUrl) - throws YarnException, IOException { - RegisterApplicationMasterResponse response = client - .registerApplicationMaster(appHostName, appHostPort, appTrackingUrl); - heartbeatThread.start(); - return response; - } - - /** - * Unregister the application master. This must be called in the end. - * @param appStatus Success/Failure status of the master - * @param appMessage Diagnostics message on failure - * @param appTrackingUrl New URL to get master info - * @throws YarnException - * @throws IOException - */ - public void unregisterApplicationMaster(FinalApplicationStatus appStatus, - String appMessage, String appTrackingUrl) throws YarnException, - IOException { - synchronized (unregisterHeartbeatLock) { - keepRunning = false; - client.unregisterApplicationMaster(appStatus, appMessage, appTrackingUrl); - } - } - - /** - * Request containers for resources before calling allocate - * @param req Resource request - */ - public void addContainerRequest(T req) { - client.addContainerRequest(req); - } - - /** - * Remove previous container request. The previous container request may have - * already been sent to the ResourceManager. So even after the remove request - * the app must be prepared to receive an allocation for the previous request - * even after the remove request - * @param req Resource request - */ - public void removeContainerRequest(T req) { - client.removeContainerRequest(req); - } - - /** - * Release containers assigned by the Resource Manager. If the app cannot use - * the container or wants to give up the container then it can release them. - * The app needs to make new requests for the released resource capability if - * it still needs it. eg. it released non-local resources - * @param containerId - */ - public void releaseAssignedContainer(ContainerId containerId) { - client.releaseAssignedContainer(containerId); - } - - /** - * Get the currently available resources in the cluster. - * A valid value is available after a call to allocate has been made - * @return Currently available resources - */ - public Resource getClusterAvailableResources() { - return client.getClusterAvailableResources(); - } - - /** - * Get the current number of nodes in the cluster. - * A valid values is available after a call to allocate has been made - * @return Current number of nodes in the cluster - */ - public int getClusterNodeCount() { - return client.getClusterNodeCount(); - } - - /** - * It returns the NMToken received on allocate call. It will not communicate - * with RM to get NMTokens. On allocate call whenever we receive new token - * along with new container AMRMClientAsync will cache this NMToken per node - * manager. This map returned should be shared with any application which is - * communicating with NodeManager (ex. NMClient / NMClientAsync) using - * NMTokens. If a new NMToken is received for the same node manager - * then it will be replaced. - */ - public ConcurrentMap getNMTokens() { - return client.getNMTokens(); - } - - private class HeartbeatThread extends Thread { - public HeartbeatThread() { - super("AMRM Heartbeater thread"); - } - - public void run() { - while (true) { - AllocateResponse response = null; - // synchronization ensures we don't send heartbeats after unregistering - synchronized (unregisterHeartbeatLock) { - if (!keepRunning) { - break; - } - - try { - response = client.allocate(progress); - } catch (YarnException ex) { - LOG.error("Yarn exception on heartbeat", ex); - savedException = ex; - // interrupt handler thread in case it waiting on the queue - handlerThread.interrupt(); - break; - } catch (IOException e) { - LOG.error("IO exception on heartbeat", e); - savedException = e; - // interrupt handler thread in case it waiting on the queue - handlerThread.interrupt(); - break; - } - } - if (response != null) { - while (true) { - try { - responseQueue.put(response); - break; - } catch (InterruptedException ex) { - LOG.info("Interrupted while waiting to put on response queue", ex); - } - } - } - - try { - Thread.sleep(heartbeatIntervalMs.get()); - } catch (InterruptedException ex) { - LOG.info("Heartbeater interrupted", ex); - } - } - } - } - - private class CallbackHandlerThread extends Thread { - public CallbackHandlerThread() { - super("AMRM Callback Handler Thread"); - } - - public void run() { - while (keepRunning) { - AllocateResponse response; - try { - if(savedException != null) { - LOG.error("Stopping callback due to: ", savedException); - handler.onError(savedException); - break; - } - response = responseQueue.take(); - } catch (InterruptedException ex) { - LOG.info("Interrupted while waiting for queue", ex); - continue; - } - - if (response.getAMCommand() != null) { - boolean stop = false; - switch(response.getAMCommand()) { - case AM_RESYNC: - case AM_SHUTDOWN: - handler.onShutdownRequest(); - LOG.info("Shutdown requested. Stopping callback."); - stop = true; - break; - default: - String msg = - "Unhandled value of AMCommand: " + response.getAMCommand(); - LOG.error(msg); - throw new YarnRuntimeException(msg); - } - if(stop) { - // should probably stop heartbeating also YARN-763 - break; - } - } - List updatedNodes = response.getUpdatedNodes(); - if (!updatedNodes.isEmpty()) { - handler.onNodesUpdated(updatedNodes); - } - - List completed = - response.getCompletedContainersStatuses(); - if (!completed.isEmpty()) { - handler.onContainersCompleted(completed); - } - - List allocated = response.getAllocatedContainers(); - if (!allocated.isEmpty()) { - handler.onContainersAllocated(allocated); - } - - progress = handler.getProgress(); - } - } - } - - public interface CallbackHandler { - - /** - * Called when the ResourceManager responds to a heartbeat with completed - * containers. If the response contains both completed containers and - * allocated containers, this will be called before containersAllocated. - */ - public void onContainersCompleted(List statuses); - - /** - * Called when the ResourceManager responds to a heartbeat with allocated - * containers. If the response containers both completed containers and - * allocated containers, this will be called after containersCompleted. - */ - public void onContainersAllocated(List containers); - - /** - * Called when the ResourceManager wants the ApplicationMaster to shutdown - * for being out of sync etc. The ApplicationMaster should not unregister - * with the RM unless the ApplicationMaster wants to be the last attempt. - */ - public void onShutdownRequest(); - - /** - * Called when nodes tracked by the ResourceManager have changed in health, - * availability etc. - */ - public void onNodesUpdated(List updatedNodes); - - public float getProgress(); - - public void onError(Exception e); - } -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClientImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClientImpl.java deleted file mode 100644 index c24b6fa..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClientImpl.java +++ /dev/null @@ -1,583 +0,0 @@ -/** -* 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.hadoop.yarn.client; - -import java.io.IOException; -import java.net.InetSocketAddress; -import java.security.PrivilegedAction; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Comparator; -import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedHashSet; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.SortedMap; -import java.util.TreeMap; -import java.util.TreeSet; -import java.util.concurrent.ConcurrentHashMap; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.classification.InterfaceAudience.Private; -import org.apache.hadoop.classification.InterfaceStability.Unstable; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.ipc.RPC; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; -import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest; -import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; -import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest; -import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest; -import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; -import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; -import org.apache.hadoop.yarn.api.records.NMToken; -import org.apache.hadoop.yarn.api.records.Priority; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.api.records.ResourceRequest; -import org.apache.hadoop.yarn.api.records.Token; -import org.apache.hadoop.yarn.client.AMRMClient.ContainerRequest; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; -import org.apache.hadoop.yarn.factories.RecordFactory; -import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; -import org.apache.hadoop.yarn.ipc.YarnRPC; -import org.apache.hadoop.yarn.util.RackResolver; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Joiner; - -// TODO check inputs for null etc. YARN-654 - -@Unstable -public class AMRMClientImpl extends AMRMClient { - - private static final Log LOG = LogFactory.getLog(AMRMClientImpl.class); - - private final RecordFactory recordFactory = - RecordFactoryProvider.getRecordFactory(null); - - private int lastResponseId = 0; - private ConcurrentHashMap nmTokens; - - protected ApplicationMasterProtocol rmClient; - protected final ApplicationAttemptId appAttemptId; - protected Resource clusterAvailableResources; - protected int clusterNodeCount; - - class ResourceRequestInfo { - ResourceRequest remoteRequest; - LinkedHashSet containerRequests; - - ResourceRequestInfo(Priority priority, String resourceName, - Resource capability) { - remoteRequest = ResourceRequest.newInstance(priority, resourceName, - capability, 0); - containerRequests = new LinkedHashSet(); - } - } - - - /** - * Class compares Resource by memory then cpu in reverse order - */ - class ResourceReverseMemoryThenCpuComparator implements Comparator { - @Override - public int compare(Resource arg0, Resource arg1) { - int mem0 = arg0.getMemory(); - int mem1 = arg1.getMemory(); - int cpu0 = arg0.getVirtualCores(); - int cpu1 = arg1.getVirtualCores(); - if(mem0 == mem1) { - if(cpu0 == cpu1) { - return 0; - } - if(cpu0 < cpu1) { - return 1; - } - return -1; - } - if(mem0 < mem1) { - return 1; - } - return -1; - } - } - - static boolean canFit(Resource arg0, Resource arg1) { - int mem0 = arg0.getMemory(); - int mem1 = arg1.getMemory(); - int cpu0 = arg0.getVirtualCores(); - int cpu1 = arg1.getVirtualCores(); - - if(mem0 <= mem1 && cpu0 <= cpu1) { - return true; - } - return false; - } - - //Key -> Priority - //Value -> Map - //Key->ResourceName (e.g., nodename, rackname, *) - //Value->Map - //Key->Resource Capability - //Value->ResourceRequest - protected final - Map>> - remoteRequestsTable = - new TreeMap>>(); - - protected final Set ask = new TreeSet( - new org.apache.hadoop.yarn.api.records.ResourceRequest.ResourceRequestComparator()); - protected final Set release = new TreeSet(); - - public AMRMClientImpl(ApplicationAttemptId appAttemptId) { - super(AMRMClientImpl.class.getName()); - this.appAttemptId = appAttemptId; - this.nmTokens = new ConcurrentHashMap(); - } - - @Override - protected void serviceInit(Configuration conf) throws Exception { - RackResolver.init(conf); - super.serviceInit(conf); - } - - @Override - protected void serviceStart() throws Exception { - final YarnConfiguration conf = new YarnConfiguration(getConfig()); - final YarnRPC rpc = YarnRPC.create(conf); - final InetSocketAddress rmAddress = conf.getSocketAddr( - YarnConfiguration.RM_SCHEDULER_ADDRESS, - YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS, - YarnConfiguration.DEFAULT_RM_SCHEDULER_PORT); - - UserGroupInformation currentUser; - try { - currentUser = UserGroupInformation.getCurrentUser(); - } catch (IOException e) { - throw new YarnRuntimeException(e); - } - - // CurrentUser should already have AMToken loaded. - rmClient = currentUser.doAs(new PrivilegedAction() { - @Override - public ApplicationMasterProtocol run() { - return (ApplicationMasterProtocol) rpc.getProxy(ApplicationMasterProtocol.class, rmAddress, - conf); - } - }); - LOG.debug("Connecting to ResourceManager at " + rmAddress); - super.serviceStart(); - } - - @Override - protected void serviceStop() throws Exception { - if (this.rmClient != null) { - RPC.stopProxy(this.rmClient); - } - super.serviceStop(); - } - - @Override - public RegisterApplicationMasterResponse registerApplicationMaster( - String appHostName, int appHostPort, String appTrackingUrl) - throws YarnException, IOException { - // do this only once ??? - RegisterApplicationMasterRequest request = recordFactory - .newRecordInstance(RegisterApplicationMasterRequest.class); - synchronized (this) { - request.setApplicationAttemptId(appAttemptId); - } - request.setHost(appHostName); - request.setRpcPort(appHostPort); - if(appTrackingUrl != null) { - request.setTrackingUrl(appTrackingUrl); - } - RegisterApplicationMasterResponse response = rmClient - .registerApplicationMaster(request); - return response; - } - - @Override - public AllocateResponse allocate(float progressIndicator) - throws YarnException, IOException { - AllocateResponse allocateResponse = null; - ArrayList askList = null; - ArrayList releaseList = null; - AllocateRequest allocateRequest = null; - - try { - synchronized (this) { - askList = new ArrayList(ask); - releaseList = new ArrayList(release); - // optimistically clear this collection assuming no RPC failure - ask.clear(); - release.clear(); - allocateRequest = - AllocateRequest.newInstance(appAttemptId, lastResponseId, - progressIndicator, askList, releaseList, null); - } - - allocateResponse = rmClient.allocate(allocateRequest); - - synchronized (this) { - // update these on successful RPC - clusterNodeCount = allocateResponse.getNumClusterNodes(); - lastResponseId = allocateResponse.getResponseId(); - clusterAvailableResources = allocateResponse.getAvailableResources(); - if (!allocateResponse.getNMTokens().isEmpty()) { - populateNMTokens(allocateResponse); - } - } - } finally { - // TODO how to differentiate remote yarn exception vs error in rpc - if(allocateResponse == null) { - // we hit an exception in allocate() - // preserve ask and release for next call to allocate() - synchronized (this) { - release.addAll(releaseList); - // requests could have been added or deleted during call to allocate - // If requests were added/removed then there is nothing to do since - // the ResourceRequest object in ask would have the actual new value. - // If ask does not have this ResourceRequest then it was unchanged and - // so we can add the value back safely. - // This assumes that there will no concurrent calls to allocate() and - // so we dont have to worry about ask being changed in the - // synchronized block at the beginning of this method. - for(ResourceRequest oldAsk : askList) { - if(!ask.contains(oldAsk)) { - ask.add(oldAsk); - } - } - } - } - } - return allocateResponse; - } - - @Private - @VisibleForTesting - protected void populateNMTokens(AllocateResponse allocateResponse) { - for (NMToken token : allocateResponse.getNMTokens()) { - String nodeId = token.getNodeId().toString(); - if (nmTokens.containsKey(nodeId)) { - LOG.debug("Replacing token for : " + nodeId); - } else { - LOG.debug("Received new token for : " + nodeId); - } - nmTokens.put(nodeId, token.getToken()); - } - } - - @Override - public void unregisterApplicationMaster(FinalApplicationStatus appStatus, - String appMessage, String appTrackingUrl) throws YarnException, - IOException { - FinishApplicationMasterRequest request = recordFactory - .newRecordInstance(FinishApplicationMasterRequest.class); - request.setAppAttemptId(appAttemptId); - request.setFinalApplicationStatus(appStatus); - if(appMessage != null) { - request.setDiagnostics(appMessage); - } - if(appTrackingUrl != null) { - request.setTrackingUrl(appTrackingUrl); - } - rmClient.finishApplicationMaster(request); - } - - @Override - public synchronized void addContainerRequest(T req) { - Set allRacks = new HashSet(); - if (req.racks != null) { - allRacks.addAll(req.racks); - if(req.racks.size() != allRacks.size()) { - Joiner joiner = Joiner.on(','); - LOG.warn("ContainerRequest has duplicate racks: " - + joiner.join(req.racks)); - } - } - allRacks.addAll(resolveRacks(req.nodes)); - - if (req.nodes != null) { - HashSet dedupedNodes = new HashSet(req.nodes); - if(dedupedNodes.size() != req.nodes.size()) { - Joiner joiner = Joiner.on(','); - LOG.warn("ContainerRequest has duplicate nodes: " - + joiner.join(req.nodes)); - } - for (String node : dedupedNodes) { - // Ensure node requests are accompanied by requests for - // corresponding rack - addResourceRequest(req.priority, node, req.capability, - req.containerCount, req); - } - } - - for (String rack : allRacks) { - addResourceRequest(req.priority, rack, req.capability, - req.containerCount, req); - } - - // Off-switch - addResourceRequest(req.priority, ResourceRequest.ANY, req.capability, - req.containerCount, req); - } - - @Override - public synchronized void removeContainerRequest(T req) { - Set allRacks = new HashSet(); - if (req.racks != null) { - allRacks.addAll(req.racks); - } - allRacks.addAll(resolveRacks(req.nodes)); - - // Update resource requests - if (req.nodes != null) { - for (String node : new HashSet(req.nodes)) { - decResourceRequest(req.priority, node, req.capability, - req.containerCount, req); - } - } - - for (String rack : allRacks) { - decResourceRequest(req.priority, rack, req.capability, - req.containerCount, req); - } - - decResourceRequest(req.priority, ResourceRequest.ANY, req.capability, - req.containerCount, req); - } - - @Override - public synchronized void releaseAssignedContainer(ContainerId containerId) { - release.add(containerId); - } - - @Override - public synchronized Resource getClusterAvailableResources() { - return clusterAvailableResources; - } - - @Override - public synchronized int getClusterNodeCount() { - return clusterNodeCount; - } - - @Override - public synchronized List> getMatchingRequests( - Priority priority, - String resourceName, - Resource capability) { - List> list = new LinkedList>(); - Map> remoteRequests = - this.remoteRequestsTable.get(priority); - if (remoteRequests == null) { - return list; - } - TreeMap reqMap = remoteRequests - .get(resourceName); - if (reqMap == null) { - return list; - } - - ResourceRequestInfo resourceRequestInfo = reqMap.get(capability); - if (resourceRequestInfo != null) { - list.add(resourceRequestInfo.containerRequests); - return list; - } - - // no exact match. Container may be larger than what was requested. - // get all resources <= capability. map is reverse sorted. - SortedMap tailMap = - reqMap.tailMap(capability); - for(Map.Entry entry : tailMap.entrySet()) { - if(canFit(entry.getKey(), capability)) { - // match found that fits in the larger resource - list.add(entry.getValue().containerRequests); - } - } - - // no match found - return list; - } - - private Set resolveRacks(List nodes) { - Set racks = new HashSet(); - if (nodes != null) { - for (String node : nodes) { - // Ensure node requests are accompanied by requests for - // corresponding rack - String rack = RackResolver.resolve(node).getNetworkLocation(); - if (rack == null) { - LOG.warn("Failed to resolve rack for node " + node + "."); - } else { - racks.add(rack); - } - } - } - - return racks; - } - - private void addResourceRequestToAsk(ResourceRequest remoteRequest) { - // This code looks weird but is needed because of the following scenario. - // A ResourceRequest is removed from the remoteRequestTable. A 0 container - // request is added to 'ask' to notify the RM about not needing it any more. - // Before the call to allocate, the user now requests more containers. If - // the locations of the 0 size request and the new request are the same - // (with the difference being only container count), then the set comparator - // will consider both to be the same and not add the new request to ask. So - // we need to check for the "same" request being present and remove it and - // then add it back. The comparator is container count agnostic. - // This should happen only rarely but we do need to guard against it. - if(ask.contains(remoteRequest)) { - ask.remove(remoteRequest); - } - ask.add(remoteRequest); - } - - private void addResourceRequest(Priority priority, String resourceName, - Resource capability, int containerCount, T req) { - Map> remoteRequests = - this.remoteRequestsTable.get(priority); - if (remoteRequests == null) { - remoteRequests = - new HashMap>(); - this.remoteRequestsTable.put(priority, remoteRequests); - if (LOG.isDebugEnabled()) { - LOG.debug("Added priority=" + priority); - } - } - TreeMap reqMap = - remoteRequests.get(resourceName); - if (reqMap == null) { - // capabilities are stored in reverse sorted order. smallest last. - reqMap = new TreeMap( - new ResourceReverseMemoryThenCpuComparator()); - remoteRequests.put(resourceName, reqMap); - } - ResourceRequestInfo resourceRequestInfo = reqMap.get(capability); - if (resourceRequestInfo == null) { - resourceRequestInfo = - new ResourceRequestInfo(priority, resourceName, capability); - reqMap.put(capability, resourceRequestInfo); - } - - resourceRequestInfo.remoteRequest.setNumContainers( - resourceRequestInfo.remoteRequest.getNumContainers() + containerCount); - - if(req instanceof StoredContainerRequest) { - resourceRequestInfo.containerRequests.add(req); - } - - // Note this down for next interaction with ResourceManager - addResourceRequestToAsk(resourceRequestInfo.remoteRequest); - - if (LOG.isDebugEnabled()) { - LOG.debug("addResourceRequest:" + " applicationId=" - + appAttemptId + " priority=" + priority.getPriority() - + " resourceName=" + resourceName + " numContainers=" - + resourceRequestInfo.remoteRequest.getNumContainers() - + " #asks=" + ask.size()); - } - } - - private void decResourceRequest(Priority priority, - String resourceName, - Resource capability, - int containerCount, - T req) { - Map> remoteRequests = - this.remoteRequestsTable.get(priority); - - if(remoteRequests == null) { - if (LOG.isDebugEnabled()) { - LOG.debug("Not decrementing resource as priority " + priority - + " is not present in request table"); - } - return; - } - - Map reqMap = remoteRequests.get(resourceName); - if (reqMap == null) { - if (LOG.isDebugEnabled()) { - LOG.debug("Not decrementing resource as " + resourceName - + " is not present in request table"); - } - return; - } - ResourceRequestInfo resourceRequestInfo = reqMap.get(capability); - - if (LOG.isDebugEnabled()) { - LOG.debug("BEFORE decResourceRequest:" + " applicationId=" - + appAttemptId + " priority=" + priority.getPriority() - + " resourceName=" + resourceName + " numContainers=" - + resourceRequestInfo.remoteRequest.getNumContainers() - + " #asks=" + ask.size()); - } - - resourceRequestInfo.remoteRequest.setNumContainers( - resourceRequestInfo.remoteRequest.getNumContainers() - containerCount); - - if(req instanceof StoredContainerRequest) { - resourceRequestInfo.containerRequests.remove(req); - } - - if(resourceRequestInfo.remoteRequest.getNumContainers() < 0) { - // guard against spurious removals - resourceRequestInfo.remoteRequest.setNumContainers(0); - } - // send the ResourceRequest to RM even if is 0 because it needs to override - // a previously sent value. If ResourceRequest was not sent previously then - // sending 0 aught to be a no-op on RM - addResourceRequestToAsk(resourceRequestInfo.remoteRequest); - - // delete entries from map if no longer needed - if (resourceRequestInfo.remoteRequest.getNumContainers() == 0) { - reqMap.remove(capability); - if (reqMap.size() == 0) { - remoteRequests.remove(resourceName); - } - if (remoteRequests.size() == 0) { - remoteRequestsTable.remove(priority); - } - } - - if (LOG.isDebugEnabled()) { - LOG.info("AFTER decResourceRequest:" + " applicationId=" - + appAttemptId + " priority=" + priority.getPriority() - + " resourceName=" + resourceName + " numContainers=" - + resourceRequestInfo.remoteRequest.getNumContainers() - + " #asks=" + ask.size()); - } - } - - @Override - public ConcurrentHashMap getNMTokens() { - return nmTokens; - } -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/NMClient.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/NMClient.java deleted file mode 100644 index 9dd3ebd..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/NMClient.java +++ /dev/null @@ -1,122 +0,0 @@ -/** - * 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.hadoop.yarn.client; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.Map; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceAudience.Private; -import org.apache.hadoop.classification.InterfaceAudience.Public; -import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.service.AbstractService; -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; -import org.apache.hadoop.yarn.api.records.ContainerStatus; -import org.apache.hadoop.yarn.api.records.NodeId; -import org.apache.hadoop.yarn.api.records.Token; -import org.apache.hadoop.yarn.exceptions.YarnException; - -@InterfaceAudience.Public -@InterfaceStability.Unstable -public abstract class NMClient extends AbstractService { - - /** - * Create a new instance of NMClient. - */ - @Public - public static NMClient createNMClient() { - NMClient client = new NMClientImpl(); - return client; - } - - /** - * Create a new instance of NMClient. - */ - @Public - public static NMClient createNMClient(String name) { - NMClient client = new NMClientImpl(name); - return client; - } - - @Private - protected NMClient(String name) { - super(name); - } - - /** - *

Start an allocated container.

- * - *

The ApplicationMaster or other applications that use the - * client must provide the details of the allocated container, including the - * Id, the assigned node's Id and the token via {@link Container}. In - * addition, the AM needs to provide the {@link ContainerLaunchContext} as - * well.

- * - * @param container the allocated container - * @param containerLaunchContext the context information needed by the - * NodeManager to launch the - * container - * @return a map between the auxiliary service names and their outputs - * @throws YarnException - * @throws IOException - */ - public abstract Map startContainer(Container container, - ContainerLaunchContext containerLaunchContext) - throws YarnException, IOException; - - /** - *

Stop an started container.

- * - * @param containerId the Id of the started container - * @param nodeId the Id of the NodeManager - * @param containerToken the security token to verify authenticity of the - * started container - * @throws YarnException - * @throws IOException - */ - public abstract void stopContainer(ContainerId containerId, NodeId nodeId, - Token containerToken) throws YarnException, IOException; - - /** - *

Query the status of a container.

- * - * @param containerId the Id of the started container - * @param nodeId the Id of the NodeManager - * @param containerToken the security token to verify authenticity of the - * started container - * @return the status of a container - * @throws YarnException - * @throws IOException - */ - public abstract ContainerStatus getContainerStatus(ContainerId containerId, NodeId nodeId, - Token containerToken) throws YarnException, IOException; - - /** - *

Set whether the containers that are started by this client, and are - * still running should be stopped when the client stops. By default, the - * feature should be enabled.

- * - * @param enabled whether the feature is enabled or not - */ - public abstract void cleanupRunningContainersOnStop(boolean enabled); - -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/NMClientAsync.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/NMClientAsync.java deleted file mode 100644 index 3b8fda2..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/NMClientAsync.java +++ /dev/null @@ -1,720 +0,0 @@ -/** - * 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.hadoop.yarn.client; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.EnumSet; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.ThreadFactory; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.locks.ReentrantReadWriteLock; -import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock; -import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.classification.InterfaceAudience.Private; -import org.apache.hadoop.classification.InterfaceStability.Evolving; -import org.apache.hadoop.classification.InterfaceStability.Unstable; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.service.AbstractService; -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; -import org.apache.hadoop.yarn.api.records.ContainerStatus; -import org.apache.hadoop.yarn.api.records.Token; -import org.apache.hadoop.yarn.api.records.NodeId; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.event.AbstractEvent; -import org.apache.hadoop.yarn.event.EventHandler; -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.hadoop.yarn.ipc.RPCUtil; -import org.apache.hadoop.yarn.state.InvalidStateTransitonException; -import org.apache.hadoop.yarn.state.MultipleArcTransition; -import org.apache.hadoop.yarn.state.SingleArcTransition; -import org.apache.hadoop.yarn.state.StateMachine; -import org.apache.hadoop.yarn.state.StateMachineFactory; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.util.concurrent.ThreadFactoryBuilder; - -/** - * NMClientAsync handles communication with all the NodeManagers - * and provides asynchronous updates on getting responses from them. It - * maintains a thread pool to communicate with individual NMs where a number of - * worker threads process requests to NMs by using {@link NMClientImpl}. The max - * size of the thread pool is configurable through - * {@link YarnConfiguration#NM_CLIENT_ASYNC_THREAD_POOL_MAX_SIZE}. - * - * It should be used in conjunction with a CallbackHandler. For example - * - *
- * {@code
- * class MyCallbackHandler implements NMClientAsync.CallbackHandler {
- *   public void onContainerStarted(ContainerId containerId,
- *       Map allServiceResponse) {
- *     [post process after the container is started, process the response]
- *   }
- *
- *   public void onContainerStatusReceived(ContainerId containerId,
- *       ContainerStatus containerStatus) {
- *     [make use of the status of the container]
- *   }
- *
- *   public void onContainerStopped(ContainerId containerId) {
- *     [post process after the container is stopped]
- *   }
- *
- *   public void onStartContainerError(
- *       ContainerId containerId, Throwable t) {
- *     [handle the raised exception]
- *   }
- *
- *   public void onGetContainerStatusError(
- *       ContainerId containerId, Throwable t) {
- *     [handle the raised exception]
- *   }
- *
- *   public void onStopContainerError(
- *       ContainerId containerId, Throwable t) {
- *     [handle the raised exception]
- *   }
- * }
- * }
- * 
- * - * The client's life-cycle should be managed like the following: - * - *
- * {@code
- * NMClientAsync asyncClient = new NMClientAsync(new MyCallbackhandler());
- * asyncClient.init(conf);
- * asyncClient.start();
- * asyncClient.startContainer(container, containerLaunchContext);
- * [... wait for container being started]
- * asyncClient.getContainerStatus(container.getId(), container.getNodeId(),
- *     container.getContainerToken());
- * [... handle the status in the callback instance]
- * asyncClient.stopContainer(container.getId(), container.getNodeId(),
- *     container.getContainerToken());
- * [... wait for container being stopped]
- * asyncClient.stop();
- * }
- * 
- */ -@Unstable -@Evolving -public class NMClientAsync extends AbstractService { - - private static final Log LOG = LogFactory.getLog(NMClientAsync.class); - - protected static final int INITIAL_THREAD_POOL_SIZE = 10; - - protected ThreadPoolExecutor threadPool; - protected int maxThreadPoolSize; - protected Thread eventDispatcherThread; - protected AtomicBoolean stopped = new AtomicBoolean(false); - protected BlockingQueue events = - new LinkedBlockingQueue(); - - protected NMClient client; - protected CallbackHandler callbackHandler; - - protected ConcurrentMap containers = - new ConcurrentHashMap(); - - public NMClientAsync(CallbackHandler callbackHandler) { - this (NMClientAsync.class.getName(), callbackHandler); - } - - public NMClientAsync(String name, CallbackHandler callbackHandler) { - this (name, new NMClientImpl(), callbackHandler); - } - - @Private - @VisibleForTesting - protected NMClientAsync(String name, NMClient client, - CallbackHandler callbackHandler) { - super(name); - this.client = client; - this.callbackHandler = callbackHandler; - } - - @Override - protected void serviceInit(Configuration conf) throws Exception { - this.maxThreadPoolSize = conf.getInt( - YarnConfiguration.NM_CLIENT_ASYNC_THREAD_POOL_MAX_SIZE, - YarnConfiguration.DEFAULT_NM_CLIENT_ASYNC_THREAD_POOL_MAX_SIZE); - LOG.info("Upper bound of the thread pool size is " + maxThreadPoolSize); - - client.init(conf); - super.serviceInit(conf); - } - - @Override - protected void serviceStart() throws Exception { - client.start(); - - ThreadFactory tf = new ThreadFactoryBuilder().setNameFormat( - this.getClass().getName() + " #%d").setDaemon(true).build(); - - // Start with a default core-pool size and change it dynamically. - int initSize = Math.min(INITIAL_THREAD_POOL_SIZE, maxThreadPoolSize); - threadPool = new ThreadPoolExecutor(initSize, Integer.MAX_VALUE, 1, - TimeUnit.HOURS, new LinkedBlockingQueue(), tf); - - eventDispatcherThread = new Thread() { - @Override - public void run() { - ContainerEvent event = null; - Set allNodes = new HashSet(); - - while (!stopped.get() && !Thread.currentThread().isInterrupted()) { - try { - event = events.take(); - } catch (InterruptedException e) { - if (!stopped.get()) { - LOG.error("Returning, thread interrupted", e); - } - return; - } - - allNodes.add(event.getNodeId().toString()); - - int threadPoolSize = threadPool.getCorePoolSize(); - - // We can increase the pool size only if haven't reached the maximum - // limit yet. - if (threadPoolSize != maxThreadPoolSize) { - - // nodes where containers will run at *this* point of time. This is - // *not* the cluster size and doesn't need to be. - int nodeNum = allNodes.size(); - int idealThreadPoolSize = Math.min(maxThreadPoolSize, nodeNum); - - if (threadPoolSize < idealThreadPoolSize) { - // Bump up the pool size to idealThreadPoolSize + - // INITIAL_POOL_SIZE, the later is just a buffer so we are not - // always increasing the pool-size - int newThreadPoolSize = Math.min(maxThreadPoolSize, - idealThreadPoolSize + INITIAL_THREAD_POOL_SIZE); - LOG.info("Set NMClientAsync thread pool size to " + - newThreadPoolSize + " as the number of nodes to talk to is " - + nodeNum); - threadPool.setCorePoolSize(newThreadPoolSize); - } - } - - // the events from the queue are handled in parallel with a thread - // pool - threadPool.execute(getContainerEventProcessor(event)); - - // TODO: Group launching of multiple containers to a single - // NodeManager into a single connection - } - } - }; - eventDispatcherThread.setName("Container Event Dispatcher"); - eventDispatcherThread.setDaemon(false); - eventDispatcherThread.start(); - - super.serviceStart(); - } - - @Override - protected void serviceStop() throws Exception { - if (stopped.getAndSet(true)) { - // return if already stopped - return; - } - if (eventDispatcherThread != null) { - eventDispatcherThread.interrupt(); - try { - eventDispatcherThread.join(); - } catch (InterruptedException e) { - LOG.error("The thread of " + eventDispatcherThread.getName() + - " didn't finish normally.", e); - } - } - if (threadPool != null) { - threadPool.shutdownNow(); - } - if (client != null) { - // If NMClientImpl doesn't stop running containers, the states doesn't - // need to be cleared. - if (!(client instanceof NMClientImpl) || - ((NMClientImpl) client).cleanupRunningContainers.get()) { - if (containers != null) { - containers.clear(); - } - } - client.stop(); - } - super.serviceStop(); - } - - public void startContainer( - Container container, ContainerLaunchContext containerLaunchContext) { - if (containers.putIfAbsent(container.getId(), - new StatefulContainer(this, container.getId())) != null) { - callbackHandler.onStartContainerError(container.getId(), - RPCUtil.getRemoteException("Container " + container.getId() + - " is already started or scheduled to start")); - } - try { - events.put(new StartContainerEvent(container, containerLaunchContext)); - } catch (InterruptedException e) { - LOG.warn("Exception when scheduling the event of starting Container " + - container.getId()); - callbackHandler.onStartContainerError(container.getId(), e); - } - } - - public void stopContainer(ContainerId containerId, NodeId nodeId, - Token containerToken) { - if (containers.get(containerId) == null) { - callbackHandler.onStopContainerError(containerId, - RPCUtil.getRemoteException("Container " + containerId + - " is neither started nor scheduled to start")); - } - try { - events.put(new ContainerEvent(containerId, nodeId, containerToken, - ContainerEventType.STOP_CONTAINER)); - } catch (InterruptedException e) { - LOG.warn("Exception when scheduling the event of stopping Container " + - containerId); - callbackHandler.onStopContainerError(containerId, e); - } - } - - public void getContainerStatus(ContainerId containerId, NodeId nodeId, - Token containerToken) { - try { - events.put(new ContainerEvent(containerId, nodeId, containerToken, - ContainerEventType.QUERY_CONTAINER)); - } catch (InterruptedException e) { - LOG.warn("Exception when scheduling the event of querying the status" + - " of Container " + containerId); - callbackHandler.onGetContainerStatusError(containerId, e); - } - } - - protected static enum ContainerState { - PREP, FAILED, RUNNING, DONE, - } - - protected boolean isCompletelyDone(StatefulContainer container) { - return container.getState() == ContainerState.DONE || - container.getState() == ContainerState.FAILED; - } - - protected ContainerEventProcessor getContainerEventProcessor( - ContainerEvent event) { - return new ContainerEventProcessor(event); - } - - /** - * The type of the event of interacting with a container - */ - protected static enum ContainerEventType { - START_CONTAINER, - STOP_CONTAINER, - QUERY_CONTAINER - } - - protected static class ContainerEvent - extends AbstractEvent{ - private ContainerId containerId; - private NodeId nodeId; - private Token containerToken; - - public ContainerEvent(ContainerId containerId, NodeId nodeId, - Token containerToken, ContainerEventType type) { - super(type); - this.containerId = containerId; - this.nodeId = nodeId; - this.containerToken = containerToken; - } - - public ContainerId getContainerId() { - return containerId; - } - - public NodeId getNodeId() { - return nodeId; - } - - public Token getContainerToken() { - return containerToken; - } - } - - protected static class StartContainerEvent extends ContainerEvent { - private Container container; - private ContainerLaunchContext containerLaunchContext; - - public StartContainerEvent(Container container, - ContainerLaunchContext containerLaunchContext) { - super(container.getId(), container.getNodeId(), - container.getContainerToken(), ContainerEventType.START_CONTAINER); - this.container = container; - this.containerLaunchContext = containerLaunchContext; - } - - public Container getContainer() { - return container; - } - - public ContainerLaunchContext getContainerLaunchContext() { - return containerLaunchContext; - } - } - - protected static class StatefulContainer implements - EventHandler { - - protected final static StateMachineFactory stateMachineFactory - = new StateMachineFactory(ContainerState.PREP) - - // Transitions from PREP state - .addTransition(ContainerState.PREP, - EnumSet.of(ContainerState.RUNNING, ContainerState.FAILED), - ContainerEventType.START_CONTAINER, - new StartContainerTransition()) - .addTransition(ContainerState.PREP, ContainerState.DONE, - ContainerEventType.STOP_CONTAINER, new OutOfOrderTransition()) - - // Transitions from RUNNING state - // RUNNING -> RUNNING should be the invalid transition - .addTransition(ContainerState.RUNNING, - EnumSet.of(ContainerState.DONE, ContainerState.FAILED), - ContainerEventType.STOP_CONTAINER, - new StopContainerTransition()) - - // Transition from DONE state - .addTransition(ContainerState.DONE, ContainerState.DONE, - EnumSet.of(ContainerEventType.START_CONTAINER, - ContainerEventType.STOP_CONTAINER)) - - // Transition from FAILED state - .addTransition(ContainerState.FAILED, ContainerState.FAILED, - EnumSet.of(ContainerEventType.START_CONTAINER, - ContainerEventType.STOP_CONTAINER)); - - protected static class StartContainerTransition implements - MultipleArcTransition { - - @Override - public ContainerState transition( - StatefulContainer container, ContainerEvent event) { - ContainerId containerId = event.getContainerId(); - try { - StartContainerEvent scEvent = null; - if (event instanceof StartContainerEvent) { - scEvent = (StartContainerEvent) event; - } - assert scEvent != null; - Map allServiceResponse = - container.nmClientAsync.client.startContainer( - scEvent.getContainer(), scEvent.getContainerLaunchContext()); - try { - container.nmClientAsync.callbackHandler.onContainerStarted( - containerId, allServiceResponse); - } catch (Throwable thr) { - // Don't process user created unchecked exception - LOG.info("Unchecked exception is thrown from onContainerStarted for " - + "Container " + containerId, thr); - } - return ContainerState.RUNNING; - } catch (YarnException e) { - return onExceptionRaised(container, event, e); - } catch (IOException e) { - return onExceptionRaised(container, event, e); - } catch (Throwable t) { - return onExceptionRaised(container, event, t); - } - } - - private ContainerState onExceptionRaised(StatefulContainer container, - ContainerEvent event, Throwable t) { - try { - container.nmClientAsync.callbackHandler.onStartContainerError( - event.getContainerId(), t); - } catch (Throwable thr) { - // Don't process user created unchecked exception - LOG.info( - "Unchecked exception is thrown from onStartContainerError for " + - "Container " + event.getContainerId(), thr); - } - return ContainerState.FAILED; - } - } - - protected static class StopContainerTransition implements - MultipleArcTransition { - - @Override - public ContainerState transition( - StatefulContainer container, ContainerEvent event) { - ContainerId containerId = event.getContainerId(); - try { - container.nmClientAsync.client.stopContainer( - containerId, event.getNodeId(), event.getContainerToken()); - try { - container.nmClientAsync.callbackHandler.onContainerStopped( - event.getContainerId()); - } catch (Throwable thr) { - // Don't process user created unchecked exception - LOG.info("Unchecked exception is thrown from onContainerStopped for " - + "Container " + event.getContainerId(), thr); - } - return ContainerState.DONE; - } catch (YarnException e) { - return onExceptionRaised(container, event, e); - } catch (IOException e) { - return onExceptionRaised(container, event, e); - } catch (Throwable t) { - return onExceptionRaised(container, event, t); - } - } - - private ContainerState onExceptionRaised(StatefulContainer container, - ContainerEvent event, Throwable t) { - try { - container.nmClientAsync.callbackHandler.onStopContainerError( - event.getContainerId(), t); - } catch (Throwable thr) { - // Don't process user created unchecked exception - LOG.info("Unchecked exception is thrown from onStopContainerError for " - + "Container " + event.getContainerId(), thr); - } - return ContainerState.FAILED; - } - } - - protected static class OutOfOrderTransition implements - SingleArcTransition { - - protected static final String STOP_BEFORE_START_ERROR_MSG = - "Container was killed before it was launched"; - - @Override - public void transition(StatefulContainer container, ContainerEvent event) { - try { - container.nmClientAsync.callbackHandler.onStartContainerError( - event.getContainerId(), - RPCUtil.getRemoteException(STOP_BEFORE_START_ERROR_MSG)); - } catch (Throwable thr) { - // Don't process user created unchecked exception - LOG.info( - "Unchecked exception is thrown from onStartContainerError for " + - "Container " + event.getContainerId(), thr); - } - } - } - - private final NMClientAsync nmClientAsync; - private final ContainerId containerId; - private final StateMachine stateMachine; - private final ReadLock readLock; - private final WriteLock writeLock; - - public StatefulContainer(NMClientAsync client, ContainerId containerId) { - this.nmClientAsync = client; - this.containerId = containerId; - stateMachine = stateMachineFactory.make(this); - ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); - readLock = lock.readLock(); - writeLock = lock.writeLock(); - } - - @Override - public void handle(ContainerEvent event) { - writeLock.lock(); - try { - try { - this.stateMachine.doTransition(event.getType(), event); - } catch (InvalidStateTransitonException e) { - LOG.error("Can't handle this event at current state", e); - } - } finally { - writeLock.unlock(); - } - } - - public ContainerId getContainerId() { - return containerId; - } - - public ContainerState getState() { - readLock.lock(); - try { - return stateMachine.getCurrentState(); - } finally { - readLock.unlock(); - } - } - } - - protected class ContainerEventProcessor implements Runnable { - protected ContainerEvent event; - - public ContainerEventProcessor(ContainerEvent event) { - this.event = event; - } - - @Override - public void run() { - ContainerId containerId = event.getContainerId(); - LOG.info("Processing Event " + event + " for Container " + containerId); - if (event.getType() == ContainerEventType.QUERY_CONTAINER) { - try { - ContainerStatus containerStatus = client.getContainerStatus( - containerId, event.getNodeId(), event.getContainerToken()); - try { - callbackHandler.onContainerStatusReceived( - containerId, containerStatus); - } catch (Throwable thr) { - // Don't process user created unchecked exception - LOG.info( - "Unchecked exception is thrown from onContainerStatusReceived" + - " for Container " + event.getContainerId(), thr); - } - } catch (YarnException e) { - onExceptionRaised(containerId, e); - } catch (IOException e) { - onExceptionRaised(containerId, e); - } catch (Throwable t) { - onExceptionRaised(containerId, t); - } - } else { - StatefulContainer container = containers.get(containerId); - if (container == null) { - LOG.info("Container " + containerId + " is already stopped or failed"); - } else { - container.handle(event); - if (isCompletelyDone(container)) { - containers.remove(containerId); - } - } - } - } - - private void onExceptionRaised(ContainerId containerId, Throwable t) { - try { - callbackHandler.onGetContainerStatusError(containerId, t); - } catch (Throwable thr) { - // Don't process user created unchecked exception - LOG.info("Unchecked exception is thrown from onGetContainerStatusError" + - " for Container " + containerId, thr); - } - } - } - - /** - *

- * The callback interface needs to be implemented by {@link NMClientAsync} - * users. The APIs are called when responses from NodeManager are - * available. - *

- * - *

- * Once a callback happens, the users can chose to act on it in blocking or - * non-blocking manner. If the action on callback is done in a blocking - * manner, some of the threads performing requests on NodeManagers may get - * blocked depending on how many threads in the pool are busy. - *

- * - *

- * The implementation of the callback function should not throw the - * unexpected exception. Otherwise, {@link NMClientAsync} will just - * catch, log and then ignore it. - *

- */ - public static interface CallbackHandler { - /** - * The API is called when NodeManager responds to indicate its - * acceptance of the starting container request - * @param containerId the Id of the container - * @param allServiceResponse a Map between the auxiliary service names and - * their outputs - */ - void onContainerStarted(ContainerId containerId, - Map allServiceResponse); - - /** - * The API is called when NodeManager responds with the status - * of the container - * @param containerId the Id of the container - * @param containerStatus the status of the container - */ - void onContainerStatusReceived(ContainerId containerId, - ContainerStatus containerStatus); - - /** - * The API is called when NodeManager responds to indicate the - * container is stopped. - * @param containerId the Id of the container - */ - void onContainerStopped(ContainerId containerId); - - /** - * The API is called when an exception is raised in the process of - * starting a container - * - * @param containerId the Id of the container - * @param t the raised exception - */ - void onStartContainerError(ContainerId containerId, Throwable t); - - /** - * The API is called when an exception is raised in the process of - * querying the status of a container - * - * @param containerId the Id of the container - * @param t the raised exception - */ - void onGetContainerStatusError(ContainerId containerId, Throwable t); - - /** - * The API is called when an exception is raised in the process of - * stopping a container - * - * @param containerId the Id of the container - * @param t the raised exception - */ - void onStopContainerError(ContainerId containerId, Throwable t); - - } - -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/NMClientImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/NMClientImpl.java deleted file mode 100644 index 042af51..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/NMClientImpl.java +++ /dev/null @@ -1,400 +0,0 @@ -/** - * 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.hadoop.yarn.client; - -import java.io.IOException; -import java.net.InetSocketAddress; -import java.nio.ByteBuffer; -import java.security.PrivilegedAction; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.atomic.AtomicBoolean; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.ipc.RPC; -import org.apache.hadoop.net.NetUtils; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.service.AbstractService; -import org.apache.hadoop.yarn.api.ContainerManagementProtocol; -import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest; -import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse; -import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; -import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse; -import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest; -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; -import org.apache.hadoop.yarn.api.records.ContainerStatus; -import org.apache.hadoop.yarn.api.records.NodeId; -import org.apache.hadoop.yarn.api.records.Token; -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.hadoop.yarn.ipc.RPCUtil; -import org.apache.hadoop.yarn.ipc.YarnRPC; -import org.apache.hadoop.yarn.security.ContainerTokenIdentifier; -import org.apache.hadoop.yarn.util.ProtoUtils; -import org.apache.hadoop.yarn.util.Records; - -/** - *

- * This class implements {@link NMClient}. All the APIs are blocking. - *

- * - *

- * By default, this client stops all the running containers that are started by - * it when it stops. It can be disabled via - * {@link #cleanupRunningContainersOnStop}, in which case containers will - * continue to run even after this client is stopped and till the application - * runs at which point ResourceManager will forcefully kill them. - *

- * - *

- * Note that the blocking APIs ensure the RPC calls to NodeManager - * are executed immediately, and the responses are received before these APIs - * return. However, when {@link #startContainer} or {@link #stopContainer} - * returns, NodeManager may still need some time to either start - * or stop the container because of its asynchronous implementation. Therefore, - * {@link #getContainerStatus} is likely to return a transit container status - * if it is executed immediately after {@link #startContainer} or - * {@link #stopContainer}. - *

- */ -public class NMClientImpl extends NMClient { - - private static final Log LOG = LogFactory.getLog(NMClientImpl.class); - - // The logically coherent operations on startedContainers is synchronized to - // ensure they are atomic - protected ConcurrentMap startedContainers = - new ConcurrentHashMap(); - - //enabled by default - protected final AtomicBoolean cleanupRunningContainers = new AtomicBoolean(true); - - public NMClientImpl() { - super(NMClientImpl.class.getName()); - } - - public NMClientImpl(String name) { - super(name); - } - - @Override - protected void serviceStop() throws Exception { - // Usually, started-containers are stopped when this client stops. Unless - // the flag cleanupRunningContainers is set to false. - if (cleanupRunningContainers.get()) { - cleanupRunningContainers(); - } - super.serviceStop(); - } - - protected synchronized void cleanupRunningContainers() { - for (StartedContainer startedContainer : startedContainers.values()) { - try { - stopContainer(startedContainer.getContainerId(), - startedContainer.getNodeId(), - startedContainer.getContainerToken()); - } catch (YarnException e) { - LOG.error("Failed to stop Container " + - startedContainer.getContainerId() + - "when stopping NMClientImpl"); - } catch (IOException e) { - LOG.error("Failed to stop Container " + - startedContainer.getContainerId() + - "when stopping NMClientImpl"); - } - } - } - - @Override - public void cleanupRunningContainersOnStop(boolean enabled) { - cleanupRunningContainers.set(enabled); - } - - protected static class StartedContainer { - private ContainerId containerId; - private NodeId nodeId; - private Token containerToken; - private boolean stopped; - - public StartedContainer(ContainerId containerId, NodeId nodeId, - Token containerToken) { - this.containerId = containerId; - this.nodeId = nodeId; - this.containerToken = containerToken; - stopped = false; - } - - public ContainerId getContainerId() { - return containerId; - } - - public NodeId getNodeId() { - return nodeId; - } - - public Token getContainerToken() { - return containerToken; - } - } - - protected static final class NMCommunicator extends AbstractService { - private ContainerId containerId; - private NodeId nodeId; - private Token containerToken; - private ContainerManagementProtocol containerManager; - - public NMCommunicator(ContainerId containerId, NodeId nodeId, - Token containerToken) { - super(NMCommunicator.class.getName()); - this.containerId = containerId; - this.nodeId = nodeId; - this.containerToken = containerToken; - } - - @Override - protected void serviceStart() throws Exception { - final YarnRPC rpc = YarnRPC.create(getConfig()); - - final InetSocketAddress containerAddress = - NetUtils.createSocketAddr(nodeId.toString()); - - // the user in createRemoteUser in this context has to be ContainerId - UserGroupInformation currentUser = - UserGroupInformation.createRemoteUser(containerId.toString()); - - org.apache.hadoop.security.token.Token token = - ProtoUtils.convertFromProtoFormat(containerToken, containerAddress); - currentUser.addToken(token); - - containerManager = currentUser - .doAs(new PrivilegedAction() { - @Override - public ContainerManagementProtocol run() { - return (ContainerManagementProtocol) rpc.getProxy(ContainerManagementProtocol.class, - containerAddress, getConfig()); - } - }); - - LOG.debug("Connecting to ContainerManager at " + containerAddress); - super.serviceStart(); - } - - @Override - protected void serviceStop() throws Exception { - if (this.containerManager != null) { - RPC.stopProxy(this.containerManager); - - if (LOG.isDebugEnabled()) { - InetSocketAddress containerAddress = - NetUtils.createSocketAddr(nodeId.toString()); - LOG.debug("Disconnecting from ContainerManager at " + - containerAddress); - } - } - super.serviceStop(); - } - - public synchronized Map startContainer( - Container container, ContainerLaunchContext containerLaunchContext) - throws YarnException, IOException { - if (!container.getId().equals(containerId)) { - throw new IllegalArgumentException( - "NMCommunicator's containerId mismatches the given Container's"); - } - StartContainerResponse startResponse = null; - try { - StartContainerRequest startRequest = - Records.newRecord(StartContainerRequest.class); - startRequest.setContainerToken(container.getContainerToken()); - startRequest.setContainerLaunchContext(containerLaunchContext); - startResponse = containerManager.startContainer(startRequest); - if (LOG.isDebugEnabled()) { - LOG.debug("Started Container " + containerId); - } - } catch (YarnException e) { - LOG.warn("Container " + containerId + " failed to start", e); - throw e; - } catch (IOException e) { - LOG.warn("Container " + containerId + " failed to start", e); - throw e; - } - return startResponse.getAllServiceResponse(); - } - - public synchronized void stopContainer() throws YarnException, - IOException { - try { - StopContainerRequest stopRequest = - Records.newRecord(StopContainerRequest.class); - stopRequest.setContainerId(containerId); - containerManager.stopContainer(stopRequest); - if (LOG.isDebugEnabled()) { - LOG.debug("Stopped Container " + containerId); - } - } catch (YarnException e) { - LOG.warn("Container " + containerId + " failed to stop", e); - throw e; - } catch (IOException e) { - LOG.warn("Container " + containerId + " failed to stop", e); - throw e; - } - } - - public synchronized ContainerStatus getContainerStatus() - throws YarnException, IOException { - GetContainerStatusResponse statusResponse = null; - try { - GetContainerStatusRequest statusRequest = - Records.newRecord(GetContainerStatusRequest.class); - statusRequest.setContainerId(containerId); - statusResponse = containerManager.getContainerStatus(statusRequest); - if (LOG.isDebugEnabled()) { - LOG.debug("Got the status of Container " + containerId); - } - } catch (YarnException e) { - LOG.warn( - "Unable to get the status of Container " + containerId, e); - throw e; - } catch (IOException e) { - LOG.warn( - "Unable to get the status of Container " + containerId, e); - throw e; - } - return statusResponse.getStatus(); - } - } - - @Override - public Map startContainer( - Container container, ContainerLaunchContext containerLaunchContext) - throws YarnException, IOException { - // Do synchronization on StartedContainer to prevent race condition - // between startContainer and stopContainer - synchronized (addStartedContainer(container)) { - Map allServiceResponse; - NMCommunicator nmCommunicator = null; - try { - nmCommunicator = new NMCommunicator(container.getId(), - container.getNodeId(), container.getContainerToken()); - nmCommunicator.init(getConfig()); - nmCommunicator.start(); - allServiceResponse = - nmCommunicator.startContainer(container, containerLaunchContext); - } catch (YarnException e) { - // Remove the started container if it failed to start - removeStartedContainer(container.getId()); - throw e; - } catch (IOException e) { - removeStartedContainer(container.getId()); - throw e; - } catch (Throwable t) { - removeStartedContainer(container.getId()); - throw RPCUtil.getRemoteException(t); - } finally { - if (nmCommunicator != null) { - nmCommunicator.stop(); - } - } - return allServiceResponse; - } - - // Three choices: - // 1. starting and releasing the proxy before and after each interaction - // 2. starting the proxy when starting the container and releasing it when - // stopping the container - // 3. starting the proxy when starting the container and releasing it when - // stopping the client - // Adopt 1 currently - } - - @Override - public void stopContainer(ContainerId containerId, NodeId nodeId, - Token containerToken) throws YarnException, IOException { - StartedContainer startedContainer = getStartedContainer(containerId); - if (startedContainer == null) { - throw RPCUtil.getRemoteException("Container " + containerId + - " is either not started yet or already stopped"); - } - // Only allow one request of stopping the container to move forward - // When entering the block, check whether the precursor has already stopped - // the container - synchronized (startedContainer) { - if (startedContainer.stopped) { - return; - } - NMCommunicator nmCommunicator = null; - try { - nmCommunicator = - new NMCommunicator(containerId, nodeId, containerToken); - nmCommunicator.init(getConfig()); - nmCommunicator.start(); - nmCommunicator.stopContainer(); - } finally { - if (nmCommunicator != null) { - nmCommunicator.stop(); - } - startedContainer.stopped = true; - removeStartedContainer(containerId); - } - } - } - - @Override - public ContainerStatus getContainerStatus(ContainerId containerId, - NodeId nodeId, Token containerToken) - throws YarnException, IOException { - NMCommunicator nmCommunicator = null; - try { - nmCommunicator = new NMCommunicator(containerId, nodeId, containerToken); - nmCommunicator.init(getConfig()); - nmCommunicator.start(); - ContainerStatus containerStatus = nmCommunicator.getContainerStatus(); - return containerStatus; - } finally { - if (nmCommunicator != null) { - nmCommunicator.stop(); - } - } - } - - protected synchronized StartedContainer addStartedContainer( - Container container) throws YarnException, IOException { - if (startedContainers.containsKey(container.getId())) { - throw RPCUtil.getRemoteException("Container " + container.getId() + - " is already started"); - } - StartedContainer startedContainer = new StartedContainer(container.getId(), - container.getNodeId(), container.getContainerToken()); - startedContainers.put(startedContainer.getContainerId(), startedContainer); - return startedContainer; - } - - protected synchronized void removeStartedContainer(ContainerId containerId) { - startedContainers.remove(containerId); - } - - protected synchronized StartedContainer getStartedContainer( - ContainerId containerId) { - return startedContainers.get(containerId); - } - -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/YarnClient.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/YarnClient.java deleted file mode 100644 index 57ef805..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/YarnClient.java +++ /dev/null @@ -1,294 +0,0 @@ -/** -* 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.hadoop.yarn.client; - -import java.io.IOException; -import java.net.InetSocketAddress; -import java.util.List; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceAudience.Private; -import org.apache.hadoop.classification.InterfaceAudience.Public; -import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.service.AbstractService; -import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.ApplicationReport; -import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; -import org.apache.hadoop.yarn.api.records.NodeReport; -import org.apache.hadoop.yarn.api.records.QueueInfo; -import org.apache.hadoop.yarn.api.records.QueueUserACLInfo; -import org.apache.hadoop.yarn.api.records.Token; -import org.apache.hadoop.yarn.api.records.YarnClusterMetrics; -import org.apache.hadoop.yarn.exceptions.YarnException; - -@InterfaceAudience.Public -@InterfaceStability.Evolving -public abstract class YarnClient extends AbstractService { - - /** - * Create a new instance of YarnClient. - */ - @Public - public static YarnClient createYarnClient() { - YarnClient client = new YarnClientImpl(); - return client; - } - - /** - * Create a new instance of YarnClient. - */ - @Public - public static YarnClient createYarnClient(InetSocketAddress rmAddress) { - YarnClient client = new YarnClientImpl(rmAddress); - return client; - } - - /** - * Create a new instance of YarnClient. - */ - @Public - public static YarnClient createYarnClient(String name, - InetSocketAddress rmAddress) { - YarnClient client = new YarnClientImpl(name, rmAddress); - return client; - } - - @Private - protected YarnClient(String name) { - super(name); - } - - /** - *

- * Obtain a new {@link ApplicationId} for submitting new applications. - *

- * - *

- * Returns a response which contains {@link ApplicationId} that can be used to - * submit a new application. See - * {@link #submitApplication(ApplicationSubmissionContext)}. - *

- * - *

- * See {@link GetNewApplicationResponse} for other information that is - * returned. - *

- * - * @return response containing the new ApplicationId to be used - * to submit an application - * @throws YarnException - * @throws IOException - */ - public abstract GetNewApplicationResponse getNewApplication() throws YarnException, - IOException; - - /** - *

- * Submit a new application to YARN. It is a blocking call, such - * that it will not return {@link ApplicationId} until the submitted - * application has been submitted and accepted by the ResourceManager. - *

- * - * @param appContext - * {@link ApplicationSubmissionContext} containing all the details - * needed to submit a new application - * @return {@link ApplicationId} of the accepted application - * @throws YarnException - * @throws IOException - * @see #getNewApplication() - */ - public abstract ApplicationId submitApplication(ApplicationSubmissionContext appContext) - throws YarnException, IOException; - - /** - *

- * Kill an application identified by given ID. - *

- * - * @param applicationId - * {@link ApplicationId} of the application that needs to be killed - * @throws YarnException - * in case of errors or if YARN rejects the request due to - * access-control restrictions. - * @throws IOException - * @see #getQueueAclsInfo() - */ - public abstract void killApplication(ApplicationId applicationId) throws YarnException, - IOException; - - /** - *

- * Get a report of the given Application. - *

- * - *

- * In secure mode, YARN verifies access to the application, queue - * etc. before accepting the request. - *

- * - *

- * If the user does not have VIEW_APP access then the following - * fields in the report will be set to stubbed values: - *

    - *
  • host - set to "N/A"
  • - *
  • RPC port - set to -1
  • - *
  • client token - set to "N/A"
  • - *
  • diagnostics - set to "N/A"
  • - *
  • tracking URL - set to "N/A"
  • - *
  • original tracking URL - set to "N/A"
  • - *
  • resource usage report - all values are -1
  • - *
- *

- * - * @param appId - * {@link ApplicationId} of the application that needs a report - * @return application report - * @throws YarnException - * @throws IOException - */ - public abstract ApplicationReport getApplicationReport(ApplicationId appId) - throws YarnException, IOException; - - /** - *

- * Get a report (ApplicationReport) of all Applications in the cluster. - *

- * - *

- * If the user does not have VIEW_APP access for an application - * then the corresponding report will be filtered as described in - * {@link #getApplicationReport(ApplicationId)}. - *

- * - * @return a list of reports of all running applications - * @throws YarnException - * @throws IOException - */ - public abstract List getApplicationList() throws YarnException, - IOException; - - /** - *

- * Get metrics ({@link YarnClusterMetrics}) about the cluster. - *

- * - * @return cluster metrics - * @throws YarnException - * @throws IOException - */ - public abstract YarnClusterMetrics getYarnClusterMetrics() throws YarnException, - IOException; - - /** - *

- * Get a report of all nodes ({@link NodeReport}) in the cluster. - *

- * - * @return A list of report of all nodes - * @throws YarnException - * @throws IOException - */ - public abstract List getNodeReports() throws YarnException, IOException; - - /** - *

- * Get a delegation token so as to be able to talk to YARN using those tokens. - * - * @param renewer - * Address of the renewer who can renew these tokens when needed by - * securely talking to YARN. - * @return a delegation token ({@link Token}) that can be used to - * talk to YARN - * @throws YarnException - * @throws IOException - */ - public abstract Token getRMDelegationToken(Text renewer) - throws YarnException, IOException; - - /** - *

- * Get information ({@link QueueInfo}) about a given queue. - *

- * - * @param queueName - * Name of the queue whose information is needed - * @return queue information - * @throws YarnException - * in case of errors or if YARN rejects the request due to - * access-control restrictions. - * @throws IOException - */ - public abstract QueueInfo getQueueInfo(String queueName) throws YarnException, - IOException; - - /** - *

- * Get information ({@link QueueInfo}) about all queues, recursively if there - * is a hierarchy - *

- * - * @return a list of queue-information for all queues - * @throws YarnException - * @throws IOException - */ - public abstract List getAllQueues() throws YarnException, IOException; - - /** - *

- * Get information ({@link QueueInfo}) about top level queues. - *

- * - * @return a list of queue-information for all the top-level queues - * @throws YarnException - * @throws IOException - */ - public abstract List getRootQueueInfos() throws YarnException, IOException; - - /** - *

- * Get information ({@link QueueInfo}) about all the immediate children queues - * of the given queue - *

- * - * @param parent - * Name of the queue whose child-queues' information is needed - * @return a list of queue-information for all queues who are direct children - * of the given parent queue. - * @throws YarnException - * @throws IOException - */ - public abstract List getChildQueueInfos(String parent) throws YarnException, - IOException; - - /** - *

- * Get information about acls for current user on all the - * existing queues. - *

- * - * @return a list of queue acls ({@link QueueUserACLInfo}) for - * current user - * @throws YarnException - * @throws IOException - */ - public abstract List getQueueAclsInfo() throws YarnException, - IOException; -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/YarnClientImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/YarnClientImpl.java deleted file mode 100644 index fc41bac..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/YarnClientImpl.java +++ /dev/null @@ -1,307 +0,0 @@ -/** -* 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.hadoop.yarn.client; - -import java.io.IOException; -import java.net.InetSocketAddress; -import java.util.ArrayList; -import java.util.List; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.ipc.RPC; -import org.apache.hadoop.yarn.api.ApplicationClientProtocol; -import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsRequest; -import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsResponse; -import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; -import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse; -import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest; -import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse; -import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest; -import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesResponse; -import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenRequest; -import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenResponse; -import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest; -import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; -import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest; -import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest; -import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest; -import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.ApplicationReport; -import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; -import org.apache.hadoop.yarn.api.records.NodeReport; -import org.apache.hadoop.yarn.api.records.QueueInfo; -import org.apache.hadoop.yarn.api.records.QueueUserACLInfo; -import org.apache.hadoop.yarn.api.records.Token; -import org.apache.hadoop.yarn.api.records.YarnApplicationState; -import org.apache.hadoop.yarn.api.records.YarnClusterMetrics; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.hadoop.yarn.ipc.YarnRPC; -import org.apache.hadoop.yarn.util.Records; - -@InterfaceAudience.Public -@InterfaceStability.Evolving -public class YarnClientImpl extends YarnClient { - - private static final Log LOG = LogFactory.getLog(YarnClientImpl.class); - - protected ApplicationClientProtocol rmClient; - protected InetSocketAddress rmAddress; - protected long statePollIntervalMillis; - - private static final String ROOT = "root"; - - public YarnClientImpl() { - this(null); - } - - public YarnClientImpl(InetSocketAddress rmAddress) { - this(YarnClientImpl.class.getName(), rmAddress); - } - - public YarnClientImpl(String name, InetSocketAddress rmAddress) { - super(name); - this.rmAddress = rmAddress; - } - - private static InetSocketAddress getRmAddress(Configuration conf) { - return conf.getSocketAddr(YarnConfiguration.RM_ADDRESS, - YarnConfiguration.DEFAULT_RM_ADDRESS, YarnConfiguration.DEFAULT_RM_PORT); - } - - @Override - protected void serviceInit(Configuration conf) throws Exception { - if (this.rmAddress == null) { - this.rmAddress = getRmAddress(conf); - } - statePollIntervalMillis = conf.getLong( - YarnConfiguration.YARN_CLIENT_APP_SUBMISSION_POLL_INTERVAL_MS, - YarnConfiguration.DEFAULT_YARN_CLIENT_APP_SUBMISSION_POLL_INTERVAL_MS); - super.serviceInit(conf); - } - - @Override - protected void serviceStart() throws Exception { - YarnRPC rpc = YarnRPC.create(getConfig()); - - this.rmClient = (ApplicationClientProtocol) rpc.getProxy( - ApplicationClientProtocol.class, rmAddress, getConfig()); - if (LOG.isDebugEnabled()) { - LOG.debug("Connecting to ResourceManager at " + rmAddress); - } - super.serviceStart(); - } - - @Override - protected void serviceStop() throws Exception { - if (this.rmClient != null) { - RPC.stopProxy(this.rmClient); - } - super.serviceStop(); - } - - @Override - public GetNewApplicationResponse getNewApplication() - throws YarnException, IOException { - GetNewApplicationRequest request = - Records.newRecord(GetNewApplicationRequest.class); - return rmClient.getNewApplication(request); - } - - @Override - public ApplicationId - submitApplication(ApplicationSubmissionContext appContext) - throws YarnException, IOException { - ApplicationId applicationId = appContext.getApplicationId(); - appContext.setApplicationId(applicationId); - SubmitApplicationRequest request = - Records.newRecord(SubmitApplicationRequest.class); - request.setApplicationSubmissionContext(appContext); - rmClient.submitApplication(request); - - int pollCount = 0; - while (true) { - YarnApplicationState state = - getApplicationReport(applicationId).getYarnApplicationState(); - if (!state.equals(YarnApplicationState.NEW) && - !state.equals(YarnApplicationState.NEW_SAVING)) { - break; - } - // Notify the client through the log every 10 poll, in case the client - // is blocked here too long. - if (++pollCount % 10 == 0) { - LOG.info("Application submission is not finished, " + - "submitted application " + applicationId + - " is still in " + state); - } - try { - Thread.sleep(statePollIntervalMillis); - } catch (InterruptedException ie) { - } - } - - - LOG.info("Submitted application " + applicationId + " to ResourceManager" - + " at " + rmAddress); - return applicationId; - } - - @Override - public void killApplication(ApplicationId applicationId) - throws YarnException, IOException { - LOG.info("Killing application " + applicationId); - KillApplicationRequest request = - Records.newRecord(KillApplicationRequest.class); - request.setApplicationId(applicationId); - rmClient.forceKillApplication(request); - } - - @Override - public ApplicationReport getApplicationReport(ApplicationId appId) - throws YarnException, IOException { - GetApplicationReportRequest request = - Records.newRecord(GetApplicationReportRequest.class); - request.setApplicationId(appId); - GetApplicationReportResponse response = - rmClient.getApplicationReport(request); - return response.getApplicationReport(); - } - - @Override - public List getApplicationList() - throws YarnException, IOException { - GetAllApplicationsRequest request = - Records.newRecord(GetAllApplicationsRequest.class); - GetAllApplicationsResponse response = rmClient.getAllApplications(request); - return response.getApplicationList(); - } - - @Override - public YarnClusterMetrics getYarnClusterMetrics() throws YarnException, - IOException { - GetClusterMetricsRequest request = - Records.newRecord(GetClusterMetricsRequest.class); - GetClusterMetricsResponse response = rmClient.getClusterMetrics(request); - return response.getClusterMetrics(); - } - - @Override - public List getNodeReports() throws YarnException, - IOException { - GetClusterNodesRequest request = - Records.newRecord(GetClusterNodesRequest.class); - GetClusterNodesResponse response = rmClient.getClusterNodes(request); - return response.getNodeReports(); - } - - @Override - public Token getRMDelegationToken(Text renewer) - throws YarnException, IOException { - /* get the token from RM */ - GetDelegationTokenRequest rmDTRequest = - Records.newRecord(GetDelegationTokenRequest.class); - rmDTRequest.setRenewer(renewer.toString()); - GetDelegationTokenResponse response = - rmClient.getDelegationToken(rmDTRequest); - return response.getRMDelegationToken(); - } - - - private GetQueueInfoRequest - getQueueInfoRequest(String queueName, boolean includeApplications, - boolean includeChildQueues, boolean recursive) { - GetQueueInfoRequest request = Records.newRecord(GetQueueInfoRequest.class); - request.setQueueName(queueName); - request.setIncludeApplications(includeApplications); - request.setIncludeChildQueues(includeChildQueues); - request.setRecursive(recursive); - return request; - } - - @Override - public QueueInfo getQueueInfo(String queueName) throws YarnException, - IOException { - GetQueueInfoRequest request = - getQueueInfoRequest(queueName, true, false, false); - Records.newRecord(GetQueueInfoRequest.class); - return rmClient.getQueueInfo(request).getQueueInfo(); - } - - @Override - public List getQueueAclsInfo() throws YarnException, - IOException { - GetQueueUserAclsInfoRequest request = - Records.newRecord(GetQueueUserAclsInfoRequest.class); - return rmClient.getQueueUserAcls(request).getUserAclsInfoList(); - } - - @Override - public List getAllQueues() throws YarnException, - IOException { - List queues = new ArrayList(); - - QueueInfo rootQueue = - rmClient.getQueueInfo(getQueueInfoRequest(ROOT, false, true, true)) - .getQueueInfo(); - getChildQueues(rootQueue, queues, true); - return queues; - } - - @Override - public List getRootQueueInfos() throws YarnException, - IOException { - List queues = new ArrayList(); - - QueueInfo rootQueue = - rmClient.getQueueInfo(getQueueInfoRequest(ROOT, false, true, true)) - .getQueueInfo(); - getChildQueues(rootQueue, queues, false); - return queues; - } - - @Override - public List getChildQueueInfos(String parent) - throws YarnException, IOException { - List queues = new ArrayList(); - - QueueInfo parentQueue = - rmClient.getQueueInfo(getQueueInfoRequest(parent, false, true, false)) - .getQueueInfo(); - getChildQueues(parentQueue, queues, true); - return queues; - } - - private void getChildQueues(QueueInfo parent, List queues, - boolean recursive) { - List childQueues = parent.getChildQueues(); - - for (QueueInfo child : childQueues) { - queues.add(child); - if (recursive) { - getChildQueues(child, queues, recursive); - } - } - } -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java new file mode 100644 index 0000000..e497005 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java @@ -0,0 +1,262 @@ +/** +* 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.hadoop.yarn.client.api; + +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.ConcurrentMap; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; +import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ContainerId; +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.Token; +import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl; +import org.apache.hadoop.yarn.exceptions.YarnException; + +import com.google.common.collect.ImmutableList; + +@InterfaceAudience.Public +@InterfaceStability.Stable +public abstract class AMRMClient extends + AbstractService { + + /** + * Create a new instance of AMRMClient. + * For usage: + *
+   * {@code
+   * AMRMClient.createAMRMClientContainerRequest(appAttemptId)
+   * }
+ * @param appAttemptId the appAttemptId associated with the AMRMClient + * @return the newly create AMRMClient instance. + */ + @Public + public static AMRMClient createAMRMClient( + ApplicationAttemptId appAttemptId) { + AMRMClient client = new AMRMClientImpl(appAttemptId); + return client; + } + + @Private + protected AMRMClient(String name) { + super(name); + } + + /** + * Object to represent container request for resources. Scheduler + * documentation should be consulted for the specifics of how the parameters + * are honored. + * All getters return immutable values. + * + * @param capability + * The {@link Resource} to be requested for each container. + * @param nodes + * Any hosts to request that the containers are placed on. + * @param racks + * Any racks to request that the containers are placed on. The racks + * corresponding to any hosts requested will be automatically added to + * this list. + * @param priority + * The priority at which to request the containers. Higher priorities have + * lower numerical values. + * @param containerCount + * The number of containers to request. + */ + public static class ContainerRequest { + final Resource capability; + final List nodes; + final List racks; + final Priority priority; + final int containerCount; + + public ContainerRequest(Resource capability, String[] nodes, + String[] racks, Priority priority, int containerCount) { + this.capability = capability; + this.nodes = (nodes != null ? ImmutableList.copyOf(nodes) : null); + this.racks = (racks != null ? ImmutableList.copyOf(racks) : null); + this.priority = priority; + this.containerCount = containerCount; + } + + public Resource getCapability() { + return capability; + } + + public List getNodes() { + return nodes; + } + + public List getRacks() { + return racks; + } + + public Priority getPriority() { + return priority; + } + + public int getContainerCount() { + return containerCount; + } + + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("Capability[").append(capability).append("]"); + sb.append("Priority[").append(priority).append("]"); + sb.append("ContainerCount[").append(containerCount).append("]"); + return sb.toString(); + } + } + + /** + * This creates a ContainerRequest for 1 container and the + * AMRMClient stores this request internally. getMatchingRequests + * can be used to retrieve these requests from AMRMClient. These requests may + * be matched with an allocated container to determine which request to assign + * the container to. removeContainerRequest must be called using + * the same assigned StoredContainerRequest object so that + * AMRMClient can remove it from its internal store. + */ + public static class StoredContainerRequest extends ContainerRequest { + public StoredContainerRequest(Resource capability, String[] nodes, + String[] racks, Priority priority) { + super(capability, nodes, racks, priority, 1); + } + } + + /** + * Register the application master. This must be called before any + * other interaction + * @param appHostName Name of the host on which master is running + * @param appHostPort Port master is listening on + * @param appTrackingUrl URL at which the master info can be seen + * @return RegisterApplicationMasterResponse + * @throws YarnException + * @throws IOException + */ + public abstract RegisterApplicationMasterResponse + registerApplicationMaster(String appHostName, + int appHostPort, + String appTrackingUrl) + throws YarnException, IOException; + + /** + * Request additional containers and receive new container allocations. + * Requests made via addContainerRequest are sent to the + * ResourceManager. New containers assigned to the master are + * retrieved. Status of completed containers and node health updates are + * also retrieved. + * This also doubles up as a heartbeat to the ResourceManager and must be + * made periodically. + * The call may not always return any new allocations of containers. + * App should not make concurrent allocate requests. May cause request loss. + * @param progressIndicator Indicates progress made by the master + * @return the response of the allocate request + * @throws YarnException + * @throws IOException + */ + public abstract AllocateResponse allocate(float progressIndicator) + throws YarnException, IOException; + + /** + * Unregister the application master. This must be called in the end. + * @param appStatus Success/Failure status of the master + * @param appMessage Diagnostics message on failure + * @param appTrackingUrl New URL to get master info + * @throws YarnException + * @throws IOException + */ + public abstract void unregisterApplicationMaster(FinalApplicationStatus appStatus, + String appMessage, + String appTrackingUrl) + throws YarnException, IOException; + + /** + * Request containers for resources before calling allocate + * @param req Resource request + */ + public abstract void addContainerRequest(T req); + + /** + * Remove previous container request. The previous container request may have + * already been sent to the ResourceManager. So even after the remove request + * the app must be prepared to receive an allocation for the previous request + * even after the remove request + * @param req Resource request + */ + public abstract void removeContainerRequest(T req); + + /** + * Release containers assigned by the Resource Manager. If the app cannot use + * the container or wants to give up the container then it can release them. + * The app needs to make new requests for the released resource capability if + * it still needs it. eg. it released non-local resources + * @param containerId + */ + public abstract void releaseAssignedContainer(ContainerId containerId); + + /** + * Get the currently available resources in the cluster. + * A valid value is available after a call to allocate has been made + * @return Currently available resources + */ + public abstract Resource getClusterAvailableResources(); + + /** + * Get the current number of nodes in the cluster. + * A valid values is available after a call to allocate has been made + * @return Current number of nodes in the cluster + */ + public abstract int getClusterNodeCount(); + + /** + * Get outstanding StoredContainerRequests matching the given + * parameters. These StoredContainerRequests should have been added via + * addContainerRequest earlier in the lifecycle. For performance, + * the AMRMClient may return its internal collection directly without creating + * a copy. Users should not perform mutable operations on the return value. + * Each collection in the list contains requests with identical + * Resource size that fit in the given capability. In a + * collection, requests will be returned in the same order as they were added. + * @return Collection of request matching the parameters + */ + public abstract List> getMatchingRequests( + Priority priority, + String resourceName, + Resource capability); + + /** + * It returns the NMToken received on allocate call. It will not communicate + * with RM to get NMTokens. On allocate call whenever we receive new token + * along with container AMRMClient will cache this NMToken per node manager. + * This map returned should be shared with any application which is + * communicating with NodeManager (ex. NMClient) using NMTokens. If a new + * NMToken is received for the same node manager then it will be replaced. + */ + public abstract ConcurrentMap getNMTokens(); +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/NMClient.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/NMClient.java new file mode 100644 index 0000000..e628745 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/NMClient.java @@ -0,0 +1,123 @@ +/** + * 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.hadoop.yarn.client.api; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Map; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; +import org.apache.hadoop.yarn.api.records.ContainerStatus; +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.api.records.Token; +import org.apache.hadoop.yarn.client.api.impl.NMClientImpl; +import org.apache.hadoop.yarn.exceptions.YarnException; + +@InterfaceAudience.Public +@InterfaceStability.Stable +public abstract class NMClient extends AbstractService { + + /** + * Create a new instance of NMClient. + */ + @Public + public static NMClient createNMClient() { + NMClient client = new NMClientImpl(); + return client; + } + + /** + * Create a new instance of NMClient. + */ + @Public + public static NMClient createNMClient(String name) { + NMClient client = new NMClientImpl(name); + return client; + } + + @Private + protected NMClient(String name) { + super(name); + } + + /** + *

Start an allocated container.

+ * + *

The ApplicationMaster or other applications that use the + * client must provide the details of the allocated container, including the + * Id, the assigned node's Id and the token via {@link Container}. In + * addition, the AM needs to provide the {@link ContainerLaunchContext} as + * well.

+ * + * @param container the allocated container + * @param containerLaunchContext the context information needed by the + * NodeManager to launch the + * container + * @return a map between the auxiliary service names and their outputs + * @throws YarnException + * @throws IOException + */ + public abstract Map startContainer(Container container, + ContainerLaunchContext containerLaunchContext) + throws YarnException, IOException; + + /** + *

Stop an started container.

+ * + * @param containerId the Id of the started container + * @param nodeId the Id of the NodeManager + * @param containerToken the security token to verify authenticity of the + * started container + * @throws YarnException + * @throws IOException + */ + public abstract void stopContainer(ContainerId containerId, NodeId nodeId, + Token containerToken) throws YarnException, IOException; + + /** + *

Query the status of a container.

+ * + * @param containerId the Id of the started container + * @param nodeId the Id of the NodeManager + * @param containerToken the security token to verify authenticity of the + * started container + * @return the status of a container + * @throws YarnException + * @throws IOException + */ + public abstract ContainerStatus getContainerStatus(ContainerId containerId, NodeId nodeId, + Token containerToken) throws YarnException, IOException; + + /** + *

Set whether the containers that are started by this client, and are + * still running should be stopped when the client stops. By default, the + * feature should be enabled.

+ * + * @param enabled whether the feature is enabled or not + */ + public abstract void cleanupRunningContainersOnStop(boolean enabled); + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java new file mode 100644 index 0000000..751c4ba --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java @@ -0,0 +1,295 @@ +/** +* 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.hadoop.yarn.client.api; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.List; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; +import org.apache.hadoop.yarn.api.records.NodeReport; +import org.apache.hadoop.yarn.api.records.QueueInfo; +import org.apache.hadoop.yarn.api.records.QueueUserACLInfo; +import org.apache.hadoop.yarn.api.records.Token; +import org.apache.hadoop.yarn.api.records.YarnClusterMetrics; +import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl; +import org.apache.hadoop.yarn.exceptions.YarnException; + +@InterfaceAudience.Public +@InterfaceStability.Stable +public abstract class YarnClient extends AbstractService { + + /** + * Create a new instance of YarnClient. + */ + @Public + public static YarnClient createYarnClient() { + YarnClient client = new YarnClientImpl(); + return client; + } + + /** + * Create a new instance of YarnClient. + */ + @Public + public static YarnClient createYarnClient(InetSocketAddress rmAddress) { + YarnClient client = new YarnClientImpl(rmAddress); + return client; + } + + /** + * Create a new instance of YarnClient. + */ + @Public + public static YarnClient createYarnClient(String name, + InetSocketAddress rmAddress) { + YarnClient client = new YarnClientImpl(name, rmAddress); + return client; + } + + @Private + protected YarnClient(String name) { + super(name); + } + + /** + *

+ * Obtain a new {@link ApplicationId} for submitting new applications. + *

+ * + *

+ * Returns a response which contains {@link ApplicationId} that can be used to + * submit a new application. See + * {@link #submitApplication(ApplicationSubmissionContext)}. + *

+ * + *

+ * See {@link GetNewApplicationResponse} for other information that is + * returned. + *

+ * + * @return response containing the new ApplicationId to be used + * to submit an application + * @throws YarnException + * @throws IOException + */ + public abstract GetNewApplicationResponse getNewApplication() throws YarnException, + IOException; + + /** + *

+ * Submit a new application to YARN. It is a blocking call, such + * that it will not return {@link ApplicationId} until the submitted + * application has been submitted and accepted by the ResourceManager. + *

+ * + * @param appContext + * {@link ApplicationSubmissionContext} containing all the details + * needed to submit a new application + * @return {@link ApplicationId} of the accepted application + * @throws YarnException + * @throws IOException + * @see #getNewApplication() + */ + public abstract ApplicationId submitApplication(ApplicationSubmissionContext appContext) + throws YarnException, IOException; + + /** + *

+ * Kill an application identified by given ID. + *

+ * + * @param applicationId + * {@link ApplicationId} of the application that needs to be killed + * @throws YarnException + * in case of errors or if YARN rejects the request due to + * access-control restrictions. + * @throws IOException + * @see #getQueueAclsInfo() + */ + public abstract void killApplication(ApplicationId applicationId) throws YarnException, + IOException; + + /** + *

+ * Get a report of the given Application. + *

+ * + *

+ * In secure mode, YARN verifies access to the application, queue + * etc. before accepting the request. + *

+ * + *

+ * If the user does not have VIEW_APP access then the following + * fields in the report will be set to stubbed values: + *

    + *
  • host - set to "N/A"
  • + *
  • RPC port - set to -1
  • + *
  • client token - set to "N/A"
  • + *
  • diagnostics - set to "N/A"
  • + *
  • tracking URL - set to "N/A"
  • + *
  • original tracking URL - set to "N/A"
  • + *
  • resource usage report - all values are -1
  • + *
+ *

+ * + * @param appId + * {@link ApplicationId} of the application that needs a report + * @return application report + * @throws YarnException + * @throws IOException + */ + public abstract ApplicationReport getApplicationReport(ApplicationId appId) + throws YarnException, IOException; + + /** + *

+ * Get a report (ApplicationReport) of all Applications in the cluster. + *

+ * + *

+ * If the user does not have VIEW_APP access for an application + * then the corresponding report will be filtered as described in + * {@link #getApplicationReport(ApplicationId)}. + *

+ * + * @return a list of reports of all running applications + * @throws YarnException + * @throws IOException + */ + public abstract List getApplicationList() throws YarnException, + IOException; + + /** + *

+ * Get metrics ({@link YarnClusterMetrics}) about the cluster. + *

+ * + * @return cluster metrics + * @throws YarnException + * @throws IOException + */ + public abstract YarnClusterMetrics getYarnClusterMetrics() throws YarnException, + IOException; + + /** + *

+ * Get a report of all nodes ({@link NodeReport}) in the cluster. + *

+ * + * @return A list of report of all nodes + * @throws YarnException + * @throws IOException + */ + public abstract List getNodeReports() throws YarnException, IOException; + + /** + *

+ * Get a delegation token so as to be able to talk to YARN using those tokens. + * + * @param renewer + * Address of the renewer who can renew these tokens when needed by + * securely talking to YARN. + * @return a delegation token ({@link Token}) that can be used to + * talk to YARN + * @throws YarnException + * @throws IOException + */ + public abstract Token getRMDelegationToken(Text renewer) + throws YarnException, IOException; + + /** + *

+ * Get information ({@link QueueInfo}) about a given queue. + *

+ * + * @param queueName + * Name of the queue whose information is needed + * @return queue information + * @throws YarnException + * in case of errors or if YARN rejects the request due to + * access-control restrictions. + * @throws IOException + */ + public abstract QueueInfo getQueueInfo(String queueName) throws YarnException, + IOException; + + /** + *

+ * Get information ({@link QueueInfo}) about all queues, recursively if there + * is a hierarchy + *

+ * + * @return a list of queue-information for all queues + * @throws YarnException + * @throws IOException + */ + public abstract List getAllQueues() throws YarnException, IOException; + + /** + *

+ * Get information ({@link QueueInfo}) about top level queues. + *

+ * + * @return a list of queue-information for all the top-level queues + * @throws YarnException + * @throws IOException + */ + public abstract List getRootQueueInfos() throws YarnException, IOException; + + /** + *

+ * Get information ({@link QueueInfo}) about all the immediate children queues + * of the given queue + *

+ * + * @param parent + * Name of the queue whose child-queues' information is needed + * @return a list of queue-information for all queues who are direct children + * of the given parent queue. + * @throws YarnException + * @throws IOException + */ + public abstract List getChildQueueInfos(String parent) throws YarnException, + IOException; + + /** + *

+ * Get information about acls for current user on all the + * existing queues. + *

+ * + * @return a list of queue acls ({@link QueueUserACLInfo}) for + * current user + * @throws YarnException + * @throws IOException + */ + public abstract List getQueueAclsInfo() throws YarnException, + IOException; +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java new file mode 100644 index 0000000..d9e46c0 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java @@ -0,0 +1,245 @@ +/** +* 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.hadoop.yarn.client.api.async; + +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Stable; +import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerStatus; +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.hadoop.yarn.api.records.NodeReport; +import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.Token; +import org.apache.hadoop.yarn.client.api.AMRMClient; +import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest; +import org.apache.hadoop.yarn.client.api.async.impl.AMRMClientAsyncImpl; +import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.service.AbstractService; + +import com.google.common.annotations.VisibleForTesting; + +/** + * AMRMClientAsync handles communication with the ResourceManager + * and provides asynchronous updates on events such as container allocations and + * completions. It contains a thread that sends periodic heartbeats to the + * ResourceManager. + * + * It should be used by implementing a CallbackHandler: + *
+ * {@code
+ * class MyCallbackHandler implements AMRMClientAsync.CallbackHandler {
+ *   public void onContainersAllocated(List containers) {
+ *     [run tasks on the containers]
+ *   }
+ *   
+ *   public void onContainersCompleted(List statuses) {
+ *     [update progress, check whether app is done]
+ *   }
+ *   
+ *   public void onNodesUpdated(List updated) {}
+ *   
+ *   public void onReboot() {}
+ * }
+ * }
+ * 
+ * + * The client's lifecycle should be managed similarly to the following: + * + *
+ * {@code
+ * AMRMClientAsync asyncClient = 
+ *     createAMRMClientAsync(appAttId, 1000, new MyCallbackhandler());
+ * asyncClient.init(conf);
+ * asyncClient.start();
+ * RegisterApplicationMasterResponse response = asyncClient
+ *    .registerApplicationMaster(appMasterHostname, appMasterRpcPort,
+ *       appMasterTrackingUrl);
+ * asyncClient.addContainerRequest(containerRequest);
+ * [... wait for application to complete]
+ * asyncClient.unregisterApplicationMaster(status, appMsg, trackingUrl);
+ * asyncClient.stop();
+ * }
+ * 
+ */ +@Public +@Stable +public abstract class AMRMClientAsync +extends AbstractService { + + protected final AMRMClient client; + protected final CallbackHandler handler; + protected final AtomicInteger heartbeatIntervalMs = new AtomicInteger(); + + public static AMRMClientAsync + createAMRMClientAsync( + ApplicationAttemptId id, + int intervalMs, + CallbackHandler callbackHandler) { + return new AMRMClientAsyncImpl(id, intervalMs, callbackHandler); + } + + public static AMRMClientAsync + createAMRMClientAsync( + AMRMClient client, + int intervalMs, + CallbackHandler callbackHandler) { + return new AMRMClientAsyncImpl(client, intervalMs, callbackHandler); + } + + protected AMRMClientAsync(ApplicationAttemptId id, int intervalMs, + CallbackHandler callbackHandler) { + this(new AMRMClientImpl(id), intervalMs, callbackHandler); + } + + @Private + @VisibleForTesting + protected AMRMClientAsync(AMRMClient client, int intervalMs, + CallbackHandler callbackHandler) { + super(AMRMClientAsync.class.getName()); + this.client = client; + this.heartbeatIntervalMs.set(intervalMs); + this.handler = callbackHandler; + } + + public void setHeartbeatInterval(int interval) { + heartbeatIntervalMs.set(interval); + } + + public abstract List> getMatchingRequests( + Priority priority, + String resourceName, + Resource capability); + + /** + * Registers this application master with the resource manager. On successful + * registration, starts the heartbeating thread. + * @throws YarnException + * @throws IOException + */ + public abstract RegisterApplicationMasterResponse registerApplicationMaster( + String appHostName, int appHostPort, String appTrackingUrl) + throws YarnException, IOException; + + /** + * Unregister the application master. This must be called in the end. + * @param appStatus Success/Failure status of the master + * @param appMessage Diagnostics message on failure + * @param appTrackingUrl New URL to get master info + * @throws YarnException + * @throws IOException + */ + public abstract void unregisterApplicationMaster( + FinalApplicationStatus appStatus, String appMessage, String appTrackingUrl) + throws YarnException, IOException; + + /** + * Request containers for resources before calling allocate + * @param req Resource request + */ + public abstract void addContainerRequest(T req); + + /** + * Remove previous container request. The previous container request may have + * already been sent to the ResourceManager. So even after the remove request + * the app must be prepared to receive an allocation for the previous request + * even after the remove request + * @param req Resource request + */ + public abstract void removeContainerRequest(T req); + + /** + * Release containers assigned by the Resource Manager. If the app cannot use + * the container or wants to give up the container then it can release them. + * The app needs to make new requests for the released resource capability if + * it still needs it. eg. it released non-local resources + * @param containerId + */ + public abstract void releaseAssignedContainer(ContainerId containerId); + + /** + * Get the currently available resources in the cluster. + * A valid value is available after a call to allocate has been made + * @return Currently available resources + */ + public abstract Resource getClusterAvailableResources(); + + /** + * Get the current number of nodes in the cluster. + * A valid values is available after a call to allocate has been made + * @return Current number of nodes in the cluster + */ + public abstract int getClusterNodeCount(); + + /** + * It returns the NMToken received on allocate call. It will not communicate + * with RM to get NMTokens. On allocate call whenever we receive new token + * along with new container AMRMClientAsync will cache this NMToken per node + * manager. This map returned should be shared with any application which is + * communicating with NodeManager (ex. NMClient / NMClientAsync) using + * NMTokens. If a new NMToken is received for the same node manager + * then it will be replaced. + */ + public abstract ConcurrentMap getNMTokens(); + + public interface CallbackHandler { + + /** + * Called when the ResourceManager responds to a heartbeat with completed + * containers. If the response contains both completed containers and + * allocated containers, this will be called before containersAllocated. + */ + public void onContainersCompleted(List statuses); + + /** + * Called when the ResourceManager responds to a heartbeat with allocated + * containers. If the response containers both completed containers and + * allocated containers, this will be called after containersCompleted. + */ + public void onContainersAllocated(List containers); + + /** + * Called when the ResourceManager wants the ApplicationMaster to shutdown + * for being out of sync etc. The ApplicationMaster should not unregister + * with the RM unless the ApplicationMaster wants to be the last attempt. + */ + public void onShutdownRequest(); + + /** + * Called when nodes tracked by the ResourceManager have changed in health, + * availability etc. + */ + public void onNodesUpdated(List updatedNodes); + + public float getProgress(); + + public void onError(Exception e); + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/NMClientAsync.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/NMClientAsync.java new file mode 100644 index 0000000..0a6a399 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/NMClientAsync.java @@ -0,0 +1,235 @@ +/** + * 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.hadoop.yarn.client.api.async; + +import java.nio.ByteBuffer; +import java.util.Map; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Stable; +import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; +import org.apache.hadoop.yarn.api.records.ContainerStatus; +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.api.records.Token; +import org.apache.hadoop.yarn.client.api.NMClient; +import org.apache.hadoop.yarn.client.api.async.impl.NMClientAsyncImpl; +import org.apache.hadoop.yarn.client.api.impl.NMClientImpl; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.service.AbstractService; + +import com.google.common.annotations.VisibleForTesting; + +/** + * NMClientAsync handles communication with all the NodeManagers + * and provides asynchronous updates on getting responses from them. It + * maintains a thread pool to communicate with individual NMs where a number of + * worker threads process requests to NMs by using {@link NMClientImpl}. The max + * size of the thread pool is configurable through + * {@link YarnConfiguration#NM_CLIENT_ASYNC_THREAD_POOL_MAX_SIZE}. + * + * It should be used in conjunction with a CallbackHandler. For example + * + *
+ * {@code
+ * class MyCallbackHandler implements NMClientAsync.CallbackHandler {
+ *   public void onContainerStarted(ContainerId containerId,
+ *       Map allServiceResponse) {
+ *     [post process after the container is started, process the response]
+ *   }
+ *
+ *   public void onContainerStatusReceived(ContainerId containerId,
+ *       ContainerStatus containerStatus) {
+ *     [make use of the status of the container]
+ *   }
+ *
+ *   public void onContainerStopped(ContainerId containerId) {
+ *     [post process after the container is stopped]
+ *   }
+ *
+ *   public void onStartContainerError(
+ *       ContainerId containerId, Throwable t) {
+ *     [handle the raised exception]
+ *   }
+ *
+ *   public void onGetContainerStatusError(
+ *       ContainerId containerId, Throwable t) {
+ *     [handle the raised exception]
+ *   }
+ *
+ *   public void onStopContainerError(
+ *       ContainerId containerId, Throwable t) {
+ *     [handle the raised exception]
+ *   }
+ * }
+ * }
+ * 
+ * + * The client's life-cycle should be managed like the following: + * + *
+ * {@code
+ * NMClientAsync asyncClient = 
+ *     NMClientAsync.createNMClientAsync(new MyCallbackhandler());
+ * asyncClient.init(conf);
+ * asyncClient.start();
+ * asyncClient.startContainer(container, containerLaunchContext);
+ * [... wait for container being started]
+ * asyncClient.getContainerStatus(container.getId(), container.getNodeId(),
+ *     container.getContainerToken());
+ * [... handle the status in the callback instance]
+ * asyncClient.stopContainer(container.getId(), container.getNodeId(),
+ *     container.getContainerToken());
+ * [... wait for container being stopped]
+ * asyncClient.stop();
+ * }
+ * 
+ */ +@Public +@Stable +public abstract class NMClientAsync extends AbstractService { + + private NMClient client; + private CallbackHandler callbackHandler; + + public static NMClientAsync createNMClientAsync(CallbackHandler callbackHandler) { + return new NMClientAsyncImpl(callbackHandler); + } + + protected NMClientAsync(CallbackHandler callbackHandler) { + this (NMClientAsync.class.getName(), callbackHandler); + } + + protected NMClientAsync(String name, CallbackHandler callbackHandler) { + this (name, new NMClientImpl(), callbackHandler); + } + + @Private + @VisibleForTesting + protected NMClientAsync(String name, NMClient client, + CallbackHandler callbackHandler) { + super(name); + this.setClient(client); + this.setCallbackHandler(callbackHandler); + } + + public abstract void startContainerAsync( + Container container, ContainerLaunchContext containerLaunchContext); + + public abstract void stopContainerAsync( + ContainerId containerId, NodeId nodeId, Token containerToken); + + public abstract void getContainerStatusAsync( + ContainerId containerId, NodeId nodeId, Token containerToken); + + public NMClient getClient() { + return client; + } + + public void setClient(NMClient client) { + this.client = client; + } + + public CallbackHandler getCallbackHandler() { + return callbackHandler; + } + + public void setCallbackHandler(CallbackHandler callbackHandler) { + this.callbackHandler = callbackHandler; + } + + /** + *

+ * The callback interface needs to be implemented by {@link NMClientAsyncXFace} + * users. The APIs are called when responses from NodeManager are + * available. + *

+ * + *

+ * Once a callback happens, the users can chose to act on it in blocking or + * non-blocking manner. If the action on callback is done in a blocking + * manner, some of the threads performing requests on NodeManagers may get + * blocked depending on how many threads in the pool are busy. + *

+ * + *

+ * The implementation of the callback function should not throw the + * unexpected exception. Otherwise, {@link NMClientAsyncXFace} will just + * catch, log and then ignore it. + *

+ */ + public static interface CallbackHandler { + /** + * The API is called when NodeManager responds to indicate its + * acceptance of the starting container request + * @param containerId the Id of the container + * @param allServiceResponse a Map between the auxiliary service names and + * their outputs + */ + void onContainerStarted(ContainerId containerId, + Map allServiceResponse); + + /** + * The API is called when NodeManager responds with the status + * of the container + * @param containerId the Id of the container + * @param containerStatus the status of the container + */ + void onContainerStatusReceived(ContainerId containerId, + ContainerStatus containerStatus); + + /** + * The API is called when NodeManager responds to indicate the + * container is stopped. + * @param containerId the Id of the container + */ + void onContainerStopped(ContainerId containerId); + + /** + * The API is called when an exception is raised in the process of + * starting a container + * + * @param containerId the Id of the container + * @param t the raised exception + */ + void onStartContainerError(ContainerId containerId, Throwable t); + + /** + * The API is called when an exception is raised in the process of + * querying the status of a container + * + * @param containerId the Id of the container + * @param t the raised exception + */ + void onGetContainerStatusError(ContainerId containerId, Throwable t); + + /** + * The API is called when an exception is raised in the process of + * stopping a container + * + * @param containerId the Id of the container + * @param t the raised exception + */ + void onStopContainerError(ContainerId containerId, Throwable t); + + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java new file mode 100644 index 0000000..3904047 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java @@ -0,0 +1,342 @@ +/** +* 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.hadoop.yarn.client.api.async.impl; + +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.LinkedBlockingQueue; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; +import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerStatus; +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.hadoop.yarn.api.records.NodeReport; +import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.Token; +import org.apache.hadoop.yarn.client.api.AMRMClient; +import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest; +import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync; +import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; + +import com.google.common.annotations.VisibleForTesting; + +@Private +@Unstable +public class AMRMClientAsyncImpl extends AMRMClientAsync{ + + private static final Log LOG = LogFactory.getLog(AMRMClientAsyncImpl.class); + + private final HeartbeatThread heartbeatThread; + private final CallbackHandlerThread handlerThread; + + private final BlockingQueue responseQueue; + + private final Object unregisterHeartbeatLock = new Object(); + + private volatile boolean keepRunning; + private volatile float progress; + + private volatile Exception savedException; + + public AMRMClientAsyncImpl(ApplicationAttemptId id, int intervalMs, + CallbackHandler callbackHandler) { + this(new AMRMClientImpl(id), intervalMs, callbackHandler); + } + + @Private + @VisibleForTesting + public AMRMClientAsyncImpl(AMRMClient client, int intervalMs, + CallbackHandler callbackHandler) { + super(client, intervalMs, callbackHandler); + heartbeatThread = new HeartbeatThread(); + handlerThread = new CallbackHandlerThread(); + responseQueue = new LinkedBlockingQueue(); + keepRunning = true; + savedException = null; + } + + @Override + protected void serviceInit(Configuration conf) throws Exception { + super.serviceInit(conf); + client.init(conf); + } + + @Override + protected void serviceStart() throws Exception { + handlerThread.start(); + client.start(); + super.serviceStart(); + } + + /** + * Tells the heartbeat and handler threads to stop and waits for them to + * terminate. Calling this method from the callback handler thread would cause + * deadlock, and thus should be avoided. + */ + @Override + protected void serviceStop() throws Exception { + if (Thread.currentThread() == handlerThread) { + throw new YarnRuntimeException("Cannot call stop from callback handler thread!"); + } + keepRunning = false; + try { + heartbeatThread.join(); + } catch (InterruptedException ex) { + LOG.error("Error joining with heartbeat thread", ex); + } + client.stop(); + try { + handlerThread.interrupt(); + handlerThread.join(); + } catch (InterruptedException ex) { + LOG.error("Error joining with hander thread", ex); + } + super.serviceStop(); + } + + public void setHeartbeatInterval(int interval) { + heartbeatIntervalMs.set(interval); + } + + public List> getMatchingRequests( + Priority priority, + String resourceName, + Resource capability) { + return client.getMatchingRequests(priority, resourceName, capability); + } + + /** + * Registers this application master with the resource manager. On successful + * registration, starts the heartbeating thread. + * @throws YarnException + * @throws IOException + */ + public RegisterApplicationMasterResponse registerApplicationMaster( + String appHostName, int appHostPort, String appTrackingUrl) + throws YarnException, IOException { + RegisterApplicationMasterResponse response = client + .registerApplicationMaster(appHostName, appHostPort, appTrackingUrl); + heartbeatThread.start(); + return response; + } + + /** + * Unregister the application master. This must be called in the end. + * @param appStatus Success/Failure status of the master + * @param appMessage Diagnostics message on failure + * @param appTrackingUrl New URL to get master info + * @throws YarnException + * @throws IOException + */ + public void unregisterApplicationMaster(FinalApplicationStatus appStatus, + String appMessage, String appTrackingUrl) throws YarnException, + IOException { + synchronized (unregisterHeartbeatLock) { + keepRunning = false; + client.unregisterApplicationMaster(appStatus, appMessage, appTrackingUrl); + } + } + + /** + * Request containers for resources before calling allocate + * @param req Resource request + */ + public void addContainerRequest(T req) { + client.addContainerRequest(req); + } + + /** + * Remove previous container request. The previous container request may have + * already been sent to the ResourceManager. So even after the remove request + * the app must be prepared to receive an allocation for the previous request + * even after the remove request + * @param req Resource request + */ + public void removeContainerRequest(T req) { + client.removeContainerRequest(req); + } + + /** + * Release containers assigned by the Resource Manager. If the app cannot use + * the container or wants to give up the container then it can release them. + * The app needs to make new requests for the released resource capability if + * it still needs it. eg. it released non-local resources + * @param containerId + */ + public void releaseAssignedContainer(ContainerId containerId) { + client.releaseAssignedContainer(containerId); + } + + /** + * Get the currently available resources in the cluster. + * A valid value is available after a call to allocate has been made + * @return Currently available resources + */ + public Resource getClusterAvailableResources() { + return client.getClusterAvailableResources(); + } + + /** + * Get the current number of nodes in the cluster. + * A valid values is available after a call to allocate has been made + * @return Current number of nodes in the cluster + */ + public int getClusterNodeCount() { + return client.getClusterNodeCount(); + } + + /** + * It returns the NMToken received on allocate call. It will not communicate + * with RM to get NMTokens. On allocate call whenever we receive new token + * along with new container AMRMClientAsync will cache this NMToken per node + * manager. This map returned should be shared with any application which is + * communicating with NodeManager (ex. NMClient / NMClientAsync) using + * NMTokens. If a new NMToken is received for the same node manager + * then it will be replaced. + */ + public ConcurrentMap getNMTokens() { + return client.getNMTokens(); + } + + private class HeartbeatThread extends Thread { + public HeartbeatThread() { + super("AMRM Heartbeater thread"); + } + + public void run() { + while (true) { + AllocateResponse response = null; + // synchronization ensures we don't send heartbeats after unregistering + synchronized (unregisterHeartbeatLock) { + if (!keepRunning) { + break; + } + + try { + response = client.allocate(progress); + } catch (YarnException ex) { + LOG.error("Yarn exception on heartbeat", ex); + savedException = ex; + // interrupt handler thread in case it waiting on the queue + handlerThread.interrupt(); + break; + } catch (IOException e) { + LOG.error("IO exception on heartbeat", e); + savedException = e; + // interrupt handler thread in case it waiting on the queue + handlerThread.interrupt(); + break; + } + } + if (response != null) { + while (true) { + try { + responseQueue.put(response); + break; + } catch (InterruptedException ex) { + LOG.info("Interrupted while waiting to put on response queue", ex); + } + } + } + + try { + Thread.sleep(heartbeatIntervalMs.get()); + } catch (InterruptedException ex) { + LOG.info("Heartbeater interrupted", ex); + } + } + } + } + + private class CallbackHandlerThread extends Thread { + public CallbackHandlerThread() { + super("AMRM Callback Handler Thread"); + } + + public void run() { + while (keepRunning) { + AllocateResponse response; + try { + if(savedException != null) { + LOG.error("Stopping callback due to: ", savedException); + handler.onError(savedException); + break; + } + response = responseQueue.take(); + } catch (InterruptedException ex) { + LOG.info("Interrupted while waiting for queue", ex); + continue; + } + + if (response.getAMCommand() != null) { + boolean stop = false; + switch(response.getAMCommand()) { + case AM_RESYNC: + case AM_SHUTDOWN: + handler.onShutdownRequest(); + LOG.info("Shutdown requested. Stopping callback."); + stop = true; + break; + default: + String msg = + "Unhandled value of AMCommand: " + response.getAMCommand(); + LOG.error(msg); + throw new YarnRuntimeException(msg); + } + if(stop) { + // should probably stop heartbeating also YARN-763 + break; + } + } + List updatedNodes = response.getUpdatedNodes(); + if (!updatedNodes.isEmpty()) { + handler.onNodesUpdated(updatedNodes); + } + + List completed = + response.getCompletedContainersStatuses(); + if (!completed.isEmpty()) { + handler.onContainersCompleted(completed); + } + + List allocated = response.getAllocatedContainers(); + if (!allocated.isEmpty()) { + handler.onContainersAllocated(allocated); + } + + progress = handler.getProgress(); + } + } + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/NMClientAsyncImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/NMClientAsyncImpl.java new file mode 100644 index 0000000..d872acb --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/NMClientAsyncImpl.java @@ -0,0 +1,582 @@ +/** + * 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.hadoop.yarn.client.api.async.impl; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.EnumSet; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock; +import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; +import org.apache.hadoop.yarn.api.records.ContainerStatus; +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.api.records.Token; +import org.apache.hadoop.yarn.client.api.NMClient; +import org.apache.hadoop.yarn.client.api.async.NMClientAsync; +import org.apache.hadoop.yarn.client.api.impl.NMClientImpl; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.event.AbstractEvent; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.ipc.RPCUtil; +import org.apache.hadoop.yarn.state.InvalidStateTransitonException; +import org.apache.hadoop.yarn.state.MultipleArcTransition; +import org.apache.hadoop.yarn.state.SingleArcTransition; +import org.apache.hadoop.yarn.state.StateMachine; +import org.apache.hadoop.yarn.state.StateMachineFactory; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.util.concurrent.ThreadFactoryBuilder; + +@Private +@Unstable +public class NMClientAsyncImpl extends NMClientAsync { + + private static final Log LOG = LogFactory.getLog(NMClientAsyncImpl.class); + + protected static final int INITIAL_THREAD_POOL_SIZE = 10; + + protected ThreadPoolExecutor threadPool; + protected int maxThreadPoolSize; + protected Thread eventDispatcherThread; + protected AtomicBoolean stopped = new AtomicBoolean(false); + protected BlockingQueue events = + new LinkedBlockingQueue(); + + protected NMClient client; + protected CallbackHandler callbackHandler; + + protected ConcurrentMap containers = + new ConcurrentHashMap(); + + public NMClientAsyncImpl(CallbackHandler callbackHandler) { + this (NMClientAsyncImpl.class.getName(), callbackHandler); + } + + public NMClientAsyncImpl(String name, CallbackHandler callbackHandler) { + this (name, new NMClientImpl(), callbackHandler); + } + + @Private + @VisibleForTesting + protected NMClientAsyncImpl(String name, NMClient client, + CallbackHandler callbackHandler) { + super(name, client, callbackHandler); + this.client = client; + this.callbackHandler = callbackHandler; + } + + @Override + protected void serviceInit(Configuration conf) throws Exception { + this.maxThreadPoolSize = conf.getInt( + YarnConfiguration.NM_CLIENT_ASYNC_THREAD_POOL_MAX_SIZE, + YarnConfiguration.DEFAULT_NM_CLIENT_ASYNC_THREAD_POOL_MAX_SIZE); + LOG.info("Upper bound of the thread pool size is " + maxThreadPoolSize); + + client.init(conf); + super.serviceInit(conf); + } + + @Override + protected void serviceStart() throws Exception { + client.start(); + + ThreadFactory tf = new ThreadFactoryBuilder().setNameFormat( + this.getClass().getName() + " #%d").setDaemon(true).build(); + + // Start with a default core-pool size and change it dynamically. + int initSize = Math.min(INITIAL_THREAD_POOL_SIZE, maxThreadPoolSize); + threadPool = new ThreadPoolExecutor(initSize, Integer.MAX_VALUE, 1, + TimeUnit.HOURS, new LinkedBlockingQueue(), tf); + + eventDispatcherThread = new Thread() { + @Override + public void run() { + ContainerEvent event = null; + Set allNodes = new HashSet(); + + while (!stopped.get() && !Thread.currentThread().isInterrupted()) { + try { + event = events.take(); + } catch (InterruptedException e) { + if (!stopped.get()) { + LOG.error("Returning, thread interrupted", e); + } + return; + } + + allNodes.add(event.getNodeId().toString()); + + int threadPoolSize = threadPool.getCorePoolSize(); + + // We can increase the pool size only if haven't reached the maximum + // limit yet. + if (threadPoolSize != maxThreadPoolSize) { + + // nodes where containers will run at *this* point of time. This is + // *not* the cluster size and doesn't need to be. + int nodeNum = allNodes.size(); + int idealThreadPoolSize = Math.min(maxThreadPoolSize, nodeNum); + + if (threadPoolSize < idealThreadPoolSize) { + // Bump up the pool size to idealThreadPoolSize + + // INITIAL_POOL_SIZE, the later is just a buffer so we are not + // always increasing the pool-size + int newThreadPoolSize = Math.min(maxThreadPoolSize, + idealThreadPoolSize + INITIAL_THREAD_POOL_SIZE); + LOG.info("Set NMClientAsync thread pool size to " + + newThreadPoolSize + " as the number of nodes to talk to is " + + nodeNum); + threadPool.setCorePoolSize(newThreadPoolSize); + } + } + + // the events from the queue are handled in parallel with a thread + // pool + threadPool.execute(getContainerEventProcessor(event)); + + // TODO: Group launching of multiple containers to a single + // NodeManager into a single connection + } + } + }; + eventDispatcherThread.setName("Container Event Dispatcher"); + eventDispatcherThread.setDaemon(false); + eventDispatcherThread.start(); + + super.serviceStart(); + } + + @Override + protected void serviceStop() throws Exception { + if (stopped.getAndSet(true)) { + // return if already stopped + return; + } + if (eventDispatcherThread != null) { + eventDispatcherThread.interrupt(); + try { + eventDispatcherThread.join(); + } catch (InterruptedException e) { + LOG.error("The thread of " + eventDispatcherThread.getName() + + " didn't finish normally.", e); + } + } + if (threadPool != null) { + threadPool.shutdownNow(); + } + if (client != null) { + // If NMClientImpl doesn't stop running containers, the states doesn't + // need to be cleared. + if (!(client instanceof NMClientImpl) || + ((NMClientImpl) client).getCleanupRunningContainers().get()) { + if (containers != null) { + containers.clear(); + } + } + client.stop(); + } + super.serviceStop(); + } + + public void startContainerAsync( + Container container, ContainerLaunchContext containerLaunchContext) { + if (containers.putIfAbsent(container.getId(), + new StatefulContainer(this, container.getId())) != null) { + callbackHandler.onStartContainerError(container.getId(), + RPCUtil.getRemoteException("Container " + container.getId() + + " is already started or scheduled to start")); + } + try { + events.put(new StartContainerEvent(container, containerLaunchContext)); + } catch (InterruptedException e) { + LOG.warn("Exception when scheduling the event of starting Container " + + container.getId()); + callbackHandler.onStartContainerError(container.getId(), e); + } + } + + public void stopContainerAsync(ContainerId containerId, NodeId nodeId, + Token containerToken) { + if (containers.get(containerId) == null) { + callbackHandler.onStopContainerError(containerId, + RPCUtil.getRemoteException("Container " + containerId + + " is neither started nor scheduled to start")); + } + try { + events.put(new ContainerEvent(containerId, nodeId, containerToken, + ContainerEventType.STOP_CONTAINER)); + } catch (InterruptedException e) { + LOG.warn("Exception when scheduling the event of stopping Container " + + containerId); + callbackHandler.onStopContainerError(containerId, e); + } + } + + public void getContainerStatusAsync(ContainerId containerId, NodeId nodeId, + Token containerToken) { + try { + events.put(new ContainerEvent(containerId, nodeId, containerToken, + ContainerEventType.QUERY_CONTAINER)); + } catch (InterruptedException e) { + LOG.warn("Exception when scheduling the event of querying the status" + + " of Container " + containerId); + callbackHandler.onGetContainerStatusError(containerId, e); + } + } + + protected static enum ContainerState { + PREP, FAILED, RUNNING, DONE, + } + + protected boolean isCompletelyDone(StatefulContainer container) { + return container.getState() == ContainerState.DONE || + container.getState() == ContainerState.FAILED; + } + + protected ContainerEventProcessor getContainerEventProcessor( + ContainerEvent event) { + return new ContainerEventProcessor(event); + } + + /** + * The type of the event of interacting with a container + */ + protected static enum ContainerEventType { + START_CONTAINER, + STOP_CONTAINER, + QUERY_CONTAINER + } + + protected static class ContainerEvent + extends AbstractEvent{ + private ContainerId containerId; + private NodeId nodeId; + private Token containerToken; + + public ContainerEvent(ContainerId containerId, NodeId nodeId, + Token containerToken, ContainerEventType type) { + super(type); + this.containerId = containerId; + this.nodeId = nodeId; + this.containerToken = containerToken; + } + + public ContainerId getContainerId() { + return containerId; + } + + public NodeId getNodeId() { + return nodeId; + } + + public Token getContainerToken() { + return containerToken; + } + } + + protected static class StartContainerEvent extends ContainerEvent { + private Container container; + private ContainerLaunchContext containerLaunchContext; + + public StartContainerEvent(Container container, + ContainerLaunchContext containerLaunchContext) { + super(container.getId(), container.getNodeId(), + container.getContainerToken(), ContainerEventType.START_CONTAINER); + this.container = container; + this.containerLaunchContext = containerLaunchContext; + } + + public Container getContainer() { + return container; + } + + public ContainerLaunchContext getContainerLaunchContext() { + return containerLaunchContext; + } + } + + protected static class StatefulContainer implements + EventHandler { + + protected final static StateMachineFactory stateMachineFactory + = new StateMachineFactory(ContainerState.PREP) + + // Transitions from PREP state + .addTransition(ContainerState.PREP, + EnumSet.of(ContainerState.RUNNING, ContainerState.FAILED), + ContainerEventType.START_CONTAINER, + new StartContainerTransition()) + .addTransition(ContainerState.PREP, ContainerState.DONE, + ContainerEventType.STOP_CONTAINER, new OutOfOrderTransition()) + + // Transitions from RUNNING state + // RUNNING -> RUNNING should be the invalid transition + .addTransition(ContainerState.RUNNING, + EnumSet.of(ContainerState.DONE, ContainerState.FAILED), + ContainerEventType.STOP_CONTAINER, + new StopContainerTransition()) + + // Transition from DONE state + .addTransition(ContainerState.DONE, ContainerState.DONE, + EnumSet.of(ContainerEventType.START_CONTAINER, + ContainerEventType.STOP_CONTAINER)) + + // Transition from FAILED state + .addTransition(ContainerState.FAILED, ContainerState.FAILED, + EnumSet.of(ContainerEventType.START_CONTAINER, + ContainerEventType.STOP_CONTAINER)); + + protected static class StartContainerTransition implements + MultipleArcTransition { + + @Override + public ContainerState transition( + StatefulContainer container, ContainerEvent event) { + ContainerId containerId = event.getContainerId(); + try { + StartContainerEvent scEvent = null; + if (event instanceof StartContainerEvent) { + scEvent = (StartContainerEvent) event; + } + assert scEvent != null; + Map allServiceResponse = + container.nmClientAsync.getClient().startContainer( + scEvent.getContainer(), scEvent.getContainerLaunchContext()); + try { + container.nmClientAsync.getCallbackHandler().onContainerStarted( + containerId, allServiceResponse); + } catch (Throwable thr) { + // Don't process user created unchecked exception + LOG.info("Unchecked exception is thrown from onContainerStarted for " + + "Container " + containerId, thr); + } + return ContainerState.RUNNING; + } catch (YarnException e) { + return onExceptionRaised(container, event, e); + } catch (IOException e) { + return onExceptionRaised(container, event, e); + } catch (Throwable t) { + return onExceptionRaised(container, event, t); + } + } + + private ContainerState onExceptionRaised(StatefulContainer container, + ContainerEvent event, Throwable t) { + try { + container.nmClientAsync.getCallbackHandler().onStartContainerError( + event.getContainerId(), t); + } catch (Throwable thr) { + // Don't process user created unchecked exception + LOG.info( + "Unchecked exception is thrown from onStartContainerError for " + + "Container " + event.getContainerId(), thr); + } + return ContainerState.FAILED; + } + } + + protected static class StopContainerTransition implements + MultipleArcTransition { + + @Override + public ContainerState transition( + StatefulContainer container, ContainerEvent event) { + ContainerId containerId = event.getContainerId(); + try { + container.nmClientAsync.getClient().stopContainer( + containerId, event.getNodeId(), event.getContainerToken()); + try { + container.nmClientAsync.getCallbackHandler().onContainerStopped( + event.getContainerId()); + } catch (Throwable thr) { + // Don't process user created unchecked exception + LOG.info("Unchecked exception is thrown from onContainerStopped for " + + "Container " + event.getContainerId(), thr); + } + return ContainerState.DONE; + } catch (YarnException e) { + return onExceptionRaised(container, event, e); + } catch (IOException e) { + return onExceptionRaised(container, event, e); + } catch (Throwable t) { + return onExceptionRaised(container, event, t); + } + } + + private ContainerState onExceptionRaised(StatefulContainer container, + ContainerEvent event, Throwable t) { + try { + container.nmClientAsync.getCallbackHandler().onStopContainerError( + event.getContainerId(), t); + } catch (Throwable thr) { + // Don't process user created unchecked exception + LOG.info("Unchecked exception is thrown from onStopContainerError for " + + "Container " + event.getContainerId(), thr); + } + return ContainerState.FAILED; + } + } + + protected static class OutOfOrderTransition implements + SingleArcTransition { + + protected static final String STOP_BEFORE_START_ERROR_MSG = + "Container was killed before it was launched"; + + @Override + public void transition(StatefulContainer container, ContainerEvent event) { + try { + container.nmClientAsync.getCallbackHandler().onStartContainerError( + event.getContainerId(), + RPCUtil.getRemoteException(STOP_BEFORE_START_ERROR_MSG)); + } catch (Throwable thr) { + // Don't process user created unchecked exception + LOG.info( + "Unchecked exception is thrown from onStartContainerError for " + + "Container " + event.getContainerId(), thr); + } + } + } + + private final NMClientAsync nmClientAsync; + private final ContainerId containerId; + private final StateMachine stateMachine; + private final ReadLock readLock; + private final WriteLock writeLock; + + public StatefulContainer(NMClientAsync client, ContainerId containerId) { + this.nmClientAsync = client; + this.containerId = containerId; + stateMachine = stateMachineFactory.make(this); + ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); + readLock = lock.readLock(); + writeLock = lock.writeLock(); + } + + @Override + public void handle(ContainerEvent event) { + writeLock.lock(); + try { + try { + this.stateMachine.doTransition(event.getType(), event); + } catch (InvalidStateTransitonException e) { + LOG.error("Can't handle this event at current state", e); + } + } finally { + writeLock.unlock(); + } + } + + public ContainerId getContainerId() { + return containerId; + } + + public ContainerState getState() { + readLock.lock(); + try { + return stateMachine.getCurrentState(); + } finally { + readLock.unlock(); + } + } + } + + protected class ContainerEventProcessor implements Runnable { + protected ContainerEvent event; + + public ContainerEventProcessor(ContainerEvent event) { + this.event = event; + } + + @Override + public void run() { + ContainerId containerId = event.getContainerId(); + LOG.info("Processing Event " + event + " for Container " + containerId); + if (event.getType() == ContainerEventType.QUERY_CONTAINER) { + try { + ContainerStatus containerStatus = client.getContainerStatus( + containerId, event.getNodeId(), event.getContainerToken()); + try { + callbackHandler.onContainerStatusReceived( + containerId, containerStatus); + } catch (Throwable thr) { + // Don't process user created unchecked exception + LOG.info( + "Unchecked exception is thrown from onContainerStatusReceived" + + " for Container " + event.getContainerId(), thr); + } + } catch (YarnException e) { + onExceptionRaised(containerId, e); + } catch (IOException e) { + onExceptionRaised(containerId, e); + } catch (Throwable t) { + onExceptionRaised(containerId, t); + } + } else { + StatefulContainer container = containers.get(containerId); + if (container == null) { + LOG.info("Container " + containerId + " is already stopped or failed"); + } else { + container.handle(event); + if (isCompletelyDone(container)) { + containers.remove(containerId); + } + } + } + } + + private void onExceptionRaised(ContainerId containerId, Throwable t) { + try { + callbackHandler.onGetContainerStatusError(containerId, t); + } catch (Throwable thr) { + // Don't process user created unchecked exception + LOG.info("Unchecked exception is thrown from onGetContainerStatusError" + + " for Container " + containerId, thr); + } + } + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java new file mode 100644 index 0000000..a61ae0c --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java @@ -0,0 +1,585 @@ +/** +* 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.hadoop.yarn.client.api.impl; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.security.PrivilegedAction; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.TreeSet; +import java.util.concurrent.ConcurrentHashMap; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.ipc.RPC; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; +import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest; +import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; +import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest; +import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest; +import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.hadoop.yarn.api.records.NMToken; +import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.ResourceRequest; +import org.apache.hadoop.yarn.api.records.Token; +import org.apache.hadoop.yarn.client.api.AMRMClient; +import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; +import org.apache.hadoop.yarn.factories.RecordFactory; +import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; +import org.apache.hadoop.yarn.ipc.YarnRPC; +import org.apache.hadoop.yarn.util.RackResolver; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Joiner; + +// TODO check inputs for null etc. YARN-654 + +@Private +@Unstable +public class AMRMClientImpl extends AMRMClient { + + private static final Log LOG = LogFactory.getLog(AMRMClientImpl.class); + + private final RecordFactory recordFactory = + RecordFactoryProvider.getRecordFactory(null); + + private int lastResponseId = 0; + private ConcurrentHashMap nmTokens; + + protected ApplicationMasterProtocol rmClient; + protected final ApplicationAttemptId appAttemptId; + protected Resource clusterAvailableResources; + protected int clusterNodeCount; + + class ResourceRequestInfo { + ResourceRequest remoteRequest; + LinkedHashSet containerRequests; + + ResourceRequestInfo(Priority priority, String resourceName, + Resource capability) { + remoteRequest = ResourceRequest.newInstance(priority, resourceName, + capability, 0); + containerRequests = new LinkedHashSet(); + } + } + + + /** + * Class compares Resource by memory then cpu in reverse order + */ + class ResourceReverseMemoryThenCpuComparator implements Comparator { + @Override + public int compare(Resource arg0, Resource arg1) { + int mem0 = arg0.getMemory(); + int mem1 = arg1.getMemory(); + int cpu0 = arg0.getVirtualCores(); + int cpu1 = arg1.getVirtualCores(); + if(mem0 == mem1) { + if(cpu0 == cpu1) { + return 0; + } + if(cpu0 < cpu1) { + return 1; + } + return -1; + } + if(mem0 < mem1) { + return 1; + } + return -1; + } + } + + static boolean canFit(Resource arg0, Resource arg1) { + int mem0 = arg0.getMemory(); + int mem1 = arg1.getMemory(); + int cpu0 = arg0.getVirtualCores(); + int cpu1 = arg1.getVirtualCores(); + + if(mem0 <= mem1 && cpu0 <= cpu1) { + return true; + } + return false; + } + + //Key -> Priority + //Value -> Map + //Key->ResourceName (e.g., nodename, rackname, *) + //Value->Map + //Key->Resource Capability + //Value->ResourceRequest + protected final + Map>> + remoteRequestsTable = + new TreeMap>>(); + + protected final Set ask = new TreeSet( + new org.apache.hadoop.yarn.api.records.ResourceRequest.ResourceRequestComparator()); + protected final Set release = new TreeSet(); + + public AMRMClientImpl(ApplicationAttemptId appAttemptId) { + super(AMRMClientImpl.class.getName()); + this.appAttemptId = appAttemptId; + this.nmTokens = new ConcurrentHashMap(); + } + + @Override + protected void serviceInit(Configuration conf) throws Exception { + RackResolver.init(conf); + super.serviceInit(conf); + } + + @Override + protected void serviceStart() throws Exception { + final YarnConfiguration conf = new YarnConfiguration(getConfig()); + final YarnRPC rpc = YarnRPC.create(conf); + final InetSocketAddress rmAddress = conf.getSocketAddr( + YarnConfiguration.RM_SCHEDULER_ADDRESS, + YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS, + YarnConfiguration.DEFAULT_RM_SCHEDULER_PORT); + + UserGroupInformation currentUser; + try { + currentUser = UserGroupInformation.getCurrentUser(); + } catch (IOException e) { + throw new YarnRuntimeException(e); + } + + // CurrentUser should already have AMToken loaded. + rmClient = currentUser.doAs(new PrivilegedAction() { + @Override + public ApplicationMasterProtocol run() { + return (ApplicationMasterProtocol) rpc.getProxy(ApplicationMasterProtocol.class, rmAddress, + conf); + } + }); + LOG.debug("Connecting to ResourceManager at " + rmAddress); + super.serviceStart(); + } + + @Override + protected void serviceStop() throws Exception { + if (this.rmClient != null) { + RPC.stopProxy(this.rmClient); + } + super.serviceStop(); + } + + @Override + public RegisterApplicationMasterResponse registerApplicationMaster( + String appHostName, int appHostPort, String appTrackingUrl) + throws YarnException, IOException { + // do this only once ??? + RegisterApplicationMasterRequest request = recordFactory + .newRecordInstance(RegisterApplicationMasterRequest.class); + synchronized (this) { + request.setApplicationAttemptId(appAttemptId); + } + request.setHost(appHostName); + request.setRpcPort(appHostPort); + if(appTrackingUrl != null) { + request.setTrackingUrl(appTrackingUrl); + } + RegisterApplicationMasterResponse response = rmClient + .registerApplicationMaster(request); + return response; + } + + @Override + public AllocateResponse allocate(float progressIndicator) + throws YarnException, IOException { + AllocateResponse allocateResponse = null; + ArrayList askList = null; + ArrayList releaseList = null; + AllocateRequest allocateRequest = null; + + try { + synchronized (this) { + askList = new ArrayList(ask); + releaseList = new ArrayList(release); + // optimistically clear this collection assuming no RPC failure + ask.clear(); + release.clear(); + allocateRequest = + AllocateRequest.newInstance(appAttemptId, lastResponseId, + progressIndicator, askList, releaseList, null); + } + + allocateResponse = rmClient.allocate(allocateRequest); + + synchronized (this) { + // update these on successful RPC + clusterNodeCount = allocateResponse.getNumClusterNodes(); + lastResponseId = allocateResponse.getResponseId(); + clusterAvailableResources = allocateResponse.getAvailableResources(); + if (!allocateResponse.getNMTokens().isEmpty()) { + populateNMTokens(allocateResponse); + } + } + } finally { + // TODO how to differentiate remote yarn exception vs error in rpc + if(allocateResponse == null) { + // we hit an exception in allocate() + // preserve ask and release for next call to allocate() + synchronized (this) { + release.addAll(releaseList); + // requests could have been added or deleted during call to allocate + // If requests were added/removed then there is nothing to do since + // the ResourceRequest object in ask would have the actual new value. + // If ask does not have this ResourceRequest then it was unchanged and + // so we can add the value back safely. + // This assumes that there will no concurrent calls to allocate() and + // so we dont have to worry about ask being changed in the + // synchronized block at the beginning of this method. + for(ResourceRequest oldAsk : askList) { + if(!ask.contains(oldAsk)) { + ask.add(oldAsk); + } + } + } + } + } + return allocateResponse; + } + + @Private + @VisibleForTesting + protected void populateNMTokens(AllocateResponse allocateResponse) { + for (NMToken token : allocateResponse.getNMTokens()) { + String nodeId = token.getNodeId().toString(); + if (nmTokens.containsKey(nodeId)) { + LOG.debug("Replacing token for : " + nodeId); + } else { + LOG.debug("Received new token for : " + nodeId); + } + nmTokens.put(nodeId, token.getToken()); + } + } + + @Override + public void unregisterApplicationMaster(FinalApplicationStatus appStatus, + String appMessage, String appTrackingUrl) throws YarnException, + IOException { + FinishApplicationMasterRequest request = recordFactory + .newRecordInstance(FinishApplicationMasterRequest.class); + request.setAppAttemptId(appAttemptId); + request.setFinalApplicationStatus(appStatus); + if(appMessage != null) { + request.setDiagnostics(appMessage); + } + if(appTrackingUrl != null) { + request.setTrackingUrl(appTrackingUrl); + } + rmClient.finishApplicationMaster(request); + } + + @Override + public synchronized void addContainerRequest(T req) { + Set allRacks = new HashSet(); + if (req.getRacks() != null) { + allRacks.addAll(req.getRacks()); + if(req.getRacks().size() != allRacks.size()) { + Joiner joiner = Joiner.on(','); + LOG.warn("ContainerRequest has duplicate racks: " + + joiner.join(req.getRacks())); + } + } + allRacks.addAll(resolveRacks(req.getNodes())); + + if (req.getNodes() != null) { + HashSet dedupedNodes = new HashSet(req.getNodes()); + if(dedupedNodes.size() != req.getNodes().size()) { + Joiner joiner = Joiner.on(','); + LOG.warn("ContainerRequest has duplicate nodes: " + + joiner.join(req.getNodes())); + } + for (String node : dedupedNodes) { + // Ensure node requests are accompanied by requests for + // corresponding rack + addResourceRequest(req.getPriority(), node, req.getCapability(), + req.getContainerCount(), req); + } + } + + for (String rack : allRacks) { + addResourceRequest(req.getPriority(), rack, req.getCapability(), + req.getContainerCount(), req); + } + + // Off-switch + addResourceRequest(req.getPriority(), ResourceRequest.ANY, req.getCapability(), + req.getContainerCount(), req); + } + + @Override + public synchronized void removeContainerRequest(T req) { + Set allRacks = new HashSet(); + if (req.getRacks() != null) { + allRacks.addAll(req.getRacks()); + } + allRacks.addAll(resolveRacks(req.getNodes())); + + // Update resource requests + if (req.getNodes() != null) { + for (String node : new HashSet(req.getNodes())) { + decResourceRequest(req.getPriority(), node, req.getCapability(), + req.getContainerCount(), req); + } + } + + for (String rack : allRacks) { + decResourceRequest(req.getPriority(), rack, req.getCapability(), + req.getContainerCount(), req); + } + + decResourceRequest(req.getPriority(), ResourceRequest.ANY, req.getCapability(), + req.getContainerCount(), req); + } + + @Override + public synchronized void releaseAssignedContainer(ContainerId containerId) { + release.add(containerId); + } + + @Override + public synchronized Resource getClusterAvailableResources() { + return clusterAvailableResources; + } + + @Override + public synchronized int getClusterNodeCount() { + return clusterNodeCount; + } + + @Override + public synchronized List> getMatchingRequests( + Priority priority, + String resourceName, + Resource capability) { + List> list = new LinkedList>(); + Map> remoteRequests = + this.remoteRequestsTable.get(priority); + if (remoteRequests == null) { + return list; + } + TreeMap reqMap = remoteRequests + .get(resourceName); + if (reqMap == null) { + return list; + } + + ResourceRequestInfo resourceRequestInfo = reqMap.get(capability); + if (resourceRequestInfo != null) { + list.add(resourceRequestInfo.containerRequests); + return list; + } + + // no exact match. Container may be larger than what was requested. + // get all resources <= capability. map is reverse sorted. + SortedMap tailMap = + reqMap.tailMap(capability); + for(Map.Entry entry : tailMap.entrySet()) { + if(canFit(entry.getKey(), capability)) { + // match found that fits in the larger resource + list.add(entry.getValue().containerRequests); + } + } + + // no match found + return list; + } + + private Set resolveRacks(List nodes) { + Set racks = new HashSet(); + if (nodes != null) { + for (String node : nodes) { + // Ensure node requests are accompanied by requests for + // corresponding rack + String rack = RackResolver.resolve(node).getNetworkLocation(); + if (rack == null) { + LOG.warn("Failed to resolve rack for node " + node + "."); + } else { + racks.add(rack); + } + } + } + + return racks; + } + + private void addResourceRequestToAsk(ResourceRequest remoteRequest) { + // This code looks weird but is needed because of the following scenario. + // A ResourceRequest is removed from the remoteRequestTable. A 0 container + // request is added to 'ask' to notify the RM about not needing it any more. + // Before the call to allocate, the user now requests more containers. If + // the locations of the 0 size request and the new request are the same + // (with the difference being only container count), then the set comparator + // will consider both to be the same and not add the new request to ask. So + // we need to check for the "same" request being present and remove it and + // then add it back. The comparator is container count agnostic. + // This should happen only rarely but we do need to guard against it. + if(ask.contains(remoteRequest)) { + ask.remove(remoteRequest); + } + ask.add(remoteRequest); + } + + private void addResourceRequest(Priority priority, String resourceName, + Resource capability, int containerCount, T req) { + Map> remoteRequests = + this.remoteRequestsTable.get(priority); + if (remoteRequests == null) { + remoteRequests = + new HashMap>(); + this.remoteRequestsTable.put(priority, remoteRequests); + if (LOG.isDebugEnabled()) { + LOG.debug("Added priority=" + priority); + } + } + TreeMap reqMap = + remoteRequests.get(resourceName); + if (reqMap == null) { + // capabilities are stored in reverse sorted order. smallest last. + reqMap = new TreeMap( + new ResourceReverseMemoryThenCpuComparator()); + remoteRequests.put(resourceName, reqMap); + } + ResourceRequestInfo resourceRequestInfo = reqMap.get(capability); + if (resourceRequestInfo == null) { + resourceRequestInfo = + new ResourceRequestInfo(priority, resourceName, capability); + reqMap.put(capability, resourceRequestInfo); + } + + resourceRequestInfo.remoteRequest.setNumContainers( + resourceRequestInfo.remoteRequest.getNumContainers() + containerCount); + + if(req instanceof StoredContainerRequest) { + resourceRequestInfo.containerRequests.add(req); + } + + // Note this down for next interaction with ResourceManager + addResourceRequestToAsk(resourceRequestInfo.remoteRequest); + + if (LOG.isDebugEnabled()) { + LOG.debug("addResourceRequest:" + " applicationId=" + + appAttemptId + " priority=" + priority.getPriority() + + " resourceName=" + resourceName + " numContainers=" + + resourceRequestInfo.remoteRequest.getNumContainers() + + " #asks=" + ask.size()); + } + } + + private void decResourceRequest(Priority priority, + String resourceName, + Resource capability, + int containerCount, + T req) { + Map> remoteRequests = + this.remoteRequestsTable.get(priority); + + if(remoteRequests == null) { + if (LOG.isDebugEnabled()) { + LOG.debug("Not decrementing resource as priority " + priority + + " is not present in request table"); + } + return; + } + + Map reqMap = remoteRequests.get(resourceName); + if (reqMap == null) { + if (LOG.isDebugEnabled()) { + LOG.debug("Not decrementing resource as " + resourceName + + " is not present in request table"); + } + return; + } + ResourceRequestInfo resourceRequestInfo = reqMap.get(capability); + + if (LOG.isDebugEnabled()) { + LOG.debug("BEFORE decResourceRequest:" + " applicationId=" + + appAttemptId + " priority=" + priority.getPriority() + + " resourceName=" + resourceName + " numContainers=" + + resourceRequestInfo.remoteRequest.getNumContainers() + + " #asks=" + ask.size()); + } + + resourceRequestInfo.remoteRequest.setNumContainers( + resourceRequestInfo.remoteRequest.getNumContainers() - containerCount); + + if(req instanceof StoredContainerRequest) { + resourceRequestInfo.containerRequests.remove(req); + } + + if(resourceRequestInfo.remoteRequest.getNumContainers() < 0) { + // guard against spurious removals + resourceRequestInfo.remoteRequest.setNumContainers(0); + } + // send the ResourceRequest to RM even if is 0 because it needs to override + // a previously sent value. If ResourceRequest was not sent previously then + // sending 0 aught to be a no-op on RM + addResourceRequestToAsk(resourceRequestInfo.remoteRequest); + + // delete entries from map if no longer needed + if (resourceRequestInfo.remoteRequest.getNumContainers() == 0) { + reqMap.remove(capability); + if (reqMap.size() == 0) { + remoteRequests.remove(resourceName); + } + if (remoteRequests.size() == 0) { + remoteRequestsTable.remove(priority); + } + } + + if (LOG.isDebugEnabled()) { + LOG.info("AFTER decResourceRequest:" + " applicationId=" + + appAttemptId + " priority=" + priority.getPriority() + + " resourceName=" + resourceName + " numContainers=" + + resourceRequestInfo.remoteRequest.getNumContainers() + + " #asks=" + ask.size()); + } + } + + @Override + public ConcurrentHashMap getNMTokens() { + return nmTokens; + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/NMClientImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/NMClientImpl.java new file mode 100644 index 0000000..7e58993 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/NMClientImpl.java @@ -0,0 +1,409 @@ +/** + * 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.hadoop.yarn.client.api.impl; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.nio.ByteBuffer; +import java.security.PrivilegedAction; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.ipc.RPC; +import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse; +import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; +import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse; +import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest; +import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; +import org.apache.hadoop.yarn.api.records.ContainerStatus; +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.api.records.Token; +import org.apache.hadoop.yarn.client.api.NMClient; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.ipc.RPCUtil; +import org.apache.hadoop.yarn.ipc.YarnRPC; +import org.apache.hadoop.yarn.security.ContainerTokenIdentifier; +import org.apache.hadoop.yarn.util.ProtoUtils; +import org.apache.hadoop.yarn.util.Records; + +/** + *

+ * This class implements {@link NMClient}. All the APIs are blocking. + *

+ * + *

+ * By default, this client stops all the running containers that are started by + * it when it stops. It can be disabled via + * {@link #cleanupRunningContainersOnStop}, in which case containers will + * continue to run even after this client is stopped and till the application + * runs at which point ResourceManager will forcefully kill them. + *

+ * + *

+ * Note that the blocking APIs ensure the RPC calls to NodeManager + * are executed immediately, and the responses are received before these APIs + * return. However, when {@link #startContainer} or {@link #stopContainer} + * returns, NodeManager may still need some time to either start + * or stop the container because of its asynchronous implementation. Therefore, + * {@link #getContainerStatus} is likely to return a transit container status + * if it is executed immediately after {@link #startContainer} or + * {@link #stopContainer}. + *

+ */ +@Private +@Unstable +public class NMClientImpl extends NMClient { + + private static final Log LOG = LogFactory.getLog(NMClientImpl.class); + + // The logically coherent operations on startedContainers is synchronized to + // ensure they are atomic + protected ConcurrentMap startedContainers = + new ConcurrentHashMap(); + + //enabled by default + private final AtomicBoolean cleanupRunningContainers = new AtomicBoolean(true); + + public NMClientImpl() { + super(NMClientImpl.class.getName()); + } + + public NMClientImpl(String name) { + super(name); + } + + @Override + protected void serviceStop() throws Exception { + // Usually, started-containers are stopped when this client stops. Unless + // the flag cleanupRunningContainers is set to false. + if (getCleanupRunningContainers().get()) { + cleanupRunningContainers(); + } + super.serviceStop(); + } + + protected synchronized void cleanupRunningContainers() { + for (StartedContainer startedContainer : startedContainers.values()) { + try { + stopContainer(startedContainer.getContainerId(), + startedContainer.getNodeId(), + startedContainer.getContainerToken()); + } catch (YarnException e) { + LOG.error("Failed to stop Container " + + startedContainer.getContainerId() + + "when stopping NMClientImpl"); + } catch (IOException e) { + LOG.error("Failed to stop Container " + + startedContainer.getContainerId() + + "when stopping NMClientImpl"); + } + } + } + + @Override + public void cleanupRunningContainersOnStop(boolean enabled) { + getCleanupRunningContainers().set(enabled); + } + + protected static class StartedContainer { + private ContainerId containerId; + private NodeId nodeId; + private Token containerToken; + private boolean stopped; + + public StartedContainer(ContainerId containerId, NodeId nodeId, + Token containerToken) { + this.containerId = containerId; + this.nodeId = nodeId; + this.containerToken = containerToken; + stopped = false; + } + + public ContainerId getContainerId() { + return containerId; + } + + public NodeId getNodeId() { + return nodeId; + } + + public Token getContainerToken() { + return containerToken; + } + } + + protected static final class NMCommunicator extends AbstractService { + private ContainerId containerId; + private NodeId nodeId; + private Token containerToken; + private ContainerManagementProtocol containerManager; + + public NMCommunicator(ContainerId containerId, NodeId nodeId, + Token containerToken) { + super(NMCommunicator.class.getName()); + this.containerId = containerId; + this.nodeId = nodeId; + this.containerToken = containerToken; + } + + @Override + protected void serviceStart() throws Exception { + final YarnRPC rpc = YarnRPC.create(getConfig()); + + final InetSocketAddress containerAddress = + NetUtils.createSocketAddr(nodeId.toString()); + + // the user in createRemoteUser in this context has to be ContainerId + UserGroupInformation currentUser = + UserGroupInformation.createRemoteUser(containerId.toString()); + + org.apache.hadoop.security.token.Token token = + ProtoUtils.convertFromProtoFormat(containerToken, containerAddress); + currentUser.addToken(token); + + containerManager = currentUser + .doAs(new PrivilegedAction() { + @Override + public ContainerManagementProtocol run() { + return (ContainerManagementProtocol) rpc.getProxy(ContainerManagementProtocol.class, + containerAddress, getConfig()); + } + }); + + LOG.debug("Connecting to ContainerManager at " + containerAddress); + super.serviceStart(); + } + + @Override + protected void serviceStop() throws Exception { + if (this.containerManager != null) { + RPC.stopProxy(this.containerManager); + + if (LOG.isDebugEnabled()) { + InetSocketAddress containerAddress = + NetUtils.createSocketAddr(nodeId.toString()); + LOG.debug("Disconnecting from ContainerManager at " + + containerAddress); + } + } + super.serviceStop(); + } + + public synchronized Map startContainer( + Container container, ContainerLaunchContext containerLaunchContext) + throws YarnException, IOException { + if (!container.getId().equals(containerId)) { + throw new IllegalArgumentException( + "NMCommunicator's containerId mismatches the given Container's"); + } + StartContainerResponse startResponse = null; + try { + StartContainerRequest startRequest = + Records.newRecord(StartContainerRequest.class); + startRequest.setContainerToken(container.getContainerToken()); + startRequest.setContainerLaunchContext(containerLaunchContext); + startResponse = containerManager.startContainer(startRequest); + if (LOG.isDebugEnabled()) { + LOG.debug("Started Container " + containerId); + } + } catch (YarnException e) { + LOG.warn("Container " + containerId + " failed to start", e); + throw e; + } catch (IOException e) { + LOG.warn("Container " + containerId + " failed to start", e); + throw e; + } + return startResponse.getAllServiceResponse(); + } + + public synchronized void stopContainer() throws YarnException, + IOException { + try { + StopContainerRequest stopRequest = + Records.newRecord(StopContainerRequest.class); + stopRequest.setContainerId(containerId); + containerManager.stopContainer(stopRequest); + if (LOG.isDebugEnabled()) { + LOG.debug("Stopped Container " + containerId); + } + } catch (YarnException e) { + LOG.warn("Container " + containerId + " failed to stop", e); + throw e; + } catch (IOException e) { + LOG.warn("Container " + containerId + " failed to stop", e); + throw e; + } + } + + public synchronized ContainerStatus getContainerStatus() + throws YarnException, IOException { + GetContainerStatusResponse statusResponse = null; + try { + GetContainerStatusRequest statusRequest = + Records.newRecord(GetContainerStatusRequest.class); + statusRequest.setContainerId(containerId); + statusResponse = containerManager.getContainerStatus(statusRequest); + if (LOG.isDebugEnabled()) { + LOG.debug("Got the status of Container " + containerId); + } + } catch (YarnException e) { + LOG.warn( + "Unable to get the status of Container " + containerId, e); + throw e; + } catch (IOException e) { + LOG.warn( + "Unable to get the status of Container " + containerId, e); + throw e; + } + return statusResponse.getStatus(); + } + } + + @Override + public Map startContainer( + Container container, ContainerLaunchContext containerLaunchContext) + throws YarnException, IOException { + // Do synchronization on StartedContainer to prevent race condition + // between startContainer and stopContainer + synchronized (addStartedContainer(container)) { + Map allServiceResponse; + NMCommunicator nmCommunicator = null; + try { + nmCommunicator = new NMCommunicator(container.getId(), + container.getNodeId(), container.getContainerToken()); + nmCommunicator.init(getConfig()); + nmCommunicator.start(); + allServiceResponse = + nmCommunicator.startContainer(container, containerLaunchContext); + } catch (YarnException e) { + // Remove the started container if it failed to start + removeStartedContainer(container.getId()); + throw e; + } catch (IOException e) { + removeStartedContainer(container.getId()); + throw e; + } catch (Throwable t) { + removeStartedContainer(container.getId()); + throw RPCUtil.getRemoteException(t); + } finally { + if (nmCommunicator != null) { + nmCommunicator.stop(); + } + } + return allServiceResponse; + } + + // Three choices: + // 1. starting and releasing the proxy before and after each interaction + // 2. starting the proxy when starting the container and releasing it when + // stopping the container + // 3. starting the proxy when starting the container and releasing it when + // stopping the client + // Adopt 1 currently + } + + @Override + public void stopContainer(ContainerId containerId, NodeId nodeId, + Token containerToken) throws YarnException, IOException { + StartedContainer startedContainer = getStartedContainer(containerId); + if (startedContainer == null) { + throw RPCUtil.getRemoteException("Container " + containerId + + " is either not started yet or already stopped"); + } + // Only allow one request of stopping the container to move forward + // When entering the block, check whether the precursor has already stopped + // the container + synchronized (startedContainer) { + if (startedContainer.stopped) { + return; + } + NMCommunicator nmCommunicator = null; + try { + nmCommunicator = + new NMCommunicator(containerId, nodeId, containerToken); + nmCommunicator.init(getConfig()); + nmCommunicator.start(); + nmCommunicator.stopContainer(); + } finally { + if (nmCommunicator != null) { + nmCommunicator.stop(); + } + startedContainer.stopped = true; + removeStartedContainer(containerId); + } + } + } + + @Override + public ContainerStatus getContainerStatus(ContainerId containerId, + NodeId nodeId, Token containerToken) + throws YarnException, IOException { + NMCommunicator nmCommunicator = null; + try { + nmCommunicator = new NMCommunicator(containerId, nodeId, containerToken); + nmCommunicator.init(getConfig()); + nmCommunicator.start(); + ContainerStatus containerStatus = nmCommunicator.getContainerStatus(); + return containerStatus; + } finally { + if (nmCommunicator != null) { + nmCommunicator.stop(); + } + } + } + + protected synchronized StartedContainer addStartedContainer( + Container container) throws YarnException, IOException { + if (startedContainers.containsKey(container.getId())) { + throw RPCUtil.getRemoteException("Container " + container.getId() + + " is already started"); + } + StartedContainer startedContainer = new StartedContainer(container.getId(), + container.getNodeId(), container.getContainerToken()); + startedContainers.put(startedContainer.getContainerId(), startedContainer); + return startedContainer; + } + + protected synchronized void removeStartedContainer(ContainerId containerId) { + startedContainers.remove(containerId); + } + + protected synchronized StartedContainer getStartedContainer( + ContainerId containerId) { + return startedContainers.get(containerId); + } + + public AtomicBoolean getCleanupRunningContainers() { + return cleanupRunningContainers; + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java new file mode 100644 index 0000000..1d12092 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java @@ -0,0 +1,308 @@ +/** +* 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.hadoop.yarn.client.api.impl; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.ipc.RPC; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; +import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest; +import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest; +import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; +import org.apache.hadoop.yarn.api.records.NodeReport; +import org.apache.hadoop.yarn.api.records.QueueInfo; +import org.apache.hadoop.yarn.api.records.QueueUserACLInfo; +import org.apache.hadoop.yarn.api.records.Token; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; +import org.apache.hadoop.yarn.api.records.YarnClusterMetrics; +import org.apache.hadoop.yarn.client.api.YarnClient; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.ipc.YarnRPC; +import org.apache.hadoop.yarn.util.Records; + +@Private +@Unstable +public class YarnClientImpl extends YarnClient { + + private static final Log LOG = LogFactory.getLog(YarnClientImpl.class); + + protected ApplicationClientProtocol rmClient; + protected InetSocketAddress rmAddress; + protected long statePollIntervalMillis; + + private static final String ROOT = "root"; + + public YarnClientImpl() { + this(null); + } + + public YarnClientImpl(InetSocketAddress rmAddress) { + this(YarnClientImpl.class.getName(), rmAddress); + } + + public YarnClientImpl(String name, InetSocketAddress rmAddress) { + super(name); + this.rmAddress = rmAddress; + } + + private static InetSocketAddress getRmAddress(Configuration conf) { + return conf.getSocketAddr(YarnConfiguration.RM_ADDRESS, + YarnConfiguration.DEFAULT_RM_ADDRESS, YarnConfiguration.DEFAULT_RM_PORT); + } + + @Override + protected void serviceInit(Configuration conf) throws Exception { + if (this.rmAddress == null) { + this.rmAddress = getRmAddress(conf); + } + statePollIntervalMillis = conf.getLong( + YarnConfiguration.YARN_CLIENT_APP_SUBMISSION_POLL_INTERVAL_MS, + YarnConfiguration.DEFAULT_YARN_CLIENT_APP_SUBMISSION_POLL_INTERVAL_MS); + super.serviceInit(conf); + } + + @Override + protected void serviceStart() throws Exception { + YarnRPC rpc = YarnRPC.create(getConfig()); + + this.rmClient = (ApplicationClientProtocol) rpc.getProxy( + ApplicationClientProtocol.class, rmAddress, getConfig()); + if (LOG.isDebugEnabled()) { + LOG.debug("Connecting to ResourceManager at " + rmAddress); + } + super.serviceStart(); + } + + @Override + protected void serviceStop() throws Exception { + if (this.rmClient != null) { + RPC.stopProxy(this.rmClient); + } + super.serviceStop(); + } + + @Override + public GetNewApplicationResponse getNewApplication() + throws YarnException, IOException { + GetNewApplicationRequest request = + Records.newRecord(GetNewApplicationRequest.class); + return rmClient.getNewApplication(request); + } + + @Override + public ApplicationId + submitApplication(ApplicationSubmissionContext appContext) + throws YarnException, IOException { + ApplicationId applicationId = appContext.getApplicationId(); + appContext.setApplicationId(applicationId); + SubmitApplicationRequest request = + Records.newRecord(SubmitApplicationRequest.class); + request.setApplicationSubmissionContext(appContext); + rmClient.submitApplication(request); + + int pollCount = 0; + while (true) { + YarnApplicationState state = + getApplicationReport(applicationId).getYarnApplicationState(); + if (!state.equals(YarnApplicationState.NEW) && + !state.equals(YarnApplicationState.NEW_SAVING)) { + break; + } + // Notify the client through the log every 10 poll, in case the client + // is blocked here too long. + if (++pollCount % 10 == 0) { + LOG.info("Application submission is not finished, " + + "submitted application " + applicationId + + " is still in " + state); + } + try { + Thread.sleep(statePollIntervalMillis); + } catch (InterruptedException ie) { + } + } + + + LOG.info("Submitted application " + applicationId + " to ResourceManager" + + " at " + rmAddress); + return applicationId; + } + + @Override + public void killApplication(ApplicationId applicationId) + throws YarnException, IOException { + LOG.info("Killing application " + applicationId); + KillApplicationRequest request = + Records.newRecord(KillApplicationRequest.class); + request.setApplicationId(applicationId); + rmClient.forceKillApplication(request); + } + + @Override + public ApplicationReport getApplicationReport(ApplicationId appId) + throws YarnException, IOException { + GetApplicationReportRequest request = + Records.newRecord(GetApplicationReportRequest.class); + request.setApplicationId(appId); + GetApplicationReportResponse response = + rmClient.getApplicationReport(request); + return response.getApplicationReport(); + } + + @Override + public List getApplicationList() + throws YarnException, IOException { + GetAllApplicationsRequest request = + Records.newRecord(GetAllApplicationsRequest.class); + GetAllApplicationsResponse response = rmClient.getAllApplications(request); + return response.getApplicationList(); + } + + @Override + public YarnClusterMetrics getYarnClusterMetrics() throws YarnException, + IOException { + GetClusterMetricsRequest request = + Records.newRecord(GetClusterMetricsRequest.class); + GetClusterMetricsResponse response = rmClient.getClusterMetrics(request); + return response.getClusterMetrics(); + } + + @Override + public List getNodeReports() throws YarnException, + IOException { + GetClusterNodesRequest request = + Records.newRecord(GetClusterNodesRequest.class); + GetClusterNodesResponse response = rmClient.getClusterNodes(request); + return response.getNodeReports(); + } + + @Override + public Token getRMDelegationToken(Text renewer) + throws YarnException, IOException { + /* get the token from RM */ + GetDelegationTokenRequest rmDTRequest = + Records.newRecord(GetDelegationTokenRequest.class); + rmDTRequest.setRenewer(renewer.toString()); + GetDelegationTokenResponse response = + rmClient.getDelegationToken(rmDTRequest); + return response.getRMDelegationToken(); + } + + + private GetQueueInfoRequest + getQueueInfoRequest(String queueName, boolean includeApplications, + boolean includeChildQueues, boolean recursive) { + GetQueueInfoRequest request = Records.newRecord(GetQueueInfoRequest.class); + request.setQueueName(queueName); + request.setIncludeApplications(includeApplications); + request.setIncludeChildQueues(includeChildQueues); + request.setRecursive(recursive); + return request; + } + + @Override + public QueueInfo getQueueInfo(String queueName) throws YarnException, + IOException { + GetQueueInfoRequest request = + getQueueInfoRequest(queueName, true, false, false); + Records.newRecord(GetQueueInfoRequest.class); + return rmClient.getQueueInfo(request).getQueueInfo(); + } + + @Override + public List getQueueAclsInfo() throws YarnException, + IOException { + GetQueueUserAclsInfoRequest request = + Records.newRecord(GetQueueUserAclsInfoRequest.class); + return rmClient.getQueueUserAcls(request).getUserAclsInfoList(); + } + + @Override + public List getAllQueues() throws YarnException, + IOException { + List queues = new ArrayList(); + + QueueInfo rootQueue = + rmClient.getQueueInfo(getQueueInfoRequest(ROOT, false, true, true)) + .getQueueInfo(); + getChildQueues(rootQueue, queues, true); + return queues; + } + + @Override + public List getRootQueueInfos() throws YarnException, + IOException { + List queues = new ArrayList(); + + QueueInfo rootQueue = + rmClient.getQueueInfo(getQueueInfoRequest(ROOT, false, true, true)) + .getQueueInfo(); + getChildQueues(rootQueue, queues, false); + return queues; + } + + @Override + public List getChildQueueInfos(String parent) + throws YarnException, IOException { + List queues = new ArrayList(); + + QueueInfo parentQueue = + rmClient.getQueueInfo(getQueueInfoRequest(parent, false, true, false)) + .getQueueInfo(); + getChildQueues(parentQueue, queues, true); + return queues; + } + + private void getChildQueues(QueueInfo parent, List queues, + boolean recursive) { + List childQueues = parent.getChildQueues(); + + for (QueueInfo child : childQueues) { + queues.add(child); + if (recursive) { + getChildQueues(child, queues, recursive); + } + } + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java index 6bcd804..312aab2 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java @@ -27,12 +27,16 @@ import org.apache.commons.cli.GnuParser; import org.apache.commons.cli.HelpFormatter; import org.apache.commons.cli.Options; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.util.ToolRunner; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.util.ConverterUtils; +@Private +@Unstable public class ApplicationCLI extends YarnCLI { private static final String APPLICATIONS_PATTERN = "%30s\t%20s\t%20s\t%10s\t%10s\t%18s\t%18s\t%15s\t%35s" + diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeCLI.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeCLI.java index d8c05a8..1de45ed 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeCLI.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeCLI.java @@ -28,12 +28,16 @@ import org.apache.commons.cli.HelpFormatter; import org.apache.commons.cli.Options; import org.apache.commons.lang.time.DateFormatUtils; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.util.ToolRunner; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.NodeReport; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.util.ConverterUtils; +@Private +@Unstable public class NodeCLI extends YarnCLI { private static final String NODES_PATTERN = "%16s\t%10s\t%17s\t%18s" + System.getProperty("line.separator"); diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/YarnCLI.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/YarnCLI.java index aa7cb8d..5f86033 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/YarnCLI.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/YarnCLI.java @@ -19,12 +19,15 @@ import java.io.PrintStream; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.conf.Configured; import org.apache.hadoop.util.Tool; -import org.apache.hadoop.yarn.client.YarnClient; -import org.apache.hadoop.yarn.client.YarnClientImpl; +import org.apache.hadoop.yarn.client.api.YarnClient; import org.apache.hadoop.yarn.conf.YarnConfiguration; +@Private +@Unstable public abstract class YarnCLI extends Configured implements Tool { public static final String STATUS_CMD = "status"; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClient.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClient.java deleted file mode 100644 index 84a7252..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClient.java +++ /dev/null @@ -1,616 +0,0 @@ -/** - * 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.hadoop.yarn.client; - -import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -import java.io.IOException; -import java.util.Collection; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Set; -import java.util.TreeSet; -import java.util.concurrent.ConcurrentHashMap; - -import junit.framework.Assert; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.service.Service.STATE; -import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; -import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest; -import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; -import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; -import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest; -import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.ApplicationReport; -import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; -import org.apache.hadoop.yarn.api.records.ContainerState; -import org.apache.hadoop.yarn.api.records.ContainerStatus; -import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; -import org.apache.hadoop.yarn.api.records.NodeReport; -import org.apache.hadoop.yarn.api.records.Priority; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.api.records.ResourceRequest; -import org.apache.hadoop.yarn.api.records.Token; -import org.apache.hadoop.yarn.api.records.YarnApplicationState; -import org.apache.hadoop.yarn.client.AMRMClient.ContainerRequest; -import org.apache.hadoop.yarn.client.AMRMClient.StoredContainerRequest; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.hadoop.yarn.server.MiniYARNCluster; -import org.apache.hadoop.yarn.util.Records; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; - -public class TestAMRMClient { - static Configuration conf = null; - static MiniYARNCluster yarnCluster = null; - static YarnClient yarnClient = null; - static List nodeReports = null; - static ApplicationAttemptId attemptId = null; - static int nodeCount = 3; - - static Resource capability; - static Priority priority; - static String node; - static String rack; - static String[] nodes; - static String[] racks; - - @BeforeClass - public static void setup() throws Exception { - // start minicluster - conf = new YarnConfiguration(); - yarnCluster = new MiniYARNCluster(TestAMRMClient.class.getName(), nodeCount, 1, 1); - yarnCluster.init(conf); - yarnCluster.start(); - - // start rm client - yarnClient = YarnClient.createYarnClient(); - yarnClient.init(conf); - yarnClient.start(); - - // get node info - nodeReports = yarnClient.getNodeReports(); - - priority = Priority.newInstance(1); - capability = Resource.newInstance(1024, 1); - - node = nodeReports.get(0).getNodeId().getHost(); - rack = nodeReports.get(0).getRackName(); - nodes = new String[]{ node }; - racks = new String[]{ rack }; - } - - @Before - public void startApp() throws Exception { - // submit new app - GetNewApplicationResponse newApp = yarnClient.getNewApplication(); - ApplicationId appId = newApp.getApplicationId(); - - ApplicationSubmissionContext appContext = Records - .newRecord(ApplicationSubmissionContext.class); - // set the application id - appContext.setApplicationId(appId); - // set the application name - appContext.setApplicationName("Test"); - // Set the priority for the application master - Priority pri = Records.newRecord(Priority.class); - pri.setPriority(0); - appContext.setPriority(pri); - // Set the queue to which this application is to be submitted in the RM - appContext.setQueue("default"); - // Set up the container launch context for the application master - ContainerLaunchContext amContainer = Records - .newRecord(ContainerLaunchContext.class); - appContext.setAMContainerSpec(amContainer); - // unmanaged AM - appContext.setUnmanagedAM(true); - // Create the request to send to the applications manager - SubmitApplicationRequest appRequest = Records - .newRecord(SubmitApplicationRequest.class); - appRequest.setApplicationSubmissionContext(appContext); - // Submit the application to the applications manager - yarnClient.submitApplication(appContext); - - // wait for app to start - while (true) { - ApplicationReport appReport = yarnClient.getApplicationReport(appId); - if (appReport.getYarnApplicationState() == YarnApplicationState.ACCEPTED) { - attemptId = appReport.getCurrentApplicationAttemptId(); - break; - } - } - } - - @After - public void cancelApp() { - attemptId = null; - } - - @AfterClass - public static void tearDown() { - if (yarnClient != null && yarnClient.getServiceState() == STATE.STARTED) { - yarnClient.stop(); - } - if (yarnCluster != null && yarnCluster.getServiceState() == STATE.STARTED) { - yarnCluster.stop(); - } - } - - @Test (timeout=60000) - public void testAMRMClientMatchingFit() throws YarnException, IOException { - AMRMClient amClient = null; - try { - // start am rm client - amClient = AMRMClient.createAMRMClient(attemptId); - amClient.init(conf); - amClient.start(); - amClient.registerApplicationMaster("Host", 10000, ""); - - Resource capability1 = Resource.newInstance(1024, 2); - Resource capability2 = Resource.newInstance(1024, 1); - Resource capability3 = Resource.newInstance(1000, 2); - Resource capability4 = Resource.newInstance(2000, 1); - Resource capability5 = Resource.newInstance(1000, 3); - Resource capability6 = Resource.newInstance(2000, 1); - - StoredContainerRequest storedContainer1 = - new StoredContainerRequest(capability1, nodes, racks, priority); - StoredContainerRequest storedContainer2 = - new StoredContainerRequest(capability2, nodes, racks, priority); - StoredContainerRequest storedContainer3 = - new StoredContainerRequest(capability3, nodes, racks, priority); - StoredContainerRequest storedContainer4 = - new StoredContainerRequest(capability4, nodes, racks, priority); - StoredContainerRequest storedContainer5 = - new StoredContainerRequest(capability5, nodes, racks, priority); - StoredContainerRequest storedContainer6 = - new StoredContainerRequest(capability6, nodes, racks, priority); - amClient.addContainerRequest(storedContainer1); - amClient.addContainerRequest(storedContainer2); - amClient.addContainerRequest(storedContainer3); - amClient.addContainerRequest(storedContainer4); - amClient.addContainerRequest(storedContainer5); - amClient.addContainerRequest(storedContainer6); - - // test matching of containers - List> matches; - StoredContainerRequest storedRequest; - // exact match - Resource testCapability1 = Resource.newInstance(1024, 2); - matches = amClient.getMatchingRequests(priority, node, testCapability1); - verifyMatches(matches, 1); - storedRequest = matches.get(0).iterator().next(); - assertTrue(storedContainer1 == storedRequest); - amClient.removeContainerRequest(storedContainer1); - - // exact matching with order maintained - Resource testCapability2 = Resource.newInstance(2000, 1); - matches = amClient.getMatchingRequests(priority, node, testCapability2); - verifyMatches(matches, 2); - // must be returned in the order they were made - int i = 0; - for(StoredContainerRequest storedRequest1 : matches.get(0)) { - if(i++ == 0) { - assertTrue(storedContainer4 == storedRequest1); - } else { - assertTrue(storedContainer6 == storedRequest1); - } - } - amClient.removeContainerRequest(storedContainer6); - - // matching with larger container. all requests returned - Resource testCapability3 = Resource.newInstance(4000, 4); - matches = amClient.getMatchingRequests(priority, node, testCapability3); - assert(matches.size() == 4); - - Resource testCapability4 = Resource.newInstance(1024, 2); - matches = amClient.getMatchingRequests(priority, node, testCapability4); - assert(matches.size() == 2); - // verify non-fitting containers are not returned and fitting ones are - for(Collection testSet : matches) { - assertTrue(testSet.size() == 1); - StoredContainerRequest testRequest = testSet.iterator().next(); - assertTrue(testRequest != storedContainer4); - assertTrue(testRequest != storedContainer5); - assert(testRequest == storedContainer2 || - testRequest == storedContainer3); - } - - Resource testCapability5 = Resource.newInstance(512, 4); - matches = amClient.getMatchingRequests(priority, node, testCapability5); - assert(matches.size() == 0); - - amClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED, - null, null); - - } finally { - if (amClient != null && amClient.getServiceState() == STATE.STARTED) { - amClient.stop(); - } - } - } - - private void verifyMatches( - List> matches, - int matchSize) { - assertTrue(matches.size() == 1); - assertTrue(matches.get(0).size() == matchSize); - } - - @Test (timeout=60000) - public void testAMRMClientMatchingFitInferredRack() throws YarnException, IOException { - AMRMClientImpl amClient = null; - try { - // start am rm client - amClient = new AMRMClientImpl(attemptId); - amClient.init(conf); - amClient.start(); - amClient.registerApplicationMaster("Host", 10000, ""); - - Resource capability = Resource.newInstance(1024, 2); - - StoredContainerRequest storedContainer1 = - new StoredContainerRequest(capability, nodes, null, priority); - amClient.addContainerRequest(storedContainer1); - - // verify matching with original node and inferred rack - List> matches; - StoredContainerRequest storedRequest; - // exact match node - matches = amClient.getMatchingRequests(priority, node, capability); - verifyMatches(matches, 1); - storedRequest = matches.get(0).iterator().next(); - assertTrue(storedContainer1 == storedRequest); - // inferred match rack - matches = amClient.getMatchingRequests(priority, rack, capability); - verifyMatches(matches, 1); - storedRequest = matches.get(0).iterator().next(); - assertTrue(storedContainer1 == storedRequest); - - // inferred rack match no longer valid after request is removed - amClient.removeContainerRequest(storedContainer1); - matches = amClient.getMatchingRequests(priority, rack, capability); - assertTrue(matches.isEmpty()); - - amClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED, - null, null); - - } finally { - if (amClient != null && amClient.getServiceState() == STATE.STARTED) { - amClient.stop(); - } - } - } - - @Test (timeout=60000) - public void testAMRMClientMatchStorage() throws YarnException, IOException { - AMRMClientImpl amClient = null; - try { - // start am rm client - amClient = - (AMRMClientImpl) AMRMClient - . createAMRMClient(attemptId); - amClient.init(conf); - amClient.start(); - amClient.registerApplicationMaster("Host", 10000, ""); - - Priority priority1 = Records.newRecord(Priority.class); - priority1.setPriority(2); - - StoredContainerRequest storedContainer1 = - new StoredContainerRequest(capability, nodes, racks, priority); - StoredContainerRequest storedContainer2 = - new StoredContainerRequest(capability, nodes, racks, priority); - StoredContainerRequest storedContainer3 = - new StoredContainerRequest(capability, null, null, priority1); - amClient.addContainerRequest(storedContainer1); - amClient.addContainerRequest(storedContainer2); - amClient.addContainerRequest(storedContainer3); - - // test addition and storage - int containersRequestedAny = amClient.remoteRequestsTable.get(priority) - .get(ResourceRequest.ANY).get(capability).remoteRequest.getNumContainers(); - assertTrue(containersRequestedAny == 2); - containersRequestedAny = amClient.remoteRequestsTable.get(priority1) - .get(ResourceRequest.ANY).get(capability).remoteRequest.getNumContainers(); - assertTrue(containersRequestedAny == 1); - List> matches = - amClient.getMatchingRequests(priority, node, capability); - verifyMatches(matches, 2); - matches = amClient.getMatchingRequests(priority, rack, capability); - verifyMatches(matches, 2); - matches = - amClient.getMatchingRequests(priority, ResourceRequest.ANY, capability); - verifyMatches(matches, 2); - matches = amClient.getMatchingRequests(priority1, rack, capability); - assertTrue(matches.isEmpty()); - matches = - amClient.getMatchingRequests(priority1, ResourceRequest.ANY, capability); - verifyMatches(matches, 1); - - // test removal - amClient.removeContainerRequest(storedContainer3); - matches = amClient.getMatchingRequests(priority, node, capability); - verifyMatches(matches, 2); - amClient.removeContainerRequest(storedContainer2); - matches = amClient.getMatchingRequests(priority, node, capability); - verifyMatches(matches, 1); - matches = amClient.getMatchingRequests(priority, rack, capability); - verifyMatches(matches, 1); - - // test matching of containers - StoredContainerRequest storedRequest = matches.get(0).iterator().next(); - assertTrue(storedContainer1 == storedRequest); - amClient.removeContainerRequest(storedContainer1); - matches = - amClient.getMatchingRequests(priority, ResourceRequest.ANY, capability); - assertTrue(matches.isEmpty()); - matches = - amClient.getMatchingRequests(priority1, ResourceRequest.ANY, capability); - assertTrue(matches.isEmpty()); - // 0 requests left. everything got cleaned up - assertTrue(amClient.remoteRequestsTable.isEmpty()); - - // go through an exemplary allocation, matching and release cycle - amClient.addContainerRequest(storedContainer1); - amClient.addContainerRequest(storedContainer3); - // RM should allocate container within 2 calls to allocate() - int allocatedContainerCount = 0; - int iterationsLeft = 2; - while (allocatedContainerCount < 2 - && iterationsLeft-- > 0) { - AllocateResponse allocResponse = amClient.allocate(0.1f); - assertTrue(amClient.ask.size() == 0); - assertTrue(amClient.release.size() == 0); - - assertTrue(nodeCount == amClient.getClusterNodeCount()); - allocatedContainerCount += allocResponse.getAllocatedContainers().size(); - for(Container container : allocResponse.getAllocatedContainers()) { - ContainerRequest expectedRequest = - container.getPriority().equals(storedContainer1.getPriority()) ? - storedContainer1 : storedContainer3; - matches = amClient.getMatchingRequests(container.getPriority(), - ResourceRequest.ANY, - container.getResource()); - // test correct matched container is returned - verifyMatches(matches, 1); - ContainerRequest matchedRequest = matches.get(0).iterator().next(); - assertTrue(matchedRequest == expectedRequest); - - // assign this container, use it and release it - amClient.releaseAssignedContainer(container.getId()); - } - if(allocatedContainerCount < containersRequestedAny) { - // sleep to let NM's heartbeat to RM and trigger allocations - sleep(1000); - } - } - - assertTrue(allocatedContainerCount == 2); - assertTrue(amClient.release.size() == 2); - assertTrue(amClient.ask.size() == 0); - AllocateResponse allocResponse = amClient.allocate(0.1f); - assertTrue(amClient.release.size() == 0); - assertTrue(amClient.ask.size() == 0); - assertTrue(allocResponse.getAllocatedContainers().size() == 0); - - - amClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED, - null, null); - - } finally { - if (amClient != null && amClient.getServiceState() == STATE.STARTED) { - amClient.stop(); - } - } - } - - @Test (timeout=60000) - public void testAMRMClient() throws YarnException, IOException { - AMRMClient amClient = null; - try { - // start am rm client - amClient = AMRMClient.createAMRMClient(attemptId); - amClient.init(conf); - amClient.start(); - - amClient.registerApplicationMaster("Host", 10000, ""); - - testAllocation((AMRMClientImpl)amClient); - - amClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED, - null, null); - - } finally { - if (amClient != null && amClient.getServiceState() == STATE.STARTED) { - amClient.stop(); - } - } - } - - private void testAllocation(final AMRMClientImpl amClient) - throws YarnException, IOException { - // setup container request - - assertTrue(amClient.ask.size() == 0); - assertTrue(amClient.release.size() == 0); - - amClient.addContainerRequest( - new ContainerRequest(capability, nodes, racks, priority, 1)); - amClient.addContainerRequest( - new ContainerRequest(capability, nodes, racks, priority, 3)); - amClient.removeContainerRequest( - new ContainerRequest(capability, nodes, racks, priority, 2)); - - int containersRequestedNode = amClient.remoteRequestsTable.get(priority) - .get(node).get(capability).remoteRequest.getNumContainers(); - int containersRequestedRack = amClient.remoteRequestsTable.get(priority) - .get(rack).get(capability).remoteRequest.getNumContainers(); - int containersRequestedAny = amClient.remoteRequestsTable.get(priority) - .get(ResourceRequest.ANY).get(capability).remoteRequest.getNumContainers(); - - assertTrue(containersRequestedNode == 2); - assertTrue(containersRequestedRack == 2); - assertTrue(containersRequestedAny == 2); - assertTrue(amClient.ask.size() == 3); - assertTrue(amClient.release.size() == 0); - - // RM should allocate container within 2 calls to allocate() - int allocatedContainerCount = 0; - int iterationsLeft = 2; - Set releases = new TreeSet(); - - ConcurrentHashMap nmTokens = amClient.getNMTokens(); - Assert.assertEquals(0, nmTokens.size()); - HashMap receivedNMTokens = new HashMap(); - - while (allocatedContainerCount < containersRequestedAny - && iterationsLeft-- > 0) { - AllocateResponse allocResponse = amClient.allocate(0.1f); - assertTrue(amClient.ask.size() == 0); - assertTrue(amClient.release.size() == 0); - - assertTrue(nodeCount == amClient.getClusterNodeCount()); - allocatedContainerCount += allocResponse.getAllocatedContainers().size(); - for(Container container : allocResponse.getAllocatedContainers()) { - ContainerId rejectContainerId = container.getId(); - releases.add(rejectContainerId); - amClient.releaseAssignedContainer(rejectContainerId); - } - Assert.assertEquals(nmTokens.size(), amClient.getNMTokens().size()); - Iterator nodeI = nmTokens.keySet().iterator(); - while (nodeI.hasNext()) { - String nodeId = nodeI.next(); - if (!receivedNMTokens.containsKey(nodeId)) { - receivedNMTokens.put(nodeId, nmTokens.get(nodeId)); - } else { - Assert.fail("Received token again for : " + nodeId); - } - } - nodeI = receivedNMTokens.keySet().iterator(); - while (nodeI.hasNext()) { - nmTokens.remove(nodeI.next()); - } - - if(allocatedContainerCount < containersRequestedAny) { - // sleep to let NM's heartbeat to RM and trigger allocations - sleep(1000); - } - } - - Assert.assertEquals(0, amClient.getNMTokens().size()); - // Should receive atleast 1 token - Assert.assertTrue(receivedNMTokens.size() > 0 - && receivedNMTokens.size() <= nodeCount); - - assertTrue(allocatedContainerCount == containersRequestedAny); - assertTrue(amClient.release.size() == 2); - assertTrue(amClient.ask.size() == 0); - - // need to tell the AMRMClient that we dont need these resources anymore - amClient.removeContainerRequest( - new ContainerRequest(capability, nodes, racks, priority, 2)); - assertTrue(amClient.ask.size() == 3); - // send 0 container count request for resources that are no longer needed - ResourceRequest snoopRequest = amClient.ask.iterator().next(); - assertTrue(snoopRequest.getNumContainers() == 0); - - // test RPC exception handling - amClient.addContainerRequest(new ContainerRequest(capability, nodes, - racks, priority, 2)); - snoopRequest = amClient.ask.iterator().next(); - assertTrue(snoopRequest.getNumContainers() == 2); - - ApplicationMasterProtocol realRM = amClient.rmClient; - try { - ApplicationMasterProtocol mockRM = mock(ApplicationMasterProtocol.class); - when(mockRM.allocate(any(AllocateRequest.class))).thenAnswer( - new Answer() { - public AllocateResponse answer(InvocationOnMock invocation) - throws Exception { - amClient.removeContainerRequest( - new ContainerRequest(capability, nodes, - racks, priority, 2)); - throw new Exception(); - } - }); - amClient.rmClient = mockRM; - amClient.allocate(0.1f); - }catch (Exception ioe) {} - finally { - amClient.rmClient = realRM; - } - - assertTrue(amClient.release.size() == 2); - assertTrue(amClient.ask.size() == 3); - snoopRequest = amClient.ask.iterator().next(); - // verify that the remove request made in between makeRequest and allocate - // has not been lost - assertTrue(snoopRequest.getNumContainers() == 0); - - iterationsLeft = 2; - // do a few iterations to ensure RM is not going send new containers - while(!releases.isEmpty() || iterationsLeft-- > 0) { - // inform RM of rejection - AllocateResponse allocResponse = amClient.allocate(0.1f); - // RM did not send new containers because AM does not need any - assertTrue(allocResponse.getAllocatedContainers().size() == 0); - if(allocResponse.getCompletedContainersStatuses().size() > 0) { - for(ContainerStatus cStatus :allocResponse - .getCompletedContainersStatuses()) { - if(releases.contains(cStatus.getContainerId())) { - assertTrue(cStatus.getState() == ContainerState.COMPLETE); - assertTrue(cStatus.getExitStatus() == -100); - releases.remove(cStatus.getContainerId()); - } - } - } - if(iterationsLeft > 0) { - // sleep to make sure NM's heartbeat - sleep(1000); - } - } - assertTrue(amClient.ask.size() == 0); - assertTrue(amClient.release.size() == 0); - } - - private void sleep(int sleepTime) { - try { - Thread.sleep(sleepTime); - } catch (InterruptedException e) { - e.printStackTrace(); - } - } - -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClientAsync.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClientAsync.java deleted file mode 100644 index dd8a1c9..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClientAsync.java +++ /dev/null @@ -1,323 +0,0 @@ -/** - * 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.hadoop.yarn.client; - -import static org.mockito.Matchers.anyFloat; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.anyString; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; - -import junit.framework.Assert; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; -import org.apache.hadoop.yarn.api.records.AMCommand; -import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.ContainerState; -import org.apache.hadoop.yarn.api.records.ContainerStatus; -import org.apache.hadoop.yarn.api.records.NMToken; -import org.apache.hadoop.yarn.api.records.NodeReport; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.client.AMRMClient.ContainerRequest; -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.junit.Test; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; - -public class TestAMRMClientAsync { - - private static final Log LOG = LogFactory.getLog(TestAMRMClientAsync.class); - - @SuppressWarnings("unchecked") - @Test(timeout=10000) - public void testAMRMClientAsync() throws Exception { - Configuration conf = new Configuration(); - final AtomicBoolean heartbeatBlock = new AtomicBoolean(true); - List completed1 = Arrays.asList( - ContainerStatus.newInstance(newContainerId(0, 0, 0, 0), - ContainerState.COMPLETE, "", 0)); - List allocated1 = Arrays.asList( - Container.newInstance(null, null, null, null, null, null)); - final AllocateResponse response1 = createAllocateResponse( - new ArrayList(), allocated1, null); - final AllocateResponse response2 = createAllocateResponse(completed1, - new ArrayList(), null); - final AllocateResponse emptyResponse = createAllocateResponse( - new ArrayList(), new ArrayList(), null); - - TestCallbackHandler callbackHandler = new TestCallbackHandler(); - final AMRMClient client = mock(AMRMClientImpl.class); - final AtomicInteger secondHeartbeatSync = new AtomicInteger(0); - when(client.allocate(anyFloat())).thenReturn(response1).thenAnswer(new Answer() { - @Override - public AllocateResponse answer(InvocationOnMock invocation) - throws Throwable { - secondHeartbeatSync.incrementAndGet(); - while(heartbeatBlock.get()) { - synchronized(heartbeatBlock) { - heartbeatBlock.wait(); - } - } - secondHeartbeatSync.incrementAndGet(); - return response2; - } - }).thenReturn(emptyResponse); - when(client.registerApplicationMaster(anyString(), anyInt(), anyString())) - .thenReturn(null); - when(client.getClusterAvailableResources()).thenAnswer(new Answer() { - @Override - public Resource answer(InvocationOnMock invocation) - throws Throwable { - // take client lock to simulate behavior of real impl - synchronized (client) { - Thread.sleep(10); - } - return null; - } - }); - - AMRMClientAsync asyncClient = - new AMRMClientAsync(client, 20, callbackHandler); - asyncClient.init(conf); - asyncClient.start(); - asyncClient.registerApplicationMaster("localhost", 1234, null); - - // while the CallbackHandler will still only be processing the first response, - // heartbeater thread should still be sending heartbeats. - // To test this, wait for the second heartbeat to be received. - while (secondHeartbeatSync.get() < 1) { - Thread.sleep(10); - } - - // heartbeat will be blocked. make sure we can call client methods at this - // time. Checks that heartbeat is not holding onto client lock - assert(secondHeartbeatSync.get() < 2); - asyncClient.getClusterAvailableResources(); - // method returned. now unblock heartbeat - assert(secondHeartbeatSync.get() < 2); - synchronized (heartbeatBlock) { - heartbeatBlock.set(false); - heartbeatBlock.notifyAll(); - } - - // allocated containers should come before completed containers - Assert.assertEquals(null, callbackHandler.takeCompletedContainers()); - - // wait for the allocated containers from the first heartbeat's response - while (callbackHandler.takeAllocatedContainers() == null) { - Assert.assertEquals(null, callbackHandler.takeCompletedContainers()); - Thread.sleep(10); - } - - // wait for the completed containers from the second heartbeat's response - while (callbackHandler.takeCompletedContainers() == null) { - Thread.sleep(10); - } - - asyncClient.stop(); - - Assert.assertEquals(null, callbackHandler.takeAllocatedContainers()); - Assert.assertEquals(null, callbackHandler.takeCompletedContainers()); - } - - @Test(timeout=10000) - public void testAMRMClientAsyncException() throws Exception { - Configuration conf = new Configuration(); - TestCallbackHandler callbackHandler = new TestCallbackHandler(); - @SuppressWarnings("unchecked") - AMRMClient client = mock(AMRMClientImpl.class); - String exStr = "TestException"; - YarnException mockException = mock(YarnException.class); - when(mockException.getMessage()).thenReturn(exStr); - when(client.allocate(anyFloat())).thenThrow(mockException); - - AMRMClientAsync asyncClient = - new AMRMClientAsync(client, 20, callbackHandler); - asyncClient.init(conf); - asyncClient.start(); - - synchronized (callbackHandler.notifier) { - asyncClient.registerApplicationMaster("localhost", 1234, null); - while(callbackHandler.savedException == null) { - try { - callbackHandler.notifier.wait(); - } catch (InterruptedException e) { - e.printStackTrace(); - } - } - } - - Assert.assertTrue(callbackHandler.savedException.getMessage().contains(exStr)); - - asyncClient.stop(); - // stopping should have joined all threads and completed all callbacks - Assert.assertTrue(callbackHandler.callbackCount == 0); - } - - @Test//(timeout=10000) - public void testAMRMClientAsyncReboot() throws Exception { - Configuration conf = new Configuration(); - TestCallbackHandler callbackHandler = new TestCallbackHandler(); - @SuppressWarnings("unchecked") - AMRMClient client = mock(AMRMClientImpl.class); - - final AllocateResponse rebootResponse = createAllocateResponse( - new ArrayList(), new ArrayList(), null); - rebootResponse.setAMCommand(AMCommand.AM_RESYNC); - when(client.allocate(anyFloat())).thenReturn(rebootResponse); - - AMRMClientAsync asyncClient = - new AMRMClientAsync(client, 20, callbackHandler); - asyncClient.init(conf); - asyncClient.start(); - - synchronized (callbackHandler.notifier) { - asyncClient.registerApplicationMaster("localhost", 1234, null); - while(callbackHandler.reboot == false) { - try { - callbackHandler.notifier.wait(); - } catch (InterruptedException e) { - e.printStackTrace(); - } - } - } - - asyncClient.stop(); - // stopping should have joined all threads and completed all callbacks - Assert.assertTrue(callbackHandler.callbackCount == 0); - } - - private AllocateResponse createAllocateResponse( - List completed, List allocated, - List nmTokens) { - AllocateResponse response = - AllocateResponse.newInstance(0, completed, allocated, - new ArrayList(), null, null, 1, null, nmTokens); - return response; - } - - public static ContainerId newContainerId(int appId, int appAttemptId, - long timestamp, int containerId) { - ApplicationId applicationId = ApplicationId.newInstance(timestamp, appId); - ApplicationAttemptId applicationAttemptId = - ApplicationAttemptId.newInstance(applicationId, appAttemptId); - return ContainerId.newInstance(applicationAttemptId, containerId); - } - - private class TestCallbackHandler implements AMRMClientAsync.CallbackHandler { - private volatile List completedContainers; - private volatile List allocatedContainers; - Exception savedException = null; - boolean reboot = false; - Object notifier = new Object(); - - int callbackCount = 0; - - public List takeCompletedContainers() { - List ret = completedContainers; - if (ret == null) { - return null; - } - completedContainers = null; - synchronized (ret) { - ret.notify(); - } - return ret; - } - - public List takeAllocatedContainers() { - List ret = allocatedContainers; - if (ret == null) { - return null; - } - allocatedContainers = null; - synchronized (ret) { - ret.notify(); - } - return ret; - } - - @Override - public void onContainersCompleted(List statuses) { - completedContainers = statuses; - // wait for containers to be taken before returning - synchronized (completedContainers) { - while (completedContainers != null) { - try { - completedContainers.wait(); - } catch (InterruptedException ex) { - LOG.error("Interrupted during wait", ex); - } - } - } - } - - @Override - public void onContainersAllocated(List containers) { - allocatedContainers = containers; - // wait for containers to be taken before returning - synchronized (allocatedContainers) { - while (allocatedContainers != null) { - try { - allocatedContainers.wait(); - } catch (InterruptedException ex) { - LOG.error("Interrupted during wait", ex); - } - } - } - } - - @Override - public void onShutdownRequest() { - reboot = true; - synchronized (notifier) { - notifier.notifyAll(); - } - } - - @Override - public void onNodesUpdated(List updatedNodes) {} - - @Override - public float getProgress() { - callbackCount++; - return 0.5f; - } - - @Override - public void onError(Exception e) { - savedException = e; - synchronized (notifier) { - notifier.notifyAll(); - } - } - } -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClientContainerRequest.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClientContainerRequest.java deleted file mode 100644 index ab2de5b..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestAMRMClientContainerRequest.java +++ /dev/null @@ -1,80 +0,0 @@ -/** - * 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.hadoop.yarn.client; - -import java.util.Arrays; -import java.util.List; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.CommonConfigurationKeysPublic; -import org.apache.hadoop.net.DNSToSwitchMapping; -import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.Priority; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.api.records.ResourceRequest; -import org.junit.Test; - -import static org.apache.hadoop.yarn.client.AMRMClientImpl.ContainerRequest; -import static org.junit.Assert.assertEquals; - -public class TestAMRMClientContainerRequest { - @Test - public void testFillInRacks() { - AMRMClientImpl client = new AMRMClientImpl( - ApplicationAttemptId.newInstance(ApplicationId.newInstance(0l, 0), 0)); - - Configuration conf = new Configuration(); - conf.setClass( - CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY, - MyResolver.class, DNSToSwitchMapping.class); - client.init(conf); - - Resource capability = Resource.newInstance(1024, 1); - ContainerRequest request = - new ContainerRequest(capability, new String[] {"host1", "host2"}, - new String[] {"/rack2"}, Priority.newInstance(1), 4); - client.addContainerRequest(request); - verifyResourceRequestLocation(client, request, "host1"); - verifyResourceRequestLocation(client, request, "host2"); - verifyResourceRequestLocation(client, request, "/rack1"); - verifyResourceRequestLocation(client, request, "/rack2"); - verifyResourceRequestLocation(client, request, ResourceRequest.ANY); - } - - private static class MyResolver implements DNSToSwitchMapping { - - @Override - public List resolve(List names) { - return Arrays.asList("/rack1"); - } - - @Override - public void reloadCachedMappings() {} - } - - private void verifyResourceRequestLocation( - AMRMClientImpl client, ContainerRequest request, - String location) { - ResourceRequest ask = client.remoteRequestsTable.get(request.priority) - .get(location).get(request.capability).remoteRequest; - assertEquals(location, ask.getResourceName()); - assertEquals(request.getContainerCount(), ask.getNumContainers()); - } -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestNMClient.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestNMClient.java deleted file mode 100644 index de45461..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestNMClient.java +++ /dev/null @@ -1,350 +0,0 @@ -/** - * 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.hadoop.yarn.client; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.List; -import java.util.Set; -import java.util.TreeSet; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.io.DataOutputBuffer; -import org.apache.hadoop.security.Credentials; -import org.apache.hadoop.service.Service.STATE; -import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; -import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; -import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest; -import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.ApplicationReport; -import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; -import org.apache.hadoop.yarn.api.records.ContainerState; -import org.apache.hadoop.yarn.api.records.ContainerStatus; -import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; -import org.apache.hadoop.yarn.api.records.NodeReport; -import org.apache.hadoop.yarn.api.records.Priority; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.api.records.ResourceRequest; -import org.apache.hadoop.yarn.api.records.YarnApplicationState; -import org.apache.hadoop.yarn.client.AMRMClient.ContainerRequest; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.hadoop.yarn.server.MiniYARNCluster; -import org.apache.hadoop.yarn.util.Records; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -public class TestNMClient { - Configuration conf = null; - MiniYARNCluster yarnCluster = null; - YarnClientImpl yarnClient = null; - AMRMClientImpl rmClient = null; - NMClientImpl nmClient = null; - List nodeReports = null; - ApplicationAttemptId attemptId = null; - int nodeCount = 3; - - @Before - public void setup() throws YarnException, IOException { - // start minicluster - conf = new YarnConfiguration(); - yarnCluster = - new MiniYARNCluster(TestAMRMClient.class.getName(), nodeCount, 1, 1); - yarnCluster.init(conf); - yarnCluster.start(); - assertNotNull(yarnCluster); - assertEquals(STATE.STARTED, yarnCluster.getServiceState()); - - // start rm client - yarnClient = (YarnClientImpl) YarnClient.createYarnClient(); - yarnClient.init(conf); - yarnClient.start(); - assertNotNull(yarnClient); - assertEquals(STATE.STARTED, yarnClient.getServiceState()); - - // get node info - nodeReports = yarnClient.getNodeReports(); - - // submit new app - GetNewApplicationResponse newApp = yarnClient.getNewApplication(); - ApplicationId appId = newApp.getApplicationId(); - - ApplicationSubmissionContext appContext = Records - .newRecord(ApplicationSubmissionContext.class); - // set the application id - appContext.setApplicationId(appId); - // set the application name - appContext.setApplicationName("Test"); - // Set the priority for the application master - Priority pri = Priority.newInstance(0); - appContext.setPriority(pri); - // Set the queue to which this application is to be submitted in the RM - appContext.setQueue("default"); - // Set up the container launch context for the application master - ContainerLaunchContext amContainer = Records - .newRecord(ContainerLaunchContext.class); - appContext.setAMContainerSpec(amContainer); - // unmanaged AM - appContext.setUnmanagedAM(true); - // Create the request to send to the applications manager - SubmitApplicationRequest appRequest = Records - .newRecord(SubmitApplicationRequest.class); - appRequest.setApplicationSubmissionContext(appContext); - // Submit the application to the applications manager - yarnClient.submitApplication(appContext); - - // wait for app to start - int iterationsLeft = 30; - while (iterationsLeft > 0) { - ApplicationReport appReport = yarnClient.getApplicationReport(appId); - if (appReport.getYarnApplicationState() == - YarnApplicationState.ACCEPTED) { - attemptId = appReport.getCurrentApplicationAttemptId(); - break; - } - sleep(1000); - --iterationsLeft; - } - if (iterationsLeft == 0) { - fail("Application hasn't bee started"); - } - - // start am rm client - rmClient = - (AMRMClientImpl) AMRMClient - . createAMRMClient(attemptId); - rmClient.init(conf); - rmClient.start(); - assertNotNull(rmClient); - assertEquals(STATE.STARTED, rmClient.getServiceState()); - - // start am nm client - nmClient = (NMClientImpl) NMClient.createNMClient(); - nmClient.init(conf); - nmClient.start(); - assertNotNull(nmClient); - assertEquals(STATE.STARTED, nmClient.getServiceState()); - } - - @After - public void tearDown() { - rmClient.stop(); - yarnClient.stop(); - yarnCluster.stop(); - } - - private void stopNmClient(boolean stopContainers) { - assertNotNull("Null nmClient", nmClient); - // leave one unclosed - assertEquals(1, nmClient.startedContainers.size()); - // default true - assertTrue(nmClient.cleanupRunningContainers.get()); - nmClient.cleanupRunningContainersOnStop(stopContainers); - assertEquals(stopContainers, nmClient.cleanupRunningContainers.get()); - nmClient.stop(); - } - - @Test (timeout = 60000) - public void testNMClientNoCleanupOnStop() - throws YarnException, IOException { - - rmClient.registerApplicationMaster("Host", 10000, ""); - - testContainerManagement(nmClient, allocateContainers(rmClient, 5)); - - rmClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED, - null, null); - // don't stop the running containers - stopNmClient(false); - assertFalse(nmClient.startedContainers. isEmpty()); - //now cleanup - nmClient.cleanupRunningContainers(); - assertEquals(0, nmClient.startedContainers.size()); - } - - @Test (timeout = 60000) - public void testNMClient() - throws YarnException, IOException { - - rmClient.registerApplicationMaster("Host", 10000, ""); - - testContainerManagement(nmClient, allocateContainers(rmClient, 5)); - - rmClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED, - null, null); - // stop the running containers on close - assertFalse(nmClient.startedContainers.isEmpty()); - nmClient.cleanupRunningContainersOnStop(true); - assertTrue(nmClient.cleanupRunningContainers.get()); - nmClient.stop(); - } - - private Set allocateContainers( - AMRMClientImpl rmClient, int num) - throws YarnException, IOException { - // setup container request - Resource capability = Resource.newInstance(1024, 0); - Priority priority = Priority.newInstance(0); - String node = nodeReports.get(0).getNodeId().getHost(); - String rack = nodeReports.get(0).getRackName(); - String[] nodes = new String[] {node}; - String[] racks = new String[] {rack}; - - for (int i = 0; i < num; ++i) { - rmClient.addContainerRequest(new ContainerRequest(capability, nodes, - racks, priority, 1)); - } - - int containersRequestedAny = rmClient.remoteRequestsTable.get(priority) - .get(ResourceRequest.ANY).get(capability).remoteRequest - .getNumContainers(); - - // RM should allocate container within 2 calls to allocate() - int allocatedContainerCount = 0; - int iterationsLeft = 2; - Set containers = new TreeSet(); - while (allocatedContainerCount < containersRequestedAny - && iterationsLeft > 0) { - AllocateResponse allocResponse = rmClient.allocate(0.1f); - - allocatedContainerCount += allocResponse.getAllocatedContainers().size(); - for(Container container : allocResponse.getAllocatedContainers()) { - containers.add(container); - } - if(allocatedContainerCount < containersRequestedAny) { - // sleep to let NM's heartbeat to RM and trigger allocations - sleep(1000); - } - - --iterationsLeft; - } - return containers; - } - - private void testContainerManagement(NMClientImpl nmClient, - Set containers) throws YarnException, IOException { - int size = containers.size(); - int i = 0; - for (Container container : containers) { - // getContainerStatus shouldn't be called before startContainer, - // otherwise, NodeManager cannot find the container - try { - nmClient.getContainerStatus(container.getId(), container.getNodeId(), - container.getContainerToken()); - fail("Exception is expected"); - } catch (YarnException e) { - assertTrue("The thrown exception is not expected", - e.getMessage().contains("is not handled by this NodeManager")); - } - - // stopContainer shouldn't be called before startContainer, - // otherwise, an exception will be thrown - try { - nmClient.stopContainer(container.getId(), container.getNodeId(), - container.getContainerToken()); - fail("Exception is expected"); - } catch (YarnException e) { - if (!e.getMessage() - .contains("is either not started yet or already stopped")) { - throw (AssertionError) - (new AssertionError("Exception is not expected: " + e).initCause( - e)); - } - } - - Credentials ts = new Credentials(); - DataOutputBuffer dob = new DataOutputBuffer(); - ts.writeTokenStorageToStream(dob); - ByteBuffer securityTokens = - ByteBuffer.wrap(dob.getData(), 0, dob.getLength()); - ContainerLaunchContext clc = - Records.newRecord(ContainerLaunchContext.class); - clc.setTokens(securityTokens); - try { - nmClient.startContainer(container, clc); - } catch (YarnException e) { - throw (AssertionError) - (new AssertionError("Exception is not expected: " + e).initCause(e)); - } - - // leave one container unclosed - if (++i < size) { - // NodeManager may still need some time to make the container started - testGetContainerStatus(container, i, ContainerState.RUNNING, "", - -1000); - - try { - nmClient.stopContainer(container.getId(), container.getNodeId(), - container.getContainerToken()); - } catch (YarnException e) { - throw (AssertionError) - (new AssertionError("Exception is not expected: " + e) - .initCause(e)); - } - - // getContainerStatus can be called after stopContainer - testGetContainerStatus(container, i, ContainerState.COMPLETE, - "Container killed by the ApplicationMaster.", 143); - } - } - } - - private void sleep(int sleepTime) { - try { - Thread.sleep(sleepTime); - } catch (InterruptedException e) { - e.printStackTrace(); - } - } - - private void testGetContainerStatus(Container container, int index, - ContainerState state, String diagnostics, int exitStatus) - throws YarnException, IOException { - while (true) { - try { - ContainerStatus status = nmClient.getContainerStatus( - container.getId(), container.getNodeId(), - container.getContainerToken()); - // NodeManager may still need some time to get the stable - // container status - if (status.getState() == state) { - assertEquals(container.getId(), status.getContainerId()); - assertTrue("" + index + ": " + status.getDiagnostics(), - status.getDiagnostics().contains(diagnostics)); - assertEquals(exitStatus, status.getExitStatus()); - break; - } - Thread.sleep(100); - } catch (InterruptedException e) { - e.printStackTrace(); - } - } - } - -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestNMClientAsync.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestNMClientAsync.java deleted file mode 100644 index 4c9c124..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestNMClientAsync.java +++ /dev/null @@ -1,550 +0,0 @@ -/** - * 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.hadoop.yarn.client; - -import static org.mockito.Matchers.any; -import static org.mockito.Mockito.doNothing; -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.Collections; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.BrokenBarrierException; -import java.util.concurrent.CyclicBarrier; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicIntegerArray; - -import junit.framework.Assert; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.service.ServiceOperations; -import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; -import org.apache.hadoop.yarn.api.records.ContainerStatus; -import org.apache.hadoop.yarn.api.records.NodeId; -import org.apache.hadoop.yarn.api.records.Token; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.hadoop.yarn.factories.RecordFactory; -import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; -import org.apache.hadoop.yarn.ipc.RPCUtil; -import org.junit.After; -import org.junit.Test; - - -public class TestNMClientAsync { - - private final RecordFactory recordFactory = - RecordFactoryProvider.getRecordFactory(null); - - private NMClientAsync asyncClient; - private NodeId nodeId; - private Token containerToken; - - @After - public void teardown() { - ServiceOperations.stop(asyncClient); - } - - @Test (timeout = 30000) - public void testNMClientAsync() throws Exception { - Configuration conf = new Configuration(); - conf.setInt(YarnConfiguration.NM_CLIENT_ASYNC_THREAD_POOL_MAX_SIZE, 10); - - // Threads to run are more than the max size of the thread pool - int expectedSuccess = 40; - int expectedFailure = 40; - - asyncClient = new MockNMClientAsync1(expectedSuccess, expectedFailure); - asyncClient.init(conf); - Assert.assertEquals("The max thread pool size is not correctly set", - 10, asyncClient.maxThreadPoolSize); - asyncClient.start(); - - - for (int i = 0; i < expectedSuccess + expectedFailure; ++i) { - if (i == expectedSuccess) { - while (!((TestCallbackHandler1) asyncClient.callbackHandler) - .isAllSuccessCallsExecuted()) { - Thread.sleep(10); - } - asyncClient.client = mockNMClient(1); - } - Container container = mockContainer(i); - ContainerLaunchContext clc = - recordFactory.newRecordInstance(ContainerLaunchContext.class); - asyncClient.startContainer(container, clc); - } - while (!((TestCallbackHandler1) asyncClient.callbackHandler) - .isStartAndQueryFailureCallsExecuted()) { - Thread.sleep(10); - } - asyncClient.client = mockNMClient(2); - ((TestCallbackHandler1) asyncClient.callbackHandler).path = false; - for (int i = 0; i < expectedFailure; ++i) { - Container container = mockContainer( - expectedSuccess + expectedFailure + i); - ContainerLaunchContext clc = - recordFactory.newRecordInstance(ContainerLaunchContext.class); - asyncClient.startContainer(container, clc); - } - while (!((TestCallbackHandler1) asyncClient.callbackHandler) - .isStopFailureCallsExecuted()) { - Thread.sleep(10); - } - for (String errorMsg : - ((TestCallbackHandler1) asyncClient.callbackHandler).errorMsgs) { - System.out.println(errorMsg); - } - Assert.assertEquals("Error occurs in CallbackHandler", 0, - ((TestCallbackHandler1) asyncClient.callbackHandler).errorMsgs.size()); - for (String errorMsg : ((MockNMClientAsync1) asyncClient).errorMsgs) { - System.out.println(errorMsg); - } - Assert.assertEquals("Error occurs in ContainerEventProcessor", 0, - ((MockNMClientAsync1) asyncClient).errorMsgs.size()); - // When the callback functions are all executed, the event processor threads - // may still not terminate and the containers may still not removed. - while (asyncClient.containers.size() > 0) { - Thread.sleep(10); - } - asyncClient.stop(); - Assert.assertFalse( - "The thread of Container Management Event Dispatcher is still alive", - asyncClient.eventDispatcherThread.isAlive()); - Assert.assertTrue("The thread pool is not shut down", - asyncClient.threadPool.isShutdown()); - } - - private class MockNMClientAsync1 extends NMClientAsync { - private Set errorMsgs = - Collections.synchronizedSet(new HashSet()); - - protected MockNMClientAsync1(int expectedSuccess, int expectedFailure) - throws YarnException, IOException { - super(MockNMClientAsync1.class.getName(), mockNMClient(0), - new TestCallbackHandler1(expectedSuccess, expectedFailure)); - } - - private class MockContainerEventProcessor extends ContainerEventProcessor { - public MockContainerEventProcessor(ContainerEvent event) { - super(event); - } - - @Override - public void run() { - try { - super.run(); - } catch (RuntimeException e) { - // If the unexpected throwable comes from error callback functions, it - // will break ContainerEventProcessor.run(). Therefore, monitor - // the exception here - errorMsgs.add("Unexpected throwable from callback functions should" + - " be ignored by Container " + event.getContainerId()); - } - } - } - - @Override - protected ContainerEventProcessor getContainerEventProcessor( - ContainerEvent event) { - return new MockContainerEventProcessor(event); - } - } - - private class TestCallbackHandler1 - implements NMClientAsync.CallbackHandler { - - private boolean path = true; - - private int expectedSuccess; - private int expectedFailure; - - private AtomicInteger actualStartSuccess = new AtomicInteger(0); - private AtomicInteger actualStartFailure = new AtomicInteger(0); - private AtomicInteger actualQuerySuccess = new AtomicInteger(0); - private AtomicInteger actualQueryFailure = new AtomicInteger(0); - private AtomicInteger actualStopSuccess = new AtomicInteger(0); - private AtomicInteger actualStopFailure = new AtomicInteger(0); - - private AtomicIntegerArray actualStartSuccessArray; - private AtomicIntegerArray actualStartFailureArray; - private AtomicIntegerArray actualQuerySuccessArray; - private AtomicIntegerArray actualQueryFailureArray; - private AtomicIntegerArray actualStopSuccessArray; - private AtomicIntegerArray actualStopFailureArray; - - private Set errorMsgs = - Collections.synchronizedSet(new HashSet()); - - public TestCallbackHandler1(int expectedSuccess, int expectedFailure) { - this.expectedSuccess = expectedSuccess; - this.expectedFailure = expectedFailure; - - actualStartSuccessArray = new AtomicIntegerArray(expectedSuccess); - actualStartFailureArray = new AtomicIntegerArray(expectedFailure); - actualQuerySuccessArray = new AtomicIntegerArray(expectedSuccess); - actualQueryFailureArray = new AtomicIntegerArray(expectedFailure); - actualStopSuccessArray = new AtomicIntegerArray(expectedSuccess); - actualStopFailureArray = new AtomicIntegerArray(expectedFailure); - } - - @Override - public void onContainerStarted(ContainerId containerId, - Map allServiceResponse) { - if (path) { - if (containerId.getId() >= expectedSuccess) { - errorMsgs.add("Container " + containerId + - " should throw the exception onContainerStarted"); - return; - } - actualStartSuccess.addAndGet(1); - actualStartSuccessArray.set(containerId.getId(), 1); - - // move on to the following success tests - asyncClient.getContainerStatus(containerId, nodeId, containerToken); - } else { - // move on to the following failure tests - asyncClient.stopContainer(containerId, nodeId, containerToken); - } - - // Shouldn't crash the test thread - throw new RuntimeException("Ignorable Exception"); - } - - @Override - public void onContainerStatusReceived(ContainerId containerId, - ContainerStatus containerStatus) { - if (containerId.getId() >= expectedSuccess) { - errorMsgs.add("Container " + containerId + - " should throw the exception onContainerStatusReceived"); - return; - } - actualQuerySuccess.addAndGet(1); - actualQuerySuccessArray.set(containerId.getId(), 1); - // move on to the following success tests - asyncClient.stopContainer(containerId, nodeId, containerToken); - - // Shouldn't crash the test thread - throw new RuntimeException("Ignorable Exception"); - } - - @Override - public void onContainerStopped(ContainerId containerId) { - if (containerId.getId() >= expectedSuccess) { - errorMsgs.add("Container " + containerId + - " should throw the exception onContainerStopped"); - return; - } - actualStopSuccess.addAndGet(1); - actualStopSuccessArray.set(containerId.getId(), 1); - - // Shouldn't crash the test thread - throw new RuntimeException("Ignorable Exception"); - } - - @Override - public void onStartContainerError(ContainerId containerId, Throwable t) { - // If the unexpected throwable comes from success callback functions, it - // will be handled by the error callback functions. Therefore, monitor - // the exception here - if (t instanceof RuntimeException) { - errorMsgs.add("Unexpected throwable from callback functions should be" + - " ignored by Container " + containerId); - } - if (containerId.getId() < expectedSuccess) { - errorMsgs.add("Container " + containerId + - " shouldn't throw the exception onStartContainerError"); - return; - } - actualStartFailure.addAndGet(1); - actualStartFailureArray.set(containerId.getId() - expectedSuccess, 1); - // move on to the following failure tests - asyncClient.getContainerStatus(containerId, nodeId, containerToken); - - // Shouldn't crash the test thread - throw new RuntimeException("Ignorable Exception"); - } - - @Override - public void onStopContainerError(ContainerId containerId, Throwable t) { - if (t instanceof RuntimeException) { - errorMsgs.add("Unexpected throwable from callback functions should be" + - " ignored by Container " + containerId); - } - if (containerId.getId() < expectedSuccess + expectedFailure) { - errorMsgs.add("Container " + containerId + - " shouldn't throw the exception onStopContainerError"); - return; - } - - actualStopFailure.addAndGet(1); - actualStopFailureArray.set( - containerId.getId() - expectedSuccess - expectedFailure, 1); - - // Shouldn't crash the test thread - throw new RuntimeException("Ignorable Exception"); - } - - @Override - public void onGetContainerStatusError(ContainerId containerId, - Throwable t) { - if (t instanceof RuntimeException) { - errorMsgs.add("Unexpected throwable from callback functions should be" - + " ignored by Container " + containerId); - } - if (containerId.getId() < expectedSuccess) { - errorMsgs.add("Container " + containerId + - " shouldn't throw the exception onGetContainerStatusError"); - return; - } - actualQueryFailure.addAndGet(1); - actualQueryFailureArray.set(containerId.getId() - expectedSuccess, 1); - - // Shouldn't crash the test thread - throw new RuntimeException("Ignorable Exception"); - } - - public boolean isAllSuccessCallsExecuted() { - boolean isAllSuccessCallsExecuted = - actualStartSuccess.get() == expectedSuccess && - actualQuerySuccess.get() == expectedSuccess && - actualStopSuccess.get() == expectedSuccess; - if (isAllSuccessCallsExecuted) { - assertAtomicIntegerArray(actualStartSuccessArray); - assertAtomicIntegerArray(actualQuerySuccessArray); - assertAtomicIntegerArray(actualStopSuccessArray); - } - return isAllSuccessCallsExecuted; - } - - public boolean isStartAndQueryFailureCallsExecuted() { - boolean isStartAndQueryFailureCallsExecuted = - actualStartFailure.get() == expectedFailure && - actualQueryFailure.get() == expectedFailure; - if (isStartAndQueryFailureCallsExecuted) { - assertAtomicIntegerArray(actualStartFailureArray); - assertAtomicIntegerArray(actualQueryFailureArray); - } - return isStartAndQueryFailureCallsExecuted; - } - - public boolean isStopFailureCallsExecuted() { - boolean isStopFailureCallsExecuted = - actualStopFailure.get() == expectedFailure; - if (isStopFailureCallsExecuted) { - assertAtomicIntegerArray(actualStopFailureArray); - } - return isStopFailureCallsExecuted; - } - - private void assertAtomicIntegerArray(AtomicIntegerArray array) { - for (int i = 0; i < array.length(); ++i) { - Assert.assertEquals(1, array.get(i)); - } - } - } - - private NMClient mockNMClient(int mode) - throws YarnException, IOException { - NMClient client = mock(NMClient.class); - switch (mode) { - case 0: - when(client.startContainer(any(Container.class), - any(ContainerLaunchContext.class))).thenReturn( - Collections.emptyMap()); - when(client.getContainerStatus(any(ContainerId.class), any(NodeId.class), - any(Token.class))).thenReturn( - recordFactory.newRecordInstance(ContainerStatus.class)); - doNothing().when(client).stopContainer(any(ContainerId.class), - any(NodeId.class), any(Token.class)); - break; - case 1: - doThrow(RPCUtil.getRemoteException("Start Exception")).when(client) - .startContainer(any(Container.class), - any(ContainerLaunchContext.class)); - doThrow(RPCUtil.getRemoteException("Query Exception")).when(client) - .getContainerStatus(any(ContainerId.class), any(NodeId.class), - any(Token.class)); - doThrow(RPCUtil.getRemoteException("Stop Exception")).when(client) - .stopContainer(any(ContainerId.class), any(NodeId.class), - any(Token.class)); - break; - case 2: - when(client.startContainer(any(Container.class), - any(ContainerLaunchContext.class))).thenReturn( - Collections.emptyMap()); - when(client.getContainerStatus(any(ContainerId.class), any(NodeId.class), - any(Token.class))).thenReturn( - recordFactory.newRecordInstance(ContainerStatus.class)); - doThrow(RPCUtil.getRemoteException("Stop Exception")).when(client) - .stopContainer(any(ContainerId.class), any(NodeId.class), - any(Token.class)); - } - return client; - } - - @Test (timeout = 10000) - public void testOutOfOrder() throws Exception { - CyclicBarrier barrierA = new CyclicBarrier(2); - CyclicBarrier barrierB = new CyclicBarrier(2); - CyclicBarrier barrierC = new CyclicBarrier(2); - asyncClient = new MockNMClientAsync2(barrierA, barrierB, barrierC); - asyncClient.init(new Configuration()); - asyncClient.start(); - - final Container container = mockContainer(1); - final ContainerLaunchContext clc = - recordFactory.newRecordInstance(ContainerLaunchContext.class); - - // start container from another thread - Thread t = new Thread() { - @Override - public void run() { - asyncClient.startContainer(container, clc); - } - }; - t.start(); - - barrierA.await(); - asyncClient.stopContainer(container.getId(), container.getNodeId(), - container.getContainerToken()); - barrierC.await(); - - Assert.assertFalse("Starting and stopping should be out of order", - ((TestCallbackHandler2) asyncClient.callbackHandler) - .exceptionOccurred.get()); - } - - private class MockNMClientAsync2 extends NMClientAsync { - private CyclicBarrier barrierA; - private CyclicBarrier barrierB; - - protected MockNMClientAsync2(CyclicBarrier barrierA, CyclicBarrier barrierB, - CyclicBarrier barrierC) throws YarnException, IOException { - super(MockNMClientAsync2.class.getName(), mockNMClient(0), - new TestCallbackHandler2(barrierC)); - this.barrierA = barrierA; - this.barrierB = barrierB; - } - - private class MockContainerEventProcessor extends ContainerEventProcessor { - - public MockContainerEventProcessor(ContainerEvent event) { - super(event); - } - - @Override - public void run() { - try { - if (event.getType() == ContainerEventType.START_CONTAINER) { - barrierA.await(); - barrierB.await(); - } - super.run(); - if (event.getType() == ContainerEventType.STOP_CONTAINER) { - barrierB.await(); - } - } catch (InterruptedException e) { - e.printStackTrace(); - } catch (BrokenBarrierException e) { - e.printStackTrace(); - } - } - } - - @Override - protected ContainerEventProcessor getContainerEventProcessor( - ContainerEvent event) { - return new MockContainerEventProcessor(event); - } - } - - private class TestCallbackHandler2 - implements NMClientAsync.CallbackHandler { - private CyclicBarrier barrierC; - private AtomicBoolean exceptionOccurred = new AtomicBoolean(false); - - public TestCallbackHandler2(CyclicBarrier barrierC) { - this.barrierC = barrierC; - } - - @Override - public void onContainerStarted(ContainerId containerId, - Map allServiceResponse) { - } - - @Override - public void onContainerStatusReceived(ContainerId containerId, - ContainerStatus containerStatus) { - } - - @Override - public void onContainerStopped(ContainerId containerId) { - } - - @Override - public void onStartContainerError(ContainerId containerId, Throwable t) { - if (!t.getMessage().equals(NMClientAsync.StatefulContainer - .OutOfOrderTransition.STOP_BEFORE_START_ERROR_MSG)) { - exceptionOccurred.set(true); - return; - } - try { - barrierC.await(); - } catch (InterruptedException e) { - e.printStackTrace(); - } catch (BrokenBarrierException e) { - e.printStackTrace(); - } - } - - @Override - public void onGetContainerStatusError(ContainerId containerId, - Throwable t) { - } - - @Override - public void onStopContainerError(ContainerId containerId, Throwable t) { - } - - } - - private Container mockContainer(int i) { - ApplicationId appId = - ApplicationId.newInstance(System.currentTimeMillis(), 1); - ApplicationAttemptId attemptId = - ApplicationAttemptId.newInstance(appId, 1); - ContainerId containerId = ContainerId.newInstance(attemptId, i); - nodeId = NodeId.newInstance("localhost", 0); - // Create an empty record - containerToken = recordFactory.newRecordInstance(Token.class); - return Container.newInstance(containerId, nodeId, null, null, null, - containerToken); - } -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestYarnClient.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestYarnClient.java deleted file mode 100644 index 768371b..0000000 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestYarnClient.java +++ /dev/null @@ -1,175 +0,0 @@ -/** -* 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.hadoop.yarn.client; - -import static org.mockito.Matchers.any; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -import java.io.IOException; -import java.util.HashMap; - -import junit.framework.Assert; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.yarn.api.ApplicationClientProtocol; -import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; -import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse; -import org.apache.hadoop.yarn.api.records.ApplicationAccessType; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.ApplicationReport; -import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; -import org.apache.hadoop.yarn.api.records.YarnApplicationState; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.hadoop.yarn.server.resourcemanager.MockRM; -import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; -import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; -import org.apache.log4j.Level; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; -import org.junit.Test; - -public class TestYarnClient { - - @Test - public void test() { - // More to come later. - } - - @Test - public void testClientStop() { - Configuration conf = new Configuration(); - ResourceManager rm = new ResourceManager(); - rm.init(conf); - rm.start(); - - YarnClient client = YarnClient.createYarnClient(); - client.init(conf); - client.start(); - client.stop(); - } - - @Test (timeout = 30000) - public void testSubmitApplication() { - Configuration conf = new Configuration(); - conf.setLong(YarnConfiguration.YARN_CLIENT_APP_SUBMISSION_POLL_INTERVAL_MS, - 100); // speed up tests - final YarnClient client = new MockYarnClient(); - client.init(conf); - client.start(); - - YarnApplicationState[] exitStates = new YarnApplicationState[] - { - YarnApplicationState.SUBMITTED, - YarnApplicationState.ACCEPTED, - YarnApplicationState.RUNNING, - YarnApplicationState.FINISHED, - YarnApplicationState.FAILED, - YarnApplicationState.KILLED - }; - for (int i = 0; i < exitStates.length; ++i) { - ApplicationSubmissionContext context = - mock(ApplicationSubmissionContext.class); - ApplicationId applicationId = ApplicationId.newInstance( - System.currentTimeMillis(), i); - when(context.getApplicationId()).thenReturn(applicationId); - ((MockYarnClient) client).setYarnApplicationState(exitStates[i]); - try { - client.submitApplication(context); - } catch (YarnException e) { - Assert.fail("Exception is not expected."); - } catch (IOException e) { - Assert.fail("Exception is not expected."); - } - verify(((MockYarnClient) client).mockReport,times(4 * i + 4)) - .getYarnApplicationState(); - } - - client.stop(); - } - - @Test(timeout = 30000) - public void testApplicationType() throws Exception { - Logger rootLogger = LogManager.getRootLogger(); - rootLogger.setLevel(Level.DEBUG); - MockRM rm = new MockRM(); - rm.start(); - RMApp app = rm.submitApp(2000); - RMApp app1 = - rm.submitApp(200, "name", "user", - new HashMap(), false, "default", -1, - null, "MAPREDUCE"); - Assert.assertEquals("YARN", app.getApplicationType()); - Assert.assertEquals("MAPREDUCE", app1.getApplicationType()); - rm.stop(); - } - - @Test(timeout = 30000) - public void testApplicationTypeLimit() throws Exception { - Logger rootLogger = LogManager.getRootLogger(); - rootLogger.setLevel(Level.DEBUG); - MockRM rm = new MockRM(); - rm.start(); - RMApp app1 = - rm.submitApp(200, "name", "user", - new HashMap(), false, "default", -1, - null, "MAPREDUCE-LENGTH-IS-20"); - Assert.assertEquals("MAPREDUCE-LENGTH-IS-", app1.getApplicationType()); - rm.stop(); - } - - private static class MockYarnClient extends YarnClientImpl { - private ApplicationReport mockReport; - - public MockYarnClient() { - super(); - } - - @Override - public void start() { - rmClient = mock(ApplicationClientProtocol.class); - GetApplicationReportResponse mockResponse = - mock(GetApplicationReportResponse.class); - mockReport = mock(ApplicationReport.class); - try{ - when(rmClient.getApplicationReport(any( - GetApplicationReportRequest.class))).thenReturn(mockResponse); - } catch (YarnException e) { - Assert.fail("Exception is not expected."); - } catch (IOException e) { - Assert.fail("Exception is not expected."); - } - when(mockResponse.getApplicationReport()).thenReturn(mockReport); - } - - @Override - public void stop() { - } - - public void setYarnApplicationState(YarnApplicationState state) { - when(mockReport.getYarnApplicationState()).thenReturn( - YarnApplicationState.NEW, YarnApplicationState.NEW_SAVING, - YarnApplicationState.NEW_SAVING, state); - } - } - -} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestAMRMClientAsync.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestAMRMClientAsync.java new file mode 100644 index 0000000..6c1fcb2 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestAMRMClientAsync.java @@ -0,0 +1,326 @@ +/** + * 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.hadoop.yarn.client.api.async.impl; + +import static org.mockito.Matchers.anyFloat; +import static org.mockito.Matchers.anyInt; +import static org.mockito.Matchers.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +import junit.framework.Assert; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; +import org.apache.hadoop.yarn.api.records.AMCommand; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerState; +import org.apache.hadoop.yarn.api.records.ContainerStatus; +import org.apache.hadoop.yarn.api.records.NMToken; +import org.apache.hadoop.yarn.api.records.NodeReport; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.client.api.AMRMClient; +import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest; +import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync; +import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.junit.Test; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +public class TestAMRMClientAsync { + + private static final Log LOG = LogFactory.getLog(TestAMRMClientAsync.class); + + @SuppressWarnings("unchecked") + @Test(timeout=10000) + public void testAMRMClientAsync() throws Exception { + Configuration conf = new Configuration(); + final AtomicBoolean heartbeatBlock = new AtomicBoolean(true); + List completed1 = Arrays.asList( + ContainerStatus.newInstance(newContainerId(0, 0, 0, 0), + ContainerState.COMPLETE, "", 0)); + List allocated1 = Arrays.asList( + Container.newInstance(null, null, null, null, null, null)); + final AllocateResponse response1 = createAllocateResponse( + new ArrayList(), allocated1, null); + final AllocateResponse response2 = createAllocateResponse(completed1, + new ArrayList(), null); + final AllocateResponse emptyResponse = createAllocateResponse( + new ArrayList(), new ArrayList(), null); + + TestCallbackHandler callbackHandler = new TestCallbackHandler(); + final AMRMClient client = mock(AMRMClientImpl.class); + final AtomicInteger secondHeartbeatSync = new AtomicInteger(0); + when(client.allocate(anyFloat())).thenReturn(response1).thenAnswer(new Answer() { + @Override + public AllocateResponse answer(InvocationOnMock invocation) + throws Throwable { + secondHeartbeatSync.incrementAndGet(); + while(heartbeatBlock.get()) { + synchronized(heartbeatBlock) { + heartbeatBlock.wait(); + } + } + secondHeartbeatSync.incrementAndGet(); + return response2; + } + }).thenReturn(emptyResponse); + when(client.registerApplicationMaster(anyString(), anyInt(), anyString())) + .thenReturn(null); + when(client.getClusterAvailableResources()).thenAnswer(new Answer() { + @Override + public Resource answer(InvocationOnMock invocation) + throws Throwable { + // take client lock to simulate behavior of real impl + synchronized (client) { + Thread.sleep(10); + } + return null; + } + }); + + AMRMClientAsync asyncClient = + AMRMClientAsync.createAMRMClientAsync(client, 20, callbackHandler); + asyncClient.init(conf); + asyncClient.start(); + asyncClient.registerApplicationMaster("localhost", 1234, null); + + // while the CallbackHandler will still only be processing the first response, + // heartbeater thread should still be sending heartbeats. + // To test this, wait for the second heartbeat to be received. + while (secondHeartbeatSync.get() < 1) { + Thread.sleep(10); + } + + // heartbeat will be blocked. make sure we can call client methods at this + // time. Checks that heartbeat is not holding onto client lock + assert(secondHeartbeatSync.get() < 2); + asyncClient.getClusterAvailableResources(); + // method returned. now unblock heartbeat + assert(secondHeartbeatSync.get() < 2); + synchronized (heartbeatBlock) { + heartbeatBlock.set(false); + heartbeatBlock.notifyAll(); + } + + // allocated containers should come before completed containers + Assert.assertEquals(null, callbackHandler.takeCompletedContainers()); + + // wait for the allocated containers from the first heartbeat's response + while (callbackHandler.takeAllocatedContainers() == null) { + Assert.assertEquals(null, callbackHandler.takeCompletedContainers()); + Thread.sleep(10); + } + + // wait for the completed containers from the second heartbeat's response + while (callbackHandler.takeCompletedContainers() == null) { + Thread.sleep(10); + } + + asyncClient.stop(); + + Assert.assertEquals(null, callbackHandler.takeAllocatedContainers()); + Assert.assertEquals(null, callbackHandler.takeCompletedContainers()); + } + + @Test(timeout=10000) + public void testAMRMClientAsyncException() throws Exception { + Configuration conf = new Configuration(); + TestCallbackHandler callbackHandler = new TestCallbackHandler(); + @SuppressWarnings("unchecked") + AMRMClient client = mock(AMRMClientImpl.class); + String exStr = "TestException"; + YarnException mockException = mock(YarnException.class); + when(mockException.getMessage()).thenReturn(exStr); + when(client.allocate(anyFloat())).thenThrow(mockException); + + AMRMClientAsync asyncClient = + AMRMClientAsync.createAMRMClientAsync(client, 20, callbackHandler); + asyncClient.init(conf); + asyncClient.start(); + + synchronized (callbackHandler.notifier) { + asyncClient.registerApplicationMaster("localhost", 1234, null); + while(callbackHandler.savedException == null) { + try { + callbackHandler.notifier.wait(); + } catch (InterruptedException e) { + e.printStackTrace(); + } + } + } + + Assert.assertTrue(callbackHandler.savedException.getMessage().contains(exStr)); + + asyncClient.stop(); + // stopping should have joined all threads and completed all callbacks + Assert.assertTrue(callbackHandler.callbackCount == 0); + } + + @Test//(timeout=10000) + public void testAMRMClientAsyncReboot() throws Exception { + Configuration conf = new Configuration(); + TestCallbackHandler callbackHandler = new TestCallbackHandler(); + @SuppressWarnings("unchecked") + AMRMClient client = mock(AMRMClientImpl.class); + + final AllocateResponse rebootResponse = createAllocateResponse( + new ArrayList(), new ArrayList(), null); + rebootResponse.setAMCommand(AMCommand.AM_RESYNC); + when(client.allocate(anyFloat())).thenReturn(rebootResponse); + + AMRMClientAsync asyncClient = + AMRMClientAsync.createAMRMClientAsync(client, 20, callbackHandler); + asyncClient.init(conf); + asyncClient.start(); + + synchronized (callbackHandler.notifier) { + asyncClient.registerApplicationMaster("localhost", 1234, null); + while(callbackHandler.reboot == false) { + try { + callbackHandler.notifier.wait(); + } catch (InterruptedException e) { + e.printStackTrace(); + } + } + } + + asyncClient.stop(); + // stopping should have joined all threads and completed all callbacks + Assert.assertTrue(callbackHandler.callbackCount == 0); + } + + private AllocateResponse createAllocateResponse( + List completed, List allocated, + List nmTokens) { + AllocateResponse response = + AllocateResponse.newInstance(0, completed, allocated, + new ArrayList(), null, null, 1, null, nmTokens); + return response; + } + + public static ContainerId newContainerId(int appId, int appAttemptId, + long timestamp, int containerId) { + ApplicationId applicationId = ApplicationId.newInstance(timestamp, appId); + ApplicationAttemptId applicationAttemptId = + ApplicationAttemptId.newInstance(applicationId, appAttemptId); + return ContainerId.newInstance(applicationAttemptId, containerId); + } + + private class TestCallbackHandler implements AMRMClientAsync.CallbackHandler { + private volatile List completedContainers; + private volatile List allocatedContainers; + Exception savedException = null; + boolean reboot = false; + Object notifier = new Object(); + + int callbackCount = 0; + + public List takeCompletedContainers() { + List ret = completedContainers; + if (ret == null) { + return null; + } + completedContainers = null; + synchronized (ret) { + ret.notify(); + } + return ret; + } + + public List takeAllocatedContainers() { + List ret = allocatedContainers; + if (ret == null) { + return null; + } + allocatedContainers = null; + synchronized (ret) { + ret.notify(); + } + return ret; + } + + @Override + public void onContainersCompleted(List statuses) { + completedContainers = statuses; + // wait for containers to be taken before returning + synchronized (completedContainers) { + while (completedContainers != null) { + try { + completedContainers.wait(); + } catch (InterruptedException ex) { + LOG.error("Interrupted during wait", ex); + } + } + } + } + + @Override + public void onContainersAllocated(List containers) { + allocatedContainers = containers; + // wait for containers to be taken before returning + synchronized (allocatedContainers) { + while (allocatedContainers != null) { + try { + allocatedContainers.wait(); + } catch (InterruptedException ex) { + LOG.error("Interrupted during wait", ex); + } + } + } + } + + @Override + public void onShutdownRequest() { + reboot = true; + synchronized (notifier) { + notifier.notifyAll(); + } + } + + @Override + public void onNodesUpdated(List updatedNodes) {} + + @Override + public float getProgress() { + callbackCount++; + return 0.5f; + } + + @Override + public void onError(Exception e) { + savedException = e; + synchronized (notifier) { + notifier.notifyAll(); + } + } + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestNMClientAsync.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestNMClientAsync.java new file mode 100644 index 0000000..f09e68f --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestNMClientAsync.java @@ -0,0 +1,552 @@ +/** + * 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.hadoop.yarn.client.api.async.impl; + +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.BrokenBarrierException; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicIntegerArray; + +import junit.framework.Assert; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.service.ServiceOperations; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; +import org.apache.hadoop.yarn.api.records.ContainerStatus; +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.api.records.Token; +import org.apache.hadoop.yarn.client.api.NMClient; +import org.apache.hadoop.yarn.client.api.async.NMClientAsync; +import org.apache.hadoop.yarn.client.api.async.impl.NMClientAsyncImpl; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.factories.RecordFactory; +import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; +import org.apache.hadoop.yarn.ipc.RPCUtil; +import org.junit.After; +import org.junit.Test; + + +public class TestNMClientAsync { + + private final RecordFactory recordFactory = + RecordFactoryProvider.getRecordFactory(null); + + private NMClientAsyncImpl asyncClient; + private NodeId nodeId; + private Token containerToken; + + @After + public void teardown() { + ServiceOperations.stop(asyncClient); + } + + @Test (timeout = 60000) + public void testNMClientAsync() throws Exception { + Configuration conf = new Configuration(); + conf.setInt(YarnConfiguration.NM_CLIENT_ASYNC_THREAD_POOL_MAX_SIZE, 10); + + // Threads to run are more than the max size of the thread pool + int expectedSuccess = 40; + int expectedFailure = 40; + + asyncClient = new MockNMClientAsync1(expectedSuccess, expectedFailure); + asyncClient.init(conf); + Assert.assertEquals("The max thread pool size is not correctly set", + 10, asyncClient.maxThreadPoolSize); + asyncClient.start(); + + + for (int i = 0; i < expectedSuccess + expectedFailure; ++i) { + if (i == expectedSuccess) { + while (!((TestCallbackHandler1) asyncClient.callbackHandler) + .isAllSuccessCallsExecuted()) { + Thread.sleep(10); + } + asyncClient.client = mockNMClient(1); + } + Container container = mockContainer(i); + ContainerLaunchContext clc = + recordFactory.newRecordInstance(ContainerLaunchContext.class); + asyncClient.startContainerAsync(container, clc); + } + while (!((TestCallbackHandler1) asyncClient.callbackHandler) + .isStartAndQueryFailureCallsExecuted()) { + Thread.sleep(10); + } + asyncClient.client = mockNMClient(2); + ((TestCallbackHandler1) asyncClient.callbackHandler).path = false; + for (int i = 0; i < expectedFailure; ++i) { + Container container = mockContainer( + expectedSuccess + expectedFailure + i); + ContainerLaunchContext clc = + recordFactory.newRecordInstance(ContainerLaunchContext.class); + asyncClient.startContainerAsync(container, clc); + } + while (!((TestCallbackHandler1) asyncClient.callbackHandler) + .isStopFailureCallsExecuted()) { + Thread.sleep(10); + } + for (String errorMsg : + ((TestCallbackHandler1) asyncClient.callbackHandler).errorMsgs) { + System.out.println(errorMsg); + } + Assert.assertEquals("Error occurs in CallbackHandler", 0, + ((TestCallbackHandler1) asyncClient.callbackHandler).errorMsgs.size()); + for (String errorMsg : ((MockNMClientAsync1) asyncClient).errorMsgs) { + System.out.println(errorMsg); + } + Assert.assertEquals("Error occurs in ContainerEventProcessor", 0, + ((MockNMClientAsync1) asyncClient).errorMsgs.size()); + // When the callback functions are all executed, the event processor threads + // may still not terminate and the containers may still not removed. + while (asyncClient.containers.size() > 0) { + Thread.sleep(10); + } + asyncClient.stop(); + Assert.assertFalse( + "The thread of Container Management Event Dispatcher is still alive", + asyncClient.eventDispatcherThread.isAlive()); + Assert.assertTrue("The thread pool is not shut down", + asyncClient.threadPool.isShutdown()); + } + + private class MockNMClientAsync1 extends NMClientAsyncImpl { + private Set errorMsgs = + Collections.synchronizedSet(new HashSet()); + + protected MockNMClientAsync1(int expectedSuccess, int expectedFailure) + throws YarnException, IOException { + super(MockNMClientAsync1.class.getName(), mockNMClient(0), + new TestCallbackHandler1(expectedSuccess, expectedFailure)); + } + + private class MockContainerEventProcessor extends ContainerEventProcessor { + public MockContainerEventProcessor(ContainerEvent event) { + super(event); + } + + @Override + public void run() { + try { + super.run(); + } catch (RuntimeException e) { + // If the unexpected throwable comes from error callback functions, it + // will break ContainerEventProcessor.run(). Therefore, monitor + // the exception here + errorMsgs.add("Unexpected throwable from callback functions should" + + " be ignored by Container " + event.getContainerId()); + } + } + } + + @Override + protected ContainerEventProcessor getContainerEventProcessor( + ContainerEvent event) { + return new MockContainerEventProcessor(event); + } + } + + private class TestCallbackHandler1 + implements NMClientAsync.CallbackHandler { + + private boolean path = true; + + private int expectedSuccess; + private int expectedFailure; + + private AtomicInteger actualStartSuccess = new AtomicInteger(0); + private AtomicInteger actualStartFailure = new AtomicInteger(0); + private AtomicInteger actualQuerySuccess = new AtomicInteger(0); + private AtomicInteger actualQueryFailure = new AtomicInteger(0); + private AtomicInteger actualStopSuccess = new AtomicInteger(0); + private AtomicInteger actualStopFailure = new AtomicInteger(0); + + private AtomicIntegerArray actualStartSuccessArray; + private AtomicIntegerArray actualStartFailureArray; + private AtomicIntegerArray actualQuerySuccessArray; + private AtomicIntegerArray actualQueryFailureArray; + private AtomicIntegerArray actualStopSuccessArray; + private AtomicIntegerArray actualStopFailureArray; + + private Set errorMsgs = + Collections.synchronizedSet(new HashSet()); + + public TestCallbackHandler1(int expectedSuccess, int expectedFailure) { + this.expectedSuccess = expectedSuccess; + this.expectedFailure = expectedFailure; + + actualStartSuccessArray = new AtomicIntegerArray(expectedSuccess); + actualStartFailureArray = new AtomicIntegerArray(expectedFailure); + actualQuerySuccessArray = new AtomicIntegerArray(expectedSuccess); + actualQueryFailureArray = new AtomicIntegerArray(expectedFailure); + actualStopSuccessArray = new AtomicIntegerArray(expectedSuccess); + actualStopFailureArray = new AtomicIntegerArray(expectedFailure); + } + + @Override + public void onContainerStarted(ContainerId containerId, + Map allServiceResponse) { + if (path) { + if (containerId.getId() >= expectedSuccess) { + errorMsgs.add("Container " + containerId + + " should throw the exception onContainerStarted"); + return; + } + actualStartSuccess.addAndGet(1); + actualStartSuccessArray.set(containerId.getId(), 1); + + // move on to the following success tests + asyncClient.getContainerStatusAsync(containerId, nodeId, containerToken); + } else { + // move on to the following failure tests + asyncClient.stopContainerAsync(containerId, nodeId, containerToken); + } + + // Shouldn't crash the test thread + throw new RuntimeException("Ignorable Exception"); + } + + @Override + public void onContainerStatusReceived(ContainerId containerId, + ContainerStatus containerStatus) { + if (containerId.getId() >= expectedSuccess) { + errorMsgs.add("Container " + containerId + + " should throw the exception onContainerStatusReceived"); + return; + } + actualQuerySuccess.addAndGet(1); + actualQuerySuccessArray.set(containerId.getId(), 1); + // move on to the following success tests + asyncClient.stopContainerAsync(containerId, nodeId, containerToken); + + // Shouldn't crash the test thread + throw new RuntimeException("Ignorable Exception"); + } + + @Override + public void onContainerStopped(ContainerId containerId) { + if (containerId.getId() >= expectedSuccess) { + errorMsgs.add("Container " + containerId + + " should throw the exception onContainerStopped"); + return; + } + actualStopSuccess.addAndGet(1); + actualStopSuccessArray.set(containerId.getId(), 1); + + // Shouldn't crash the test thread + throw new RuntimeException("Ignorable Exception"); + } + + @Override + public void onStartContainerError(ContainerId containerId, Throwable t) { + // If the unexpected throwable comes from success callback functions, it + // will be handled by the error callback functions. Therefore, monitor + // the exception here + if (t instanceof RuntimeException) { + errorMsgs.add("Unexpected throwable from callback functions should be" + + " ignored by Container " + containerId); + } + if (containerId.getId() < expectedSuccess) { + errorMsgs.add("Container " + containerId + + " shouldn't throw the exception onStartContainerError"); + return; + } + actualStartFailure.addAndGet(1); + actualStartFailureArray.set(containerId.getId() - expectedSuccess, 1); + // move on to the following failure tests + asyncClient.getContainerStatusAsync(containerId, nodeId, containerToken); + + // Shouldn't crash the test thread + throw new RuntimeException("Ignorable Exception"); + } + + @Override + public void onStopContainerError(ContainerId containerId, Throwable t) { + if (t instanceof RuntimeException) { + errorMsgs.add("Unexpected throwable from callback functions should be" + + " ignored by Container " + containerId); + } + if (containerId.getId() < expectedSuccess + expectedFailure) { + errorMsgs.add("Container " + containerId + + " shouldn't throw the exception onStopContainerError"); + return; + } + + actualStopFailure.addAndGet(1); + actualStopFailureArray.set( + containerId.getId() - expectedSuccess - expectedFailure, 1); + + // Shouldn't crash the test thread + throw new RuntimeException("Ignorable Exception"); + } + + @Override + public void onGetContainerStatusError(ContainerId containerId, + Throwable t) { + if (t instanceof RuntimeException) { + errorMsgs.add("Unexpected throwable from callback functions should be" + + " ignored by Container " + containerId); + } + if (containerId.getId() < expectedSuccess) { + errorMsgs.add("Container " + containerId + + " shouldn't throw the exception onGetContainerStatusError"); + return; + } + actualQueryFailure.addAndGet(1); + actualQueryFailureArray.set(containerId.getId() - expectedSuccess, 1); + + // Shouldn't crash the test thread + throw new RuntimeException("Ignorable Exception"); + } + + public boolean isAllSuccessCallsExecuted() { + boolean isAllSuccessCallsExecuted = + actualStartSuccess.get() == expectedSuccess && + actualQuerySuccess.get() == expectedSuccess && + actualStopSuccess.get() == expectedSuccess; + if (isAllSuccessCallsExecuted) { + assertAtomicIntegerArray(actualStartSuccessArray); + assertAtomicIntegerArray(actualQuerySuccessArray); + assertAtomicIntegerArray(actualStopSuccessArray); + } + return isAllSuccessCallsExecuted; + } + + public boolean isStartAndQueryFailureCallsExecuted() { + boolean isStartAndQueryFailureCallsExecuted = + actualStartFailure.get() == expectedFailure && + actualQueryFailure.get() == expectedFailure; + if (isStartAndQueryFailureCallsExecuted) { + assertAtomicIntegerArray(actualStartFailureArray); + assertAtomicIntegerArray(actualQueryFailureArray); + } + return isStartAndQueryFailureCallsExecuted; + } + + public boolean isStopFailureCallsExecuted() { + boolean isStopFailureCallsExecuted = + actualStopFailure.get() == expectedFailure; + if (isStopFailureCallsExecuted) { + assertAtomicIntegerArray(actualStopFailureArray); + } + return isStopFailureCallsExecuted; + } + + private void assertAtomicIntegerArray(AtomicIntegerArray array) { + for (int i = 0; i < array.length(); ++i) { + Assert.assertEquals(1, array.get(i)); + } + } + } + + private NMClient mockNMClient(int mode) + throws YarnException, IOException { + NMClient client = mock(NMClient.class); + switch (mode) { + case 0: + when(client.startContainer(any(Container.class), + any(ContainerLaunchContext.class))).thenReturn( + Collections.emptyMap()); + when(client.getContainerStatus(any(ContainerId.class), any(NodeId.class), + any(Token.class))).thenReturn( + recordFactory.newRecordInstance(ContainerStatus.class)); + doNothing().when(client).stopContainer(any(ContainerId.class), + any(NodeId.class), any(Token.class)); + break; + case 1: + doThrow(RPCUtil.getRemoteException("Start Exception")).when(client) + .startContainer(any(Container.class), + any(ContainerLaunchContext.class)); + doThrow(RPCUtil.getRemoteException("Query Exception")).when(client) + .getContainerStatus(any(ContainerId.class), any(NodeId.class), + any(Token.class)); + doThrow(RPCUtil.getRemoteException("Stop Exception")).when(client) + .stopContainer(any(ContainerId.class), any(NodeId.class), + any(Token.class)); + break; + case 2: + when(client.startContainer(any(Container.class), + any(ContainerLaunchContext.class))).thenReturn( + Collections.emptyMap()); + when(client.getContainerStatus(any(ContainerId.class), any(NodeId.class), + any(Token.class))).thenReturn( + recordFactory.newRecordInstance(ContainerStatus.class)); + doThrow(RPCUtil.getRemoteException("Stop Exception")).when(client) + .stopContainer(any(ContainerId.class), any(NodeId.class), + any(Token.class)); + } + return client; + } + + @Test (timeout = 10000) + public void testOutOfOrder() throws Exception { + CyclicBarrier barrierA = new CyclicBarrier(2); + CyclicBarrier barrierB = new CyclicBarrier(2); + CyclicBarrier barrierC = new CyclicBarrier(2); + asyncClient = new MockNMClientAsync2(barrierA, barrierB, barrierC); + asyncClient.init(new Configuration()); + asyncClient.start(); + + final Container container = mockContainer(1); + final ContainerLaunchContext clc = + recordFactory.newRecordInstance(ContainerLaunchContext.class); + + // start container from another thread + Thread t = new Thread() { + @Override + public void run() { + asyncClient.startContainerAsync(container, clc); + } + }; + t.start(); + + barrierA.await(); + asyncClient.stopContainerAsync(container.getId(), container.getNodeId(), + container.getContainerToken()); + barrierC.await(); + + Assert.assertFalse("Starting and stopping should be out of order", + ((TestCallbackHandler2) asyncClient.callbackHandler).exceptionOccurred.get()); + } + + private class MockNMClientAsync2 extends NMClientAsyncImpl { + private CyclicBarrier barrierA; + private CyclicBarrier barrierB; + + protected MockNMClientAsync2(CyclicBarrier barrierA, CyclicBarrier barrierB, + CyclicBarrier barrierC) throws YarnException, IOException { + super(MockNMClientAsync2.class.getName(), mockNMClient(0), + new TestCallbackHandler2(barrierC)); + this.barrierA = barrierA; + this.barrierB = barrierB; + } + + private class MockContainerEventProcessor extends ContainerEventProcessor { + + public MockContainerEventProcessor(ContainerEvent event) { + super(event); + } + + @Override + public void run() { + try { + if (event.getType() == ContainerEventType.START_CONTAINER) { + barrierA.await(); + barrierB.await(); + } + super.run(); + if (event.getType() == ContainerEventType.STOP_CONTAINER) { + barrierB.await(); + } + } catch (InterruptedException e) { + e.printStackTrace(); + } catch (BrokenBarrierException e) { + e.printStackTrace(); + } + } + } + + @Override + protected ContainerEventProcessor getContainerEventProcessor( + ContainerEvent event) { + return new MockContainerEventProcessor(event); + } + } + + private class TestCallbackHandler2 + implements NMClientAsync.CallbackHandler { + private CyclicBarrier barrierC; + private AtomicBoolean exceptionOccurred = new AtomicBoolean(false); + + public TestCallbackHandler2(CyclicBarrier barrierC) { + this.barrierC = barrierC; + } + + @Override + public void onContainerStarted(ContainerId containerId, + Map allServiceResponse) { + } + + @Override + public void onContainerStatusReceived(ContainerId containerId, + ContainerStatus containerStatus) { + } + + @Override + public void onContainerStopped(ContainerId containerId) { + } + + @Override + public void onStartContainerError(ContainerId containerId, Throwable t) { + if (!t.getMessage().equals(NMClientAsyncImpl.StatefulContainer + .OutOfOrderTransition.STOP_BEFORE_START_ERROR_MSG)) { + exceptionOccurred.set(true); + return; + } + try { + barrierC.await(); + } catch (InterruptedException e) { + e.printStackTrace(); + } catch (BrokenBarrierException e) { + e.printStackTrace(); + } + } + + @Override + public void onGetContainerStatusError(ContainerId containerId, + Throwable t) { + } + + @Override + public void onStopContainerError(ContainerId containerId, Throwable t) { + } + + } + + private Container mockContainer(int i) { + ApplicationId appId = + ApplicationId.newInstance(System.currentTimeMillis(), 1); + ApplicationAttemptId attemptId = + ApplicationAttemptId.newInstance(appId, 1); + ContainerId containerId = ContainerId.newInstance(attemptId, i); + nodeId = NodeId.newInstance("localhost", 0); + // Create an empty record + containerToken = recordFactory.newRecordInstance(Token.class); + return Container.newInstance(containerId, nodeId, null, null, null, + containerToken); + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java new file mode 100644 index 0000000..9398f98 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java @@ -0,0 +1,619 @@ +/** + * 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.hadoop.yarn.client.api.impl; + +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import java.util.TreeSet; +import java.util.concurrent.ConcurrentHashMap; + +import junit.framework.Assert; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.service.Service.STATE; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; +import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest; +import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; +import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; +import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; +import org.apache.hadoop.yarn.api.records.ContainerState; +import org.apache.hadoop.yarn.api.records.ContainerStatus; +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.hadoop.yarn.api.records.NodeReport; +import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.ResourceRequest; +import org.apache.hadoop.yarn.api.records.Token; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; +import org.apache.hadoop.yarn.client.api.AMRMClient; +import org.apache.hadoop.yarn.client.api.YarnClient; +import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest; +import org.apache.hadoop.yarn.client.api.AMRMClient.StoredContainerRequest; +import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.server.MiniYARNCluster; +import org.apache.hadoop.yarn.util.Records; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +public class TestAMRMClient { + static Configuration conf = null; + static MiniYARNCluster yarnCluster = null; + static YarnClient yarnClient = null; + static List nodeReports = null; + static ApplicationAttemptId attemptId = null; + static int nodeCount = 3; + + static Resource capability; + static Priority priority; + static String node; + static String rack; + static String[] nodes; + static String[] racks; + + @BeforeClass + public static void setup() throws Exception { + // start minicluster + conf = new YarnConfiguration(); + yarnCluster = new MiniYARNCluster(TestAMRMClient.class.getName(), nodeCount, 1, 1); + yarnCluster.init(conf); + yarnCluster.start(); + + // start rm client + yarnClient = YarnClient.createYarnClient(); + yarnClient.init(conf); + yarnClient.start(); + + // get node info + nodeReports = yarnClient.getNodeReports(); + + priority = Priority.newInstance(1); + capability = Resource.newInstance(1024, 1); + + node = nodeReports.get(0).getNodeId().getHost(); + rack = nodeReports.get(0).getRackName(); + nodes = new String[]{ node }; + racks = new String[]{ rack }; + } + + @Before + public void startApp() throws Exception { + // submit new app + GetNewApplicationResponse newApp = yarnClient.getNewApplication(); + ApplicationId appId = newApp.getApplicationId(); + + ApplicationSubmissionContext appContext = Records + .newRecord(ApplicationSubmissionContext.class); + // set the application id + appContext.setApplicationId(appId); + // set the application name + appContext.setApplicationName("Test"); + // Set the priority for the application master + Priority pri = Records.newRecord(Priority.class); + pri.setPriority(0); + appContext.setPriority(pri); + // Set the queue to which this application is to be submitted in the RM + appContext.setQueue("default"); + // Set up the container launch context for the application master + ContainerLaunchContext amContainer = Records + .newRecord(ContainerLaunchContext.class); + appContext.setAMContainerSpec(amContainer); + // unmanaged AM + appContext.setUnmanagedAM(true); + // Create the request to send to the applications manager + SubmitApplicationRequest appRequest = Records + .newRecord(SubmitApplicationRequest.class); + appRequest.setApplicationSubmissionContext(appContext); + // Submit the application to the applications manager + yarnClient.submitApplication(appContext); + + // wait for app to start + while (true) { + ApplicationReport appReport = yarnClient.getApplicationReport(appId); + if (appReport.getYarnApplicationState() == YarnApplicationState.ACCEPTED) { + attemptId = appReport.getCurrentApplicationAttemptId(); + break; + } + } + } + + @After + public void cancelApp() { + attemptId = null; + } + + @AfterClass + public static void tearDown() { + if (yarnClient != null && yarnClient.getServiceState() == STATE.STARTED) { + yarnClient.stop(); + } + if (yarnCluster != null && yarnCluster.getServiceState() == STATE.STARTED) { + yarnCluster.stop(); + } + } + + @Test (timeout=60000) + public void testAMRMClientMatchingFit() throws YarnException, IOException { + AMRMClient amClient = null; + try { + // start am rm client + amClient = AMRMClient.createAMRMClient(attemptId); + amClient.init(conf); + amClient.start(); + amClient.registerApplicationMaster("Host", 10000, ""); + + Resource capability1 = Resource.newInstance(1024, 2); + Resource capability2 = Resource.newInstance(1024, 1); + Resource capability3 = Resource.newInstance(1000, 2); + Resource capability4 = Resource.newInstance(2000, 1); + Resource capability5 = Resource.newInstance(1000, 3); + Resource capability6 = Resource.newInstance(2000, 1); + + StoredContainerRequest storedContainer1 = + new StoredContainerRequest(capability1, nodes, racks, priority); + StoredContainerRequest storedContainer2 = + new StoredContainerRequest(capability2, nodes, racks, priority); + StoredContainerRequest storedContainer3 = + new StoredContainerRequest(capability3, nodes, racks, priority); + StoredContainerRequest storedContainer4 = + new StoredContainerRequest(capability4, nodes, racks, priority); + StoredContainerRequest storedContainer5 = + new StoredContainerRequest(capability5, nodes, racks, priority); + StoredContainerRequest storedContainer6 = + new StoredContainerRequest(capability6, nodes, racks, priority); + amClient.addContainerRequest(storedContainer1); + amClient.addContainerRequest(storedContainer2); + amClient.addContainerRequest(storedContainer3); + amClient.addContainerRequest(storedContainer4); + amClient.addContainerRequest(storedContainer5); + amClient.addContainerRequest(storedContainer6); + + // test matching of containers + List> matches; + StoredContainerRequest storedRequest; + // exact match + Resource testCapability1 = Resource.newInstance(1024, 2); + matches = amClient.getMatchingRequests(priority, node, testCapability1); + verifyMatches(matches, 1); + storedRequest = matches.get(0).iterator().next(); + assertTrue(storedContainer1 == storedRequest); + amClient.removeContainerRequest(storedContainer1); + + // exact matching with order maintained + Resource testCapability2 = Resource.newInstance(2000, 1); + matches = amClient.getMatchingRequests(priority, node, testCapability2); + verifyMatches(matches, 2); + // must be returned in the order they were made + int i = 0; + for(StoredContainerRequest storedRequest1 : matches.get(0)) { + if(i++ == 0) { + assertTrue(storedContainer4 == storedRequest1); + } else { + assertTrue(storedContainer6 == storedRequest1); + } + } + amClient.removeContainerRequest(storedContainer6); + + // matching with larger container. all requests returned + Resource testCapability3 = Resource.newInstance(4000, 4); + matches = amClient.getMatchingRequests(priority, node, testCapability3); + assert(matches.size() == 4); + + Resource testCapability4 = Resource.newInstance(1024, 2); + matches = amClient.getMatchingRequests(priority, node, testCapability4); + assert(matches.size() == 2); + // verify non-fitting containers are not returned and fitting ones are + for(Collection testSet : matches) { + assertTrue(testSet.size() == 1); + StoredContainerRequest testRequest = testSet.iterator().next(); + assertTrue(testRequest != storedContainer4); + assertTrue(testRequest != storedContainer5); + assert(testRequest == storedContainer2 || + testRequest == storedContainer3); + } + + Resource testCapability5 = Resource.newInstance(512, 4); + matches = amClient.getMatchingRequests(priority, node, testCapability5); + assert(matches.size() == 0); + + amClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED, + null, null); + + } finally { + if (amClient != null && amClient.getServiceState() == STATE.STARTED) { + amClient.stop(); + } + } + } + + private void verifyMatches( + List> matches, + int matchSize) { + assertTrue(matches.size() == 1); + assertTrue(matches.get(0).size() == matchSize); + } + + @Test (timeout=60000) + public void testAMRMClientMatchingFitInferredRack() throws YarnException, IOException { + AMRMClientImpl amClient = null; + try { + // start am rm client + amClient = new AMRMClientImpl(attemptId); + amClient.init(conf); + amClient.start(); + amClient.registerApplicationMaster("Host", 10000, ""); + + Resource capability = Resource.newInstance(1024, 2); + + StoredContainerRequest storedContainer1 = + new StoredContainerRequest(capability, nodes, null, priority); + amClient.addContainerRequest(storedContainer1); + + // verify matching with original node and inferred rack + List> matches; + StoredContainerRequest storedRequest; + // exact match node + matches = amClient.getMatchingRequests(priority, node, capability); + verifyMatches(matches, 1); + storedRequest = matches.get(0).iterator().next(); + assertTrue(storedContainer1 == storedRequest); + // inferred match rack + matches = amClient.getMatchingRequests(priority, rack, capability); + verifyMatches(matches, 1); + storedRequest = matches.get(0).iterator().next(); + assertTrue(storedContainer1 == storedRequest); + + // inferred rack match no longer valid after request is removed + amClient.removeContainerRequest(storedContainer1); + matches = amClient.getMatchingRequests(priority, rack, capability); + assertTrue(matches.isEmpty()); + + amClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED, + null, null); + + } finally { + if (amClient != null && amClient.getServiceState() == STATE.STARTED) { + amClient.stop(); + } + } + } + + @Test (timeout=60000) + public void testAMRMClientMatchStorage() throws YarnException, IOException { + AMRMClientImpl amClient = null; + try { + // start am rm client + amClient = + (AMRMClientImpl) AMRMClient + . createAMRMClient(attemptId); + amClient.init(conf); + amClient.start(); + amClient.registerApplicationMaster("Host", 10000, ""); + + Priority priority1 = Records.newRecord(Priority.class); + priority1.setPriority(2); + + StoredContainerRequest storedContainer1 = + new StoredContainerRequest(capability, nodes, racks, priority); + StoredContainerRequest storedContainer2 = + new StoredContainerRequest(capability, nodes, racks, priority); + StoredContainerRequest storedContainer3 = + new StoredContainerRequest(capability, null, null, priority1); + amClient.addContainerRequest(storedContainer1); + amClient.addContainerRequest(storedContainer2); + amClient.addContainerRequest(storedContainer3); + + // test addition and storage + int containersRequestedAny = amClient.remoteRequestsTable.get(priority) + .get(ResourceRequest.ANY).get(capability).remoteRequest.getNumContainers(); + assertTrue(containersRequestedAny == 2); + containersRequestedAny = amClient.remoteRequestsTable.get(priority1) + .get(ResourceRequest.ANY).get(capability).remoteRequest.getNumContainers(); + assertTrue(containersRequestedAny == 1); + List> matches = + amClient.getMatchingRequests(priority, node, capability); + verifyMatches(matches, 2); + matches = amClient.getMatchingRequests(priority, rack, capability); + verifyMatches(matches, 2); + matches = + amClient.getMatchingRequests(priority, ResourceRequest.ANY, capability); + verifyMatches(matches, 2); + matches = amClient.getMatchingRequests(priority1, rack, capability); + assertTrue(matches.isEmpty()); + matches = + amClient.getMatchingRequests(priority1, ResourceRequest.ANY, capability); + verifyMatches(matches, 1); + + // test removal + amClient.removeContainerRequest(storedContainer3); + matches = amClient.getMatchingRequests(priority, node, capability); + verifyMatches(matches, 2); + amClient.removeContainerRequest(storedContainer2); + matches = amClient.getMatchingRequests(priority, node, capability); + verifyMatches(matches, 1); + matches = amClient.getMatchingRequests(priority, rack, capability); + verifyMatches(matches, 1); + + // test matching of containers + StoredContainerRequest storedRequest = matches.get(0).iterator().next(); + assertTrue(storedContainer1 == storedRequest); + amClient.removeContainerRequest(storedContainer1); + matches = + amClient.getMatchingRequests(priority, ResourceRequest.ANY, capability); + assertTrue(matches.isEmpty()); + matches = + amClient.getMatchingRequests(priority1, ResourceRequest.ANY, capability); + assertTrue(matches.isEmpty()); + // 0 requests left. everything got cleaned up + assertTrue(amClient.remoteRequestsTable.isEmpty()); + + // go through an exemplary allocation, matching and release cycle + amClient.addContainerRequest(storedContainer1); + amClient.addContainerRequest(storedContainer3); + // RM should allocate container within 2 calls to allocate() + int allocatedContainerCount = 0; + int iterationsLeft = 2; + while (allocatedContainerCount < 2 + && iterationsLeft-- > 0) { + AllocateResponse allocResponse = amClient.allocate(0.1f); + assertTrue(amClient.ask.size() == 0); + assertTrue(amClient.release.size() == 0); + + assertTrue(nodeCount == amClient.getClusterNodeCount()); + allocatedContainerCount += allocResponse.getAllocatedContainers().size(); + for(Container container : allocResponse.getAllocatedContainers()) { + ContainerRequest expectedRequest = + container.getPriority().equals(storedContainer1.getPriority()) ? + storedContainer1 : storedContainer3; + matches = amClient.getMatchingRequests(container.getPriority(), + ResourceRequest.ANY, + container.getResource()); + // test correct matched container is returned + verifyMatches(matches, 1); + ContainerRequest matchedRequest = matches.get(0).iterator().next(); + assertTrue(matchedRequest == expectedRequest); + + // assign this container, use it and release it + amClient.releaseAssignedContainer(container.getId()); + } + if(allocatedContainerCount < containersRequestedAny) { + // sleep to let NM's heartbeat to RM and trigger allocations + sleep(1000); + } + } + + assertTrue(allocatedContainerCount == 2); + assertTrue(amClient.release.size() == 2); + assertTrue(amClient.ask.size() == 0); + AllocateResponse allocResponse = amClient.allocate(0.1f); + assertTrue(amClient.release.size() == 0); + assertTrue(amClient.ask.size() == 0); + assertTrue(allocResponse.getAllocatedContainers().size() == 0); + + + amClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED, + null, null); + + } finally { + if (amClient != null && amClient.getServiceState() == STATE.STARTED) { + amClient.stop(); + } + } + } + + @Test (timeout=60000) + public void testAMRMClient() throws YarnException, IOException { + AMRMClient amClient = null; + try { + // start am rm client + amClient = AMRMClient.createAMRMClient(attemptId); + amClient.init(conf); + amClient.start(); + + amClient.registerApplicationMaster("Host", 10000, ""); + + testAllocation((AMRMClientImpl)amClient); + + amClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED, + null, null); + + } finally { + if (amClient != null && amClient.getServiceState() == STATE.STARTED) { + amClient.stop(); + } + } + } + + private void testAllocation(final AMRMClientImpl amClient) + throws YarnException, IOException { + // setup container request + + assertTrue(amClient.ask.size() == 0); + assertTrue(amClient.release.size() == 0); + + amClient.addContainerRequest( + new ContainerRequest(capability, nodes, racks, priority, 1)); + amClient.addContainerRequest( + new ContainerRequest(capability, nodes, racks, priority, 3)); + amClient.removeContainerRequest( + new ContainerRequest(capability, nodes, racks, priority, 2)); + + int containersRequestedNode = amClient.remoteRequestsTable.get(priority) + .get(node).get(capability).remoteRequest.getNumContainers(); + int containersRequestedRack = amClient.remoteRequestsTable.get(priority) + .get(rack).get(capability).remoteRequest.getNumContainers(); + int containersRequestedAny = amClient.remoteRequestsTable.get(priority) + .get(ResourceRequest.ANY).get(capability).remoteRequest.getNumContainers(); + + assertTrue(containersRequestedNode == 2); + assertTrue(containersRequestedRack == 2); + assertTrue(containersRequestedAny == 2); + assertTrue(amClient.ask.size() == 3); + assertTrue(amClient.release.size() == 0); + + // RM should allocate container within 2 calls to allocate() + int allocatedContainerCount = 0; + int iterationsLeft = 2; + Set releases = new TreeSet(); + + ConcurrentHashMap nmTokens = amClient.getNMTokens(); + Assert.assertEquals(0, nmTokens.size()); + HashMap receivedNMTokens = new HashMap(); + + while (allocatedContainerCount < containersRequestedAny + && iterationsLeft-- > 0) { + AllocateResponse allocResponse = amClient.allocate(0.1f); + assertTrue(amClient.ask.size() == 0); + assertTrue(amClient.release.size() == 0); + + assertTrue(nodeCount == amClient.getClusterNodeCount()); + allocatedContainerCount += allocResponse.getAllocatedContainers().size(); + for(Container container : allocResponse.getAllocatedContainers()) { + ContainerId rejectContainerId = container.getId(); + releases.add(rejectContainerId); + amClient.releaseAssignedContainer(rejectContainerId); + } + Assert.assertEquals(nmTokens.size(), amClient.getNMTokens().size()); + Iterator nodeI = nmTokens.keySet().iterator(); + while (nodeI.hasNext()) { + String nodeId = nodeI.next(); + if (!receivedNMTokens.containsKey(nodeId)) { + receivedNMTokens.put(nodeId, nmTokens.get(nodeId)); + } else { + Assert.fail("Received token again for : " + nodeId); + } + } + nodeI = receivedNMTokens.keySet().iterator(); + while (nodeI.hasNext()) { + nmTokens.remove(nodeI.next()); + } + + if(allocatedContainerCount < containersRequestedAny) { + // sleep to let NM's heartbeat to RM and trigger allocations + sleep(1000); + } + } + + Assert.assertEquals(0, amClient.getNMTokens().size()); + // Should receive atleast 1 token + Assert.assertTrue(receivedNMTokens.size() > 0 + && receivedNMTokens.size() <= nodeCount); + + assertTrue(allocatedContainerCount == containersRequestedAny); + assertTrue(amClient.release.size() == 2); + assertTrue(amClient.ask.size() == 0); + + // need to tell the AMRMClient that we dont need these resources anymore + amClient.removeContainerRequest( + new ContainerRequest(capability, nodes, racks, priority, 2)); + assertTrue(amClient.ask.size() == 3); + // send 0 container count request for resources that are no longer needed + ResourceRequest snoopRequest = amClient.ask.iterator().next(); + assertTrue(snoopRequest.getNumContainers() == 0); + + // test RPC exception handling + amClient.addContainerRequest(new ContainerRequest(capability, nodes, + racks, priority, 2)); + snoopRequest = amClient.ask.iterator().next(); + assertTrue(snoopRequest.getNumContainers() == 2); + + ApplicationMasterProtocol realRM = amClient.rmClient; + try { + ApplicationMasterProtocol mockRM = mock(ApplicationMasterProtocol.class); + when(mockRM.allocate(any(AllocateRequest.class))).thenAnswer( + new Answer() { + public AllocateResponse answer(InvocationOnMock invocation) + throws Exception { + amClient.removeContainerRequest( + new ContainerRequest(capability, nodes, + racks, priority, 2)); + throw new Exception(); + } + }); + amClient.rmClient = mockRM; + amClient.allocate(0.1f); + }catch (Exception ioe) {} + finally { + amClient.rmClient = realRM; + } + + assertTrue(amClient.release.size() == 2); + assertTrue(amClient.ask.size() == 3); + snoopRequest = amClient.ask.iterator().next(); + // verify that the remove request made in between makeRequest and allocate + // has not been lost + assertTrue(snoopRequest.getNumContainers() == 0); + + iterationsLeft = 2; + // do a few iterations to ensure RM is not going send new containers + while(!releases.isEmpty() || iterationsLeft-- > 0) { + // inform RM of rejection + AllocateResponse allocResponse = amClient.allocate(0.1f); + // RM did not send new containers because AM does not need any + assertTrue(allocResponse.getAllocatedContainers().size() == 0); + if(allocResponse.getCompletedContainersStatuses().size() > 0) { + for(ContainerStatus cStatus :allocResponse + .getCompletedContainersStatuses()) { + if(releases.contains(cStatus.getContainerId())) { + assertTrue(cStatus.getState() == ContainerState.COMPLETE); + assertTrue(cStatus.getExitStatus() == -100); + releases.remove(cStatus.getContainerId()); + } + } + } + if(iterationsLeft > 0) { + // sleep to make sure NM's heartbeat + sleep(1000); + } + } + assertTrue(amClient.ask.size() == 0); + assertTrue(amClient.release.size() == 0); + } + + private void sleep(int sleepTime) { + try { + Thread.sleep(sleepTime); + } catch (InterruptedException e) { + e.printStackTrace(); + } + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientContainerRequest.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientContainerRequest.java new file mode 100644 index 0000000..735b8ed --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientContainerRequest.java @@ -0,0 +1,81 @@ +/** + * 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.hadoop.yarn.client.api.impl; + +import java.util.Arrays; +import java.util.List; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.net.DNSToSwitchMapping; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.ResourceRequest; +import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest; +import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class TestAMRMClientContainerRequest { + @Test + public void testFillInRacks() { + AMRMClientImpl client = new AMRMClientImpl( + ApplicationAttemptId.newInstance(ApplicationId.newInstance(0l, 0), 0)); + + Configuration conf = new Configuration(); + conf.setClass( + CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY, + MyResolver.class, DNSToSwitchMapping.class); + client.init(conf); + + Resource capability = Resource.newInstance(1024, 1); + ContainerRequest request = + new ContainerRequest(capability, new String[] {"host1", "host2"}, + new String[] {"/rack2"}, Priority.newInstance(1), 4); + client.addContainerRequest(request); + verifyResourceRequestLocation(client, request, "host1"); + verifyResourceRequestLocation(client, request, "host2"); + verifyResourceRequestLocation(client, request, "/rack1"); + verifyResourceRequestLocation(client, request, "/rack2"); + verifyResourceRequestLocation(client, request, ResourceRequest.ANY); + } + + private static class MyResolver implements DNSToSwitchMapping { + + @Override + public List resolve(List names) { + return Arrays.asList("/rack1"); + } + + @Override + public void reloadCachedMappings() {} + } + + private void verifyResourceRequestLocation( + AMRMClientImpl client, ContainerRequest request, + String location) { + ResourceRequest ask = client.remoteRequestsTable.get(request.getPriority()) + .get(location).get(request.getCapability()).remoteRequest; + assertEquals(location, ask.getResourceName()); + assertEquals(request.getContainerCount(), ask.getNumContainers()); + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java new file mode 100644 index 0000000..232903b --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java @@ -0,0 +1,356 @@ +/** + * 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.hadoop.yarn.client.api.impl; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Set; +import java.util.TreeSet; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.DataOutputBuffer; +import org.apache.hadoop.security.Credentials; +import org.apache.hadoop.service.Service.STATE; +import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; +import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; +import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; +import org.apache.hadoop.yarn.api.records.ContainerState; +import org.apache.hadoop.yarn.api.records.ContainerStatus; +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.hadoop.yarn.api.records.NodeReport; +import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.ResourceRequest; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; +import org.apache.hadoop.yarn.client.api.AMRMClient; +import org.apache.hadoop.yarn.client.api.NMClient; +import org.apache.hadoop.yarn.client.api.YarnClient; +import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest; +import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl; +import org.apache.hadoop.yarn.client.api.impl.NMClientImpl; +import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.server.MiniYARNCluster; +import org.apache.hadoop.yarn.util.Records; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +public class TestNMClient { + Configuration conf = null; + MiniYARNCluster yarnCluster = null; + YarnClientImpl yarnClient = null; + AMRMClientImpl rmClient = null; + NMClientImpl nmClient = null; + List nodeReports = null; + ApplicationAttemptId attemptId = null; + int nodeCount = 3; + + @Before + public void setup() throws YarnException, IOException { + // start minicluster + conf = new YarnConfiguration(); + yarnCluster = + new MiniYARNCluster(TestAMRMClient.class.getName(), nodeCount, 1, 1); + yarnCluster.init(conf); + yarnCluster.start(); + assertNotNull(yarnCluster); + assertEquals(STATE.STARTED, yarnCluster.getServiceState()); + + // start rm client + yarnClient = (YarnClientImpl) YarnClient.createYarnClient(); + yarnClient.init(conf); + yarnClient.start(); + assertNotNull(yarnClient); + assertEquals(STATE.STARTED, yarnClient.getServiceState()); + + // get node info + nodeReports = yarnClient.getNodeReports(); + + // submit new app + GetNewApplicationResponse newApp = yarnClient.getNewApplication(); + ApplicationId appId = newApp.getApplicationId(); + + ApplicationSubmissionContext appContext = Records + .newRecord(ApplicationSubmissionContext.class); + // set the application id + appContext.setApplicationId(appId); + // set the application name + appContext.setApplicationName("Test"); + // Set the priority for the application master + Priority pri = Priority.newInstance(0); + appContext.setPriority(pri); + // Set the queue to which this application is to be submitted in the RM + appContext.setQueue("default"); + // Set up the container launch context for the application master + ContainerLaunchContext amContainer = Records + .newRecord(ContainerLaunchContext.class); + appContext.setAMContainerSpec(amContainer); + // unmanaged AM + appContext.setUnmanagedAM(true); + // Create the request to send to the applications manager + SubmitApplicationRequest appRequest = Records + .newRecord(SubmitApplicationRequest.class); + appRequest.setApplicationSubmissionContext(appContext); + // Submit the application to the applications manager + yarnClient.submitApplication(appContext); + + // wait for app to start + int iterationsLeft = 30; + while (iterationsLeft > 0) { + ApplicationReport appReport = yarnClient.getApplicationReport(appId); + if (appReport.getYarnApplicationState() == + YarnApplicationState.ACCEPTED) { + attemptId = appReport.getCurrentApplicationAttemptId(); + break; + } + sleep(1000); + --iterationsLeft; + } + if (iterationsLeft == 0) { + fail("Application hasn't bee started"); + } + + // start am rm client + rmClient = + (AMRMClientImpl) AMRMClient + . createAMRMClient(attemptId); + rmClient.init(conf); + rmClient.start(); + assertNotNull(rmClient); + assertEquals(STATE.STARTED, rmClient.getServiceState()); + + // start am nm client + nmClient = (NMClientImpl) NMClient.createNMClient(); + nmClient.init(conf); + nmClient.start(); + assertNotNull(nmClient); + assertEquals(STATE.STARTED, nmClient.getServiceState()); + } + + @After + public void tearDown() { + rmClient.stop(); + yarnClient.stop(); + yarnCluster.stop(); + } + + private void stopNmClient(boolean stopContainers) { + assertNotNull("Null nmClient", nmClient); + // leave one unclosed + assertEquals(1, nmClient.startedContainers.size()); + // default true + assertTrue(nmClient.getCleanupRunningContainers().get()); + nmClient.cleanupRunningContainersOnStop(stopContainers); + assertEquals(stopContainers, nmClient.getCleanupRunningContainers().get()); + nmClient.stop(); + } + + @Test (timeout = 60000) + public void testNMClientNoCleanupOnStop() + throws YarnException, IOException { + + rmClient.registerApplicationMaster("Host", 10000, ""); + + testContainerManagement(nmClient, allocateContainers(rmClient, 5)); + + rmClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED, + null, null); + // don't stop the running containers + stopNmClient(false); + assertFalse(nmClient.startedContainers. isEmpty()); + //now cleanup + nmClient.cleanupRunningContainers(); + assertEquals(0, nmClient.startedContainers.size()); + } + + @Test (timeout = 60000) + public void testNMClient() + throws YarnException, IOException { + + rmClient.registerApplicationMaster("Host", 10000, ""); + + testContainerManagement(nmClient, allocateContainers(rmClient, 5)); + + rmClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED, + null, null); + // stop the running containers on close + assertFalse(nmClient.startedContainers.isEmpty()); + nmClient.cleanupRunningContainersOnStop(true); + assertTrue(nmClient.getCleanupRunningContainers().get()); + nmClient.stop(); + } + + private Set allocateContainers( + AMRMClientImpl rmClient, int num) + throws YarnException, IOException { + // setup container request + Resource capability = Resource.newInstance(1024, 0); + Priority priority = Priority.newInstance(0); + String node = nodeReports.get(0).getNodeId().getHost(); + String rack = nodeReports.get(0).getRackName(); + String[] nodes = new String[] {node}; + String[] racks = new String[] {rack}; + + for (int i = 0; i < num; ++i) { + rmClient.addContainerRequest(new ContainerRequest(capability, nodes, + racks, priority, 1)); + } + + int containersRequestedAny = rmClient.remoteRequestsTable.get(priority) + .get(ResourceRequest.ANY).get(capability).remoteRequest + .getNumContainers(); + + // RM should allocate container within 2 calls to allocate() + int allocatedContainerCount = 0; + int iterationsLeft = 2; + Set containers = new TreeSet(); + while (allocatedContainerCount < containersRequestedAny + && iterationsLeft > 0) { + AllocateResponse allocResponse = rmClient.allocate(0.1f); + + allocatedContainerCount += allocResponse.getAllocatedContainers().size(); + for(Container container : allocResponse.getAllocatedContainers()) { + containers.add(container); + } + if(allocatedContainerCount < containersRequestedAny) { + // sleep to let NM's heartbeat to RM and trigger allocations + sleep(1000); + } + + --iterationsLeft; + } + return containers; + } + + private void testContainerManagement(NMClientImpl nmClient, + Set containers) throws YarnException, IOException { + int size = containers.size(); + int i = 0; + for (Container container : containers) { + // getContainerStatus shouldn't be called before startContainer, + // otherwise, NodeManager cannot find the container + try { + nmClient.getContainerStatus(container.getId(), container.getNodeId(), + container.getContainerToken()); + fail("Exception is expected"); + } catch (YarnException e) { + assertTrue("The thrown exception is not expected", + e.getMessage().contains("is not handled by this NodeManager")); + } + + // stopContainer shouldn't be called before startContainer, + // otherwise, an exception will be thrown + try { + nmClient.stopContainer(container.getId(), container.getNodeId(), + container.getContainerToken()); + fail("Exception is expected"); + } catch (YarnException e) { + if (!e.getMessage() + .contains("is either not started yet or already stopped")) { + throw (AssertionError) + (new AssertionError("Exception is not expected: " + e).initCause( + e)); + } + } + + Credentials ts = new Credentials(); + DataOutputBuffer dob = new DataOutputBuffer(); + ts.writeTokenStorageToStream(dob); + ByteBuffer securityTokens = + ByteBuffer.wrap(dob.getData(), 0, dob.getLength()); + ContainerLaunchContext clc = + Records.newRecord(ContainerLaunchContext.class); + clc.setTokens(securityTokens); + try { + nmClient.startContainer(container, clc); + } catch (YarnException e) { + throw (AssertionError) + (new AssertionError("Exception is not expected: " + e).initCause(e)); + } + + // leave one container unclosed + if (++i < size) { + // NodeManager may still need some time to make the container started + testGetContainerStatus(container, i, ContainerState.RUNNING, "", + -1000); + + try { + nmClient.stopContainer(container.getId(), container.getNodeId(), + container.getContainerToken()); + } catch (YarnException e) { + throw (AssertionError) + (new AssertionError("Exception is not expected: " + e) + .initCause(e)); + } + + // getContainerStatus can be called after stopContainer + testGetContainerStatus(container, i, ContainerState.COMPLETE, + "Container killed by the ApplicationMaster.", 143); + } + } + } + + private void sleep(int sleepTime) { + try { + Thread.sleep(sleepTime); + } catch (InterruptedException e) { + e.printStackTrace(); + } + } + + private void testGetContainerStatus(Container container, int index, + ContainerState state, String diagnostics, int exitStatus) + throws YarnException, IOException { + while (true) { + try { + ContainerStatus status = nmClient.getContainerStatus( + container.getId(), container.getNodeId(), + container.getContainerToken()); + // NodeManager may still need some time to get the stable + // container status + if (status.getState() == state) { + assertEquals(container.getId(), status.getContainerId()); + assertTrue("" + index + ": " + status.getDiagnostics(), + status.getDiagnostics().contains(diagnostics)); + assertEquals(exitStatus, status.getExitStatus()); + break; + } + Thread.sleep(100); + } catch (InterruptedException e) { + e.printStackTrace(); + } + } + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java new file mode 100644 index 0000000..1c9b750 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java @@ -0,0 +1,177 @@ +/** +* 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.hadoop.yarn.client.api.impl; + +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.util.HashMap; + +import junit.framework.Assert; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse; +import org.apache.hadoop.yarn.api.records.ApplicationAccessType; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; +import org.apache.hadoop.yarn.client.api.YarnClient; +import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.server.resourcemanager.MockRM; +import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; +import org.apache.log4j.Level; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.junit.Test; + +public class TestYarnClient { + + @Test + public void test() { + // More to come later. + } + + @Test + public void testClientStop() { + Configuration conf = new Configuration(); + ResourceManager rm = new ResourceManager(); + rm.init(conf); + rm.start(); + + YarnClient client = YarnClient.createYarnClient(); + client.init(conf); + client.start(); + client.stop(); + } + + @Test (timeout = 30000) + public void testSubmitApplication() { + Configuration conf = new Configuration(); + conf.setLong(YarnConfiguration.YARN_CLIENT_APP_SUBMISSION_POLL_INTERVAL_MS, + 100); // speed up tests + final YarnClient client = new MockYarnClient(); + client.init(conf); + client.start(); + + YarnApplicationState[] exitStates = new YarnApplicationState[] + { + YarnApplicationState.SUBMITTED, + YarnApplicationState.ACCEPTED, + YarnApplicationState.RUNNING, + YarnApplicationState.FINISHED, + YarnApplicationState.FAILED, + YarnApplicationState.KILLED + }; + for (int i = 0; i < exitStates.length; ++i) { + ApplicationSubmissionContext context = + mock(ApplicationSubmissionContext.class); + ApplicationId applicationId = ApplicationId.newInstance( + System.currentTimeMillis(), i); + when(context.getApplicationId()).thenReturn(applicationId); + ((MockYarnClient) client).setYarnApplicationState(exitStates[i]); + try { + client.submitApplication(context); + } catch (YarnException e) { + Assert.fail("Exception is not expected."); + } catch (IOException e) { + Assert.fail("Exception is not expected."); + } + verify(((MockYarnClient) client).mockReport,times(4 * i + 4)) + .getYarnApplicationState(); + } + + client.stop(); + } + + @Test(timeout = 30000) + public void testApplicationType() throws Exception { + Logger rootLogger = LogManager.getRootLogger(); + rootLogger.setLevel(Level.DEBUG); + MockRM rm = new MockRM(); + rm.start(); + RMApp app = rm.submitApp(2000); + RMApp app1 = + rm.submitApp(200, "name", "user", + new HashMap(), false, "default", -1, + null, "MAPREDUCE"); + Assert.assertEquals("YARN", app.getApplicationType()); + Assert.assertEquals("MAPREDUCE", app1.getApplicationType()); + rm.stop(); + } + + @Test(timeout = 30000) + public void testApplicationTypeLimit() throws Exception { + Logger rootLogger = LogManager.getRootLogger(); + rootLogger.setLevel(Level.DEBUG); + MockRM rm = new MockRM(); + rm.start(); + RMApp app1 = + rm.submitApp(200, "name", "user", + new HashMap(), false, "default", -1, + null, "MAPREDUCE-LENGTH-IS-20"); + Assert.assertEquals("MAPREDUCE-LENGTH-IS-", app1.getApplicationType()); + rm.stop(); + } + + private static class MockYarnClient extends YarnClientImpl { + private ApplicationReport mockReport; + + public MockYarnClient() { + super(); + } + + @Override + public void start() { + rmClient = mock(ApplicationClientProtocol.class); + GetApplicationReportResponse mockResponse = + mock(GetApplicationReportResponse.class); + mockReport = mock(ApplicationReport.class); + try{ + when(rmClient.getApplicationReport(any( + GetApplicationReportRequest.class))).thenReturn(mockResponse); + } catch (YarnException e) { + Assert.fail("Exception is not expected."); + } catch (IOException e) { + Assert.fail("Exception is not expected."); + } + when(mockResponse.getApplicationReport()).thenReturn(mockReport); + } + + @Override + public void stop() { + } + + public void setYarnApplicationState(YarnApplicationState state) { + when(mockReport.getYarnApplicationState()).thenReturn( + YarnApplicationState.NEW, YarnApplicationState.NEW_SAVING, + YarnApplicationState.NEW_SAVING, state); + } + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java index 49d7867..365bc8e 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java @@ -46,7 +46,7 @@ import org.apache.hadoop.yarn.api.records.NodeState; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.YarnApplicationState; -import org.apache.hadoop.yarn.client.YarnClient; +import org.apache.hadoop.yarn.client.api.YarnClient; import org.apache.hadoop.yarn.util.Records; import org.junit.Before; import org.junit.Test;