Procházet zdrojové kódy

YARN-83. Change package of YarnClient to include apache (Bikas Saha via tgraves). This als includes YARN-29.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.23@1443118 13f79535-47bb-0310-9956-ffa450edef68
Thomas Graves před 12 roky
rodič
revize
857529699d
16 změnil soubory, kde provedl 1781 přidání a 105 odebrání
  1. 5 0
      hadoop-yarn-project/CHANGES.txt
  2. 4 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml
  3. 20 102
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java
  4. 19 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/dev-support/findbugs-exclude.xml
  5. 55 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/pom.xml
  6. 234 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/YarnClient.java
  7. 258 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/YarnClientImpl.java
  8. 159 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
  9. 145 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeCLI.java
  10. 63 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/YarnCLI.java
  11. 234 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/hadoop/yarn/client/YarnClient.java
  12. 258 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/hadoop/yarn/client/YarnClientImpl.java
  13. 47 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestYarnClient.java
  14. 247 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
  15. 30 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/hadoop/yarn/client/TestYarnClient.java
  16. 3 3
      hadoop-yarn-project/hadoop-yarn/pom.xml

+ 5 - 0
hadoop-yarn-project/CHANGES.txt

@@ -25,6 +25,11 @@ Release 0.23.7 - UNRELEASED
     YARN-364. AggregatedLogDeletionService can take too long to delete logs
     (jlowe)
 
+    YARN-29. Add a yarn-client module (Vinod Kumar Vavilapalli via tgraves)
+
+    YARN-83. Change package of YarnClient to include apache (Bikas Saha via
+    tgraves)
+
 Release 0.23.6 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 4 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml

@@ -41,6 +41,10 @@
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-yarn-common</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-client</artifactId>
+    </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-yarn-server-nodemanager</artifactId>

+ 20 - 102
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java

@@ -22,7 +22,6 @@ import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
-import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -44,20 +43,8 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.ClientRMProtocol;
-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.GetNewApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoResponse;
 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;
@@ -73,13 +60,13 @@ import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 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.YarnClientImpl;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
-import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.Records;
 
-
 /**
  * Client for Distributed Shell application submission to YARN.
  * 
@@ -113,19 +100,13 @@ import org.apache.hadoop.yarn.util.Records;
  */
 @InterfaceAudience.Public
 @InterfaceStability.Unstable
-public class Client {
+public class Client extends YarnClientImpl {
 
   private static final Log LOG = LogFactory.getLog(Client.class);
 
   // Configuration
   private Configuration conf;
 
-  // RPC to communicate to RM
-  private YarnRPC rpc;
-
-  // Handle to talk to the Resource Manager/Applications Manager
-  private ClientRMProtocol applicationsManager;
-
   // Application master specific info to register a new Application with RM/ASM
   private String appName = "";
   // App master priority
@@ -198,9 +179,9 @@ public class Client {
   /**
    */
   public Client(Configuration conf) throws Exception  {
-    // Set up the configuration and RPC
+    super();
     this.conf = conf;
-    rpc = YarnRPC.create(conf);
+    init(conf);
   }
 
   /**
@@ -331,22 +312,17 @@ public class Client {
    * @throws IOException
    */
   public boolean run() throws IOException {
-    LOG.info("Starting Client");
 
-    // Connect to ResourceManager 	
-    connectToASM();
-    assert(applicationsManager != null);		
+    LOG.info("Running Client");
+    start();
 
-    // Use ClientRMProtocol handle to general cluster information 
-    GetClusterMetricsRequest clusterMetricsReq = Records.newRecord(GetClusterMetricsRequest.class);
-    GetClusterMetricsResponse clusterMetricsResp = applicationsManager.getClusterMetrics(clusterMetricsReq);
+    YarnClusterMetrics clusterMetrics = super.getYarnClusterMetrics();
     LOG.info("Got Cluster metric info from ASM" 
-        + ", numNodeManagers=" + clusterMetricsResp.getClusterMetrics().getNumNodeManagers());
+        + ", numNodeManagers=" + clusterMetrics.getNumNodeManagers());
 
-    GetClusterNodesRequest clusterNodesReq = Records.newRecord(GetClusterNodesRequest.class);
-    GetClusterNodesResponse clusterNodesResp = applicationsManager.getClusterNodes(clusterNodesReq);
+    List<NodeReport> clusterNodeReports = super.getNodeReports();
     LOG.info("Got Cluster node info from ASM");
-    for (NodeReport node : clusterNodesResp.getNodeReports()) {
+    for (NodeReport node : clusterNodeReports) {
       LOG.info("Got node report from ASM for"
           + ", nodeId=" + node.getNodeId() 
           + ", nodeAddress" + node.getHttpAddress()
@@ -355,9 +331,7 @@ public class Client {
           + ", nodeHealthStatus" + node.getNodeHealthStatus());
     }
 
-    GetQueueInfoRequest queueInfoReq = Records.newRecord(GetQueueInfoRequest.class);
-    GetQueueInfoResponse queueInfoResp = applicationsManager.getQueueInfo(queueInfoReq);		
-    QueueInfo queueInfo = queueInfoResp.getQueueInfo();
+    QueueInfo queueInfo = super.getQueueInfo(this.amQueue);		
     LOG.info("Queue info"
         + ", queueName=" + queueInfo.getQueueName()
         + ", queueCurrentCapacity=" + queueInfo.getCurrentCapacity()
@@ -365,9 +339,7 @@ public class Client {
         + ", queueApplicationCount=" + queueInfo.getApplications().size()
         + ", queueChildQueueCount=" + queueInfo.getChildQueues().size());		
 
-    GetQueueUserAclsInfoRequest queueUserAclsReq = Records.newRecord(GetQueueUserAclsInfoRequest.class);
-    GetQueueUserAclsInfoResponse queueUserAclsResp = applicationsManager.getQueueUserAcls(queueUserAclsReq);				
-    List<QueueUserACLInfo> listAclInfo = queueUserAclsResp.getUserAclsInfoList();
+    List<QueueUserACLInfo> listAclInfo = super.getQueueAclsInfo();				
     for (QueueUserACLInfo aclInfo : listAclInfo) {
       for (QueueACL userAcl : aclInfo.getUserAcls()) {
         LOG.info("User ACL Info for Queue"
@@ -377,7 +349,7 @@ public class Client {
     }		
 
     // Get a new application id 
-    GetNewApplicationResponse newApp = getApplication();
+    GetNewApplicationResponse newApp = super.getNewApplication();
     ApplicationId appId = newApp.getApplicationId();
 
     // TODO get min/max resource capabilities from RM and change memory ask if needed
@@ -598,16 +570,12 @@ public class Client {
     // TODO can it be empty? 
     appContext.setUser(amUser);
 
-    // 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
     // SubmitApplicationResponse submitResp = applicationsManager.submitApplication(appRequest);
     // Ignore the response as either a valid response object is returned on success 
     // or an exception thrown to denote some form of a failure
     LOG.info("Submitting application to ASM");
-    applicationsManager.submitApplication(appRequest);
+    super.submitApplication(appContext);
 
     // TODO
     // Try submitting the same request again
@@ -637,10 +605,7 @@ public class Client {
       }
 
       // Get application report for the appId we are interested in 
-      GetApplicationReportRequest reportRequest = Records.newRecord(GetApplicationReportRequest.class);
-      reportRequest.setApplicationId(appId);
-      GetApplicationReportResponse reportResponse = applicationsManager.getApplicationReport(reportRequest);
-      ApplicationReport report = reportResponse.getApplicationReport();
+      ApplicationReport report = super.getApplicationReport(appId);
 
       LOG.info("Got application report from ASM for"
           + ", appId=" + appId.getId()
@@ -679,7 +644,7 @@ public class Client {
 
       if (System.currentTimeMillis() > (clientStartTime + clientTimeout)) {
         LOG.info("Reached client specified timeout for application. Killing application");
-        killApplication(appId);
+        forceKillApplication(appId);
         return false;				
       }
     }			
@@ -691,61 +656,14 @@ public class Client {
    * @param appId Application Id to be killed. 
    * @throws YarnRemoteException
    */
-  private void killApplication(ApplicationId appId) throws YarnRemoteException {
-    KillApplicationRequest request = Records.newRecord(KillApplicationRequest.class);		
+  private void forceKillApplication(ApplicationId appId) throws YarnRemoteException {
     // TODO clarify whether multiple jobs with the same app id can be submitted and be running at 
     // the same time. 
     // If yes, can we kill a particular attempt only?
-    request.setApplicationId(appId);
-    // KillApplicationResponse response = applicationsManager.forceKillApplication(request);		
+
     // Response can be ignored as it is non-null on success or 
     // throws an exception in case of failures
-    applicationsManager.forceKillApplication(request);	
-  }
-
-  /**
-   * Connect to the Resource Manager/Applications Manager
-   * @return Handle to communicate with the ASM
-   * @throws IOException 
-   */
-  private void connectToASM() throws IOException {
-
-    /*
-		UserGroupInformation user = UserGroupInformation.getCurrentUser();
-		applicationsManager = user.doAs(new PrivilegedAction<ClientRMProtocol>() {
-			public ClientRMProtocol run() {
-				InetSocketAddress rmAddress = NetUtils.createSocketAddr(conf.get(
-					YarnConfiguration.RM_SCHEDULER_ADDRESS,
-					YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS));		
-				LOG.info("Connecting to ResourceManager at " + rmAddress);
-				Configuration appsManagerServerConf = new Configuration(conf);
-				appsManagerServerConf.setClass(YarnConfiguration.YARN_SECURITY_INFO,
-				ClientRMSecurityInfo.class, SecurityInfo.class);
-				ClientRMProtocol asm = ((ClientRMProtocol) rpc.getProxy(ClientRMProtocol.class, rmAddress, appsManagerServerConf));
-				return asm;
-			}
-		});
-     */
-    YarnConfiguration yarnConf = new YarnConfiguration(conf);
-    InetSocketAddress rmAddress = yarnConf.getSocketAddr(
-        YarnConfiguration.RM_ADDRESS,
-        YarnConfiguration.DEFAULT_RM_ADDRESS,
-        YarnConfiguration.DEFAULT_RM_PORT);
-    LOG.info("Connecting to ResourceManager at " + rmAddress);
-    applicationsManager = ((ClientRMProtocol) rpc.getProxy(
-        ClientRMProtocol.class, rmAddress, conf));
-  }		
-
-  /**
-   * Get a new application from the ASM 
-   * @return New Application
-   * @throws YarnRemoteException
-   */
-  private GetNewApplicationResponse getApplication() throws YarnRemoteException {
-    GetNewApplicationRequest request = Records.newRecord(GetNewApplicationRequest.class);		
-    GetNewApplicationResponse response = applicationsManager.getNewApplication(request);
-    LOG.info("Got new application id=" + response.getApplicationId());		
-    return response;		
+    super.killApplication(appId);	
   }
 
   private static String getTestRuntimeClasspath() {

+ 19 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/dev-support/findbugs-exclude.xml

@@ -0,0 +1,19 @@
+<!--
+   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.
+-->
+
+<FindBugsFilter>
+</FindBugsFilter>

+ 55 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/pom.xml

@@ -0,0 +1,55 @@
+<?xml version="1.0"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<project xmlns:pom="http://maven.apache.org/POM/4.0.0">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>hadoop-yarn</artifactId>
+    <groupId>org.apache.hadoop</groupId>
+    <version>0.23.7-SNAPSHOT</version>
+  </parent>
+  <groupId>org.apache.hadoop</groupId>
+  <artifactId>hadoop-yarn-client</artifactId>
+  <version>0.23.7-SNAPSHOT</version>
+  <name>hadoop-yarn-client</name>
+
+  <dependencies>
+  	<dependency>
+  		<groupId>org.apache.hadoop</groupId>
+  		<artifactId>hadoop-yarn-api</artifactId>
+  	</dependency>
+  	<dependency>
+  		<groupId>org.apache.hadoop</groupId>
+  		<artifactId>hadoop-yarn-common</artifactId>
+  	</dependency>
+        <dependency>
+                <groupId>org.apache.hadoop</groupId>
+                <artifactId>hadoop-yarn-server-resourcemanager</artifactId>
+                <scope>test</scope>
+        </dependency>
+        <dependency>
+                <groupId>org.apache.hadoop</groupId>
+                <artifactId>hadoop-yarn-server-tests</artifactId>
+                <scope>test</scope>
+                <type>test-jar</type>
+        </dependency>
+        <dependency>
+                <groupId>org.apache.hadoop</groupId>
+                <artifactId>hadoop-yarn-server-resourcemanager</artifactId>
+                <scope>test</scope>
+                <type>test-jar</type>
+        </dependency>
+
+  </dependencies>
+</project>

+ 234 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/YarnClient.java

@@ -0,0 +1,234 @@
+/**
+* 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.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.Text;
+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.DelegationToken;
+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.YarnClusterMetrics;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.service.Service;
+
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface YarnClient extends Service {
+
+  /**
+   * <p>
+   * Obtain a new {@link ApplicationId} for submitting new applications.
+   * </p>
+   * 
+   * <p>
+   * Returns a response which contains {@link ApplicationId} that can be used to
+   * submit a new application. See
+   * {@link #submitApplication(ApplicationSubmissionContext)}.
+   * </p>
+   * 
+   * <p>
+   * See {@link GetNewApplicationResponse} for other information that is
+   * returned.
+   * </p>
+   * 
+   * @return response containing the new <code>ApplicationId</code> to be used
+   *         to submit an application
+   * @throws YarnRemoteException
+   */
+  GetNewApplicationResponse getNewApplication() throws YarnRemoteException;
+
+  /**
+   * <p>
+   * Submit a new application to <code>YARN.</code>
+   * </p>
+   * 
+   * @param appContext
+   *          {@link ApplicationSubmissionContext} containing all the details
+   *          needed to submit a new application
+   * @return {@link ApplicationId} of the accepted application
+   * @throws YarnRemoteException
+   * @see #getNewApplication()
+   */
+  ApplicationId submitApplication(ApplicationSubmissionContext appContext)
+      throws YarnRemoteException;
+
+  /**
+   * <p>
+   * Kill an application identified by given ID.
+   * </p>
+   * 
+   * @param applicationId
+   *          {@link ApplicationId} of the application that needs to be killed
+   * @throws YarnRemoteException
+   *           in case of errors or if YARN rejects the request due to
+   *           access-control restrictions.
+   * @see #getQueueAclsInfo()
+   */
+  void killApplication(ApplicationId applicationId) throws YarnRemoteException;
+
+  /**
+   * <p>
+   * Get a report of the given Application.
+   * </p>
+   * 
+   * <p>
+   * In secure mode, <code>YARN</code> verifies access to the application, queue
+   * etc. before accepting the request.
+   * </p>
+   * 
+   * <p>
+   * If the user does not have <code>VIEW_APP</code> access then the following
+   * fields in the report will be set to stubbed values:
+   * <ul>
+   * <li>host - set to "N/A"</li>
+   * <li>RPC port - set to -1</li>
+   * <li>client token - set to "N/A"</li>
+   * <li>diagnostics - set to "N/A"</li>
+   * <li>tracking URL - set to "N/A"</li>
+   * <li>original tracking URL - set to "N/A"</li>
+   * <li>resource usage report - all values are -1</li>
+   * </ul>
+   * </p>
+   * 
+   * @param appId
+   *          {@link ApplicationId} of the application that needs a report
+   * @return application report
+   * @throws YarnRemoteException
+   */
+  ApplicationReport getApplicationReport(ApplicationId appId)
+      throws YarnRemoteException;
+
+  /**
+   * <p>
+   * Get a report (ApplicationReport) of all Applications in the cluster.
+   * </p>
+   * 
+   * <p>
+   * If the user does not have <code>VIEW_APP</code> access for an application
+   * then the corresponding report will be filtered as described in
+   * {@link #getApplicationReport(ApplicationId)}.
+   * </p>
+   * 
+   * @return a list of reports of all running applications
+   * @throws YarnRemoteException
+   */
+  List<ApplicationReport> getApplicationList() throws YarnRemoteException;
+
+  /**
+   * <p>
+   * Get metrics ({@link YarnClusterMetrics}) about the cluster.
+   * </p>
+   * 
+   * @return cluster metrics
+   * @throws YarnRemoteException
+   */
+  YarnClusterMetrics getYarnClusterMetrics() throws YarnRemoteException;
+
+  /**
+   * <p>
+   * Get a report of all nodes ({@link NodeReport}) in the cluster.
+   * </p>
+   * 
+   * @return A list of report of all nodes
+   * @throws YarnRemoteException
+   */
+  List<NodeReport> getNodeReports() throws YarnRemoteException;
+
+  /**
+   * <p>
+   * 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 DelegationToken}) that can be used to
+   *         talk to YARN
+   * @throws YarnRemoteException
+   */
+  DelegationToken getRMDelegationToken(Text renewer) throws YarnRemoteException;
+
+  /**
+   * <p>
+   * Get information ({@link QueueInfo}) about a given <em>queue</em>.
+   * </p>
+   * 
+   * @param queueName
+   *          Name of the queue whose information is needed
+   * @return queue information
+   * @throws YarnRemoteException
+   *           in case of errors or if YARN rejects the request due to
+   *           access-control restrictions.
+   */
+  QueueInfo getQueueInfo(String queueName) throws YarnRemoteException;
+
+  /**
+   * <p>
+   * Get information ({@link QueueInfo}) about all queues, recursively if there
+   * is a hierarchy
+   * </p>
+   * 
+   * @return a list of queue-information for all queues
+   * @throws YarnRemoteException
+   */
+  List<QueueInfo> getAllQueues() throws YarnRemoteException;
+
+  /**
+   * <p>
+   * Get information ({@link QueueInfo}) about top level queues.
+   * </p>
+   * 
+   * @return a list of queue-information for all the top-level queues
+   * @throws YarnRemoteException
+   */
+  List<QueueInfo> getRootQueueInfos() throws YarnRemoteException;
+
+  /**
+   * <p>
+   * Get information ({@link QueueInfo}) about all the immediate children queues
+   * of the given queue
+   * </p>
+   * 
+   * @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 YarnRemoteException
+   */
+  List<QueueInfo> getChildQueueInfos(String parent) throws YarnRemoteException;
+
+  /**
+   * <p>
+   * Get information about <em>acls</em> for <em>current user</em> on all the
+   * existing queues.
+   * </p>
+   * 
+   * @return a list of queue acls ({@link QueueUserACLInfo}) for
+   *         <em>current user</em>
+   * @throws YarnRemoteException
+   */
+  List<QueueUserACLInfo> getQueueAclsInfo() throws YarnRemoteException;
+}

+ 258 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/YarnClientImpl.java

@@ -0,0 +1,258 @@
+/**
+* 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.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.ClientRMProtocol;
+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.DelegationToken;
+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.YarnClusterMetrics;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.service.AbstractService;
+import org.apache.hadoop.yarn.util.Records;
+
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class YarnClientImpl extends AbstractService implements YarnClient {
+
+  private static final Log LOG = LogFactory.getLog(YarnClientImpl.class);
+
+  protected ClientRMProtocol rmClient;
+  protected InetSocketAddress rmAddress;
+
+  private static final String ROOT = "root";
+
+  public YarnClientImpl() {
+    super(YarnClientImpl.class.getName());
+  }
+
+  private static InetSocketAddress getRmAddress(Configuration conf) {
+    return conf.getSocketAddr(YarnConfiguration.RM_ADDRESS,
+      YarnConfiguration.DEFAULT_RM_ADDRESS, YarnConfiguration.DEFAULT_RM_PORT);
+  }
+
+  @Override
+  public synchronized void init(Configuration conf) {
+    this.rmAddress = getRmAddress(conf);
+    super.init(conf);
+  }
+
+  @Override
+  public synchronized void start() {
+    YarnRPC rpc = YarnRPC.create(getConfig());
+
+    this.rmClient =
+        (ClientRMProtocol) rpc.getProxy(ClientRMProtocol.class, rmAddress,
+          getConfig());
+    LOG.debug("Connecting to ResourceManager at " + rmAddress);
+    super.start();
+  }
+
+  @Override
+  public synchronized void stop() {
+    RPC.stopProxy(this.rmClient);
+    super.stop();
+  }
+
+  @Override
+  public GetNewApplicationResponse getNewApplication()
+      throws YarnRemoteException {
+    GetNewApplicationRequest request =
+        Records.newRecord(GetNewApplicationRequest.class);
+    return rmClient.getNewApplication(request);
+  }
+
+  @Override
+  public ApplicationId
+      submitApplication(ApplicationSubmissionContext appContext)
+          throws YarnRemoteException {
+    ApplicationId applicationId = appContext.getApplicationId();
+    appContext.setApplicationId(applicationId);
+    SubmitApplicationRequest request =
+        Records.newRecord(SubmitApplicationRequest.class);
+    request.setApplicationSubmissionContext(appContext);
+    rmClient.submitApplication(request);
+    LOG.info("Submitted application " + applicationId + " to ResourceManager"
+        + " at " + rmAddress);
+    return applicationId;
+  }
+
+  @Override
+  public void killApplication(ApplicationId applicationId)
+      throws YarnRemoteException {
+    LOG.info("Killing application " + applicationId);
+    KillApplicationRequest request =
+        Records.newRecord(KillApplicationRequest.class);
+    request.setApplicationId(applicationId);
+    rmClient.forceKillApplication(request);
+  }
+
+  @Override
+  public ApplicationReport getApplicationReport(ApplicationId appId)
+      throws YarnRemoteException {
+    GetApplicationReportRequest request =
+        Records.newRecord(GetApplicationReportRequest.class);
+    request.setApplicationId(appId);
+    GetApplicationReportResponse response =
+        rmClient.getApplicationReport(request);
+    return response.getApplicationReport();
+  }
+
+  @Override
+  public List<ApplicationReport> getApplicationList()
+      throws YarnRemoteException {
+    GetAllApplicationsRequest request =
+        Records.newRecord(GetAllApplicationsRequest.class);
+    GetAllApplicationsResponse response = rmClient.getAllApplications(request);
+    return response.getApplicationList();
+  }
+
+  @Override
+  public YarnClusterMetrics getYarnClusterMetrics() throws YarnRemoteException {
+    GetClusterMetricsRequest request =
+        Records.newRecord(GetClusterMetricsRequest.class);
+    GetClusterMetricsResponse response = rmClient.getClusterMetrics(request);
+    return response.getClusterMetrics();
+  }
+
+  @Override
+  public List<NodeReport> getNodeReports() throws YarnRemoteException {
+    GetClusterNodesRequest request =
+        Records.newRecord(GetClusterNodesRequest.class);
+    GetClusterNodesResponse response = rmClient.getClusterNodes(request);
+    return response.getNodeReports();
+  }
+
+  @Override
+  public DelegationToken getRMDelegationToken(Text renewer)
+      throws YarnRemoteException {
+    /* 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 YarnRemoteException {
+    GetQueueInfoRequest request =
+        getQueueInfoRequest(queueName, true, false, false);
+    Records.newRecord(GetQueueInfoRequest.class);
+    return rmClient.getQueueInfo(request).getQueueInfo();
+  }
+
+  @Override
+  public List<QueueUserACLInfo> getQueueAclsInfo() throws YarnRemoteException {
+    GetQueueUserAclsInfoRequest request =
+        Records.newRecord(GetQueueUserAclsInfoRequest.class);
+    return rmClient.getQueueUserAcls(request).getUserAclsInfoList();
+  }
+
+  @Override
+  public List<QueueInfo> getAllQueues() throws YarnRemoteException {
+    List<QueueInfo> queues = new ArrayList<QueueInfo>();
+
+    QueueInfo rootQueue =
+        rmClient.getQueueInfo(getQueueInfoRequest(ROOT, false, true, true))
+          .getQueueInfo();
+    getChildQueues(rootQueue, queues, true);
+    return queues;
+  }
+
+  @Override
+  public List<QueueInfo> getRootQueueInfos() throws YarnRemoteException {
+    List<QueueInfo> queues = new ArrayList<QueueInfo>();
+
+    QueueInfo rootQueue =
+        rmClient.getQueueInfo(getQueueInfoRequest(ROOT, false, true, true))
+          .getQueueInfo();
+    getChildQueues(rootQueue, queues, false);
+    return queues;
+  }
+
+  @Override
+  public List<QueueInfo> getChildQueueInfos(String parent)
+      throws YarnRemoteException {
+    List<QueueInfo> queues = new ArrayList<QueueInfo>();
+
+    QueueInfo parentQueue =
+        rmClient.getQueueInfo(getQueueInfoRequest(parent, false, true, false))
+          .getQueueInfo();
+    getChildQueues(parentQueue, queues, true);
+    return queues;
+  }
+
+  private void getChildQueues(QueueInfo parent, List<QueueInfo> queues,
+      boolean recursive) {
+    List<QueueInfo> childQueues = parent.getChildQueues();
+
+    for (QueueInfo child : childQueues) {
+      queues.add(child);
+      if (recursive) {
+        getChildQueues(child, queues, recursive);
+      }
+    }
+  }
+}

+ 159 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java

@@ -0,0 +1,159 @@
+/**
+ * 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.cli;
+
+import java.io.PrintWriter;
+import java.util.List;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Options;
+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.YarnRemoteException;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+
+public class ApplicationCLI extends YarnCLI {
+  private static final String APPLICATIONS_PATTERN = "%30s\t%20s\t%10s\t%10s\t%18s\t%18s\t%35s\n";
+
+  public static void main(String[] args) throws Exception {
+    ApplicationCLI cli = new ApplicationCLI();
+    cli.setSysOutPrintStream(System.out);
+    cli.setSysErrPrintStream(System.err);
+    int res = ToolRunner.run(cli, args);
+    cli.stop();
+    System.exit(res);
+  }
+
+  @Override
+  public int run(String[] args) throws Exception {
+
+    Options opts = new Options();
+    opts.addOption(STATUS_CMD, true, "Prints the status of the application.");
+    opts.addOption(LIST_CMD, false, "Lists all the Applications from RM.");
+    opts.addOption(KILL_CMD, true, "Kills the application.");
+    CommandLine cliParser = new GnuParser().parse(opts, args);
+
+    int exitCode = -1;
+    if (cliParser.hasOption(STATUS_CMD)) {
+      if (args.length != 2) {
+        printUsage(opts);
+        return exitCode;
+      }
+      printApplicationReport(cliParser.getOptionValue(STATUS_CMD));
+    } else if (cliParser.hasOption(LIST_CMD)) {
+      listAllApplications();
+    } else if (cliParser.hasOption(KILL_CMD)) {
+      if (args.length != 2) {
+        printUsage(opts);
+        return exitCode;
+      }
+      killApplication(cliParser.getOptionValue(KILL_CMD));
+    } else {
+      syserr.println("Invalid Command Usage : ");
+      printUsage(opts);
+    }
+    return 0;
+  }
+
+  /**
+   * It prints the usage of the command
+   * 
+   * @param opts
+   */
+  private void printUsage(Options opts) {
+    new HelpFormatter().printHelp("application", opts);
+  }
+
+  /**
+   * Lists all the applications present in the Resource Manager
+   * 
+   * @throws YarnRemoteException
+   */
+  private void listAllApplications() throws YarnRemoteException {
+    PrintWriter writer = new PrintWriter(sysout);
+    List<ApplicationReport> appsReport = client.getApplicationList();
+
+    writer.println("Total Applications:" + appsReport.size());
+    writer.printf(APPLICATIONS_PATTERN, "Application-Id",
+        "Application-Name", "User", "Queue", "State", "Final-State",
+        "Tracking-URL");
+    for (ApplicationReport appReport : appsReport) {
+      writer.printf(APPLICATIONS_PATTERN, appReport.getApplicationId(),
+          appReport.getName(), appReport.getUser(), appReport.getQueue(),
+          appReport.getYarnApplicationState(), appReport
+              .getFinalApplicationStatus(), appReport.getOriginalTrackingUrl());
+    }
+    writer.flush();
+  }
+
+  /**
+   * Kills the application with the application id as appId
+   * 
+   * @param applicationId
+   * @throws YarnRemoteException
+   */
+  private void killApplication(String applicationId) throws YarnRemoteException {
+    ApplicationId appId = ConverterUtils.toApplicationId(applicationId);
+    sysout.println("Killing application " + applicationId);
+    client.killApplication(appId);
+  }
+
+  /**
+   * Prints the application report for an application id.
+   * 
+   * @param applicationId
+   * @throws YarnRemoteException
+   */
+  private void printApplicationReport(String applicationId)
+      throws YarnRemoteException {
+    ApplicationReport appReport = client.getApplicationReport(ConverterUtils
+        .toApplicationId(applicationId));
+    StringBuffer appReportStr = new StringBuffer();
+    if (appReport != null) {
+      appReportStr.append("Application Report : ");
+      appReportStr.append("\n\tApplication-Id : ");
+      appReportStr.append(appReport.getApplicationId());
+      appReportStr.append("\n\tApplication-Name : ");
+      appReportStr.append(appReport.getName());
+      appReportStr.append("\n\tUser : ");
+      appReportStr.append(appReport.getUser());
+      appReportStr.append("\n\tQueue : ");
+      appReportStr.append(appReport.getQueue());
+      appReportStr.append("\n\tStart-Time : ");
+      appReportStr.append(appReport.getStartTime());
+      appReportStr.append("\n\tFinish-Time : ");
+      appReportStr.append(appReport.getFinishTime());
+      appReportStr.append("\n\tState : ");
+      appReportStr.append(appReport.getYarnApplicationState());
+      appReportStr.append("\n\tFinal-State : ");
+      appReportStr.append(appReport.getFinalApplicationStatus());
+      appReportStr.append("\n\tTracking-URL : ");
+      appReportStr.append(appReport.getOriginalTrackingUrl());
+      appReportStr.append("\n\tDiagnostics : ");
+      appReportStr.append(appReport.getDiagnostics());
+    } else {
+      appReportStr.append("Application with id '" + applicationId
+          + "' doesn't exist in RM.");
+    }
+    sysout.println(appReportStr.toString());
+  }
+
+}

+ 145 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeCLI.java

@@ -0,0 +1,145 @@
+/**
+ * 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.cli;
+
+import java.io.PrintWriter;
+import java.util.List;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Options;
+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.YarnRemoteException;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+
+public class NodeCLI extends YarnCLI {
+  private static final String NODES_PATTERN = "%16s\t%17s\t%26s\t%18s\n";
+  public static void main(String[] args) throws Exception {
+    NodeCLI cli = new NodeCLI();
+    cli.setSysOutPrintStream(System.out);
+    cli.setSysErrPrintStream(System.err);
+    int res = ToolRunner.run(cli, args);
+    cli.stop();
+    System.exit(res);
+  }
+
+  @Override
+  public int run(String[] args) throws Exception {
+
+    Options opts = new Options();
+    opts.addOption(STATUS_CMD, true, "Prints the status report of the node.");
+    opts.addOption(LIST_CMD, false, "Lists all the nodes.");
+    CommandLine cliParser = new GnuParser().parse(opts, args);
+
+    int exitCode = -1;
+    if (cliParser.hasOption("status")) {
+      if (args.length != 2) {
+        printUsage(opts);
+        return exitCode;
+      }
+      printNodeStatus(cliParser.getOptionValue("status"));
+    } else if (cliParser.hasOption("list")) {
+      listClusterNodes();
+    } else {
+      syserr.println("Invalid Command Usage : ");
+      printUsage(opts);
+    }
+    return 0;
+  }
+
+  /**
+   * It prints the usage of the command
+   * 
+   * @param opts
+   */
+  private void printUsage(Options opts) {
+    new HelpFormatter().printHelp("node", opts);
+  }
+
+  /**
+   * Lists all the nodes present in the cluster
+   * 
+   * @throws YarnRemoteException
+   */
+  private void listClusterNodes() throws YarnRemoteException {
+    PrintWriter writer = new PrintWriter(sysout);
+    List<NodeReport> nodesReport = client.getNodeReports();
+    writer.println("Total Nodes:" + nodesReport.size());
+    writer.printf(NODES_PATTERN, "Node-Id", "Node-Http-Address",
+        "Health-Status(isNodeHealthy)", "Running-Containers");
+    for (NodeReport nodeReport : nodesReport) {
+      writer.printf(NODES_PATTERN, nodeReport.getNodeId(),
+          nodeReport.getHttpAddress(), nodeReport
+          .getNodeHealthStatus().getIsNodeHealthy(), nodeReport
+          .getNumContainers());
+    }
+    writer.flush();
+  }
+
+  /**
+   * Prints the node report for node id.
+   * 
+   * @param nodeIdStr
+   * @throws YarnRemoteException
+   */
+  private void printNodeStatus(String nodeIdStr) throws YarnRemoteException {
+    NodeId nodeId = ConverterUtils.toNodeId(nodeIdStr);
+    List<NodeReport> nodesReport = client.getNodeReports();
+    StringBuffer nodeReportStr = new StringBuffer();
+    NodeReport nodeReport = null;
+    for (NodeReport report : nodesReport) {
+      if (!report.getNodeId().equals(nodeId)) {
+        continue;
+      }
+      nodeReport = report;
+      nodeReportStr.append("Node Report : ");
+      nodeReportStr.append("\n\tNode-Id : ");
+      nodeReportStr.append(nodeReport.getNodeId());
+      nodeReportStr.append("\n\tRack : ");
+      nodeReportStr.append(nodeReport.getRackName());
+      nodeReportStr.append("\n\tNode-Http-Address : ");
+      nodeReportStr.append(nodeReport.getHttpAddress());
+      nodeReportStr.append("\n\tHealth-Status(isNodeHealthy) : ");
+      nodeReportStr.append(nodeReport.getNodeHealthStatus()
+          .getIsNodeHealthy());
+      nodeReportStr.append("\n\tLast-Last-Health-Update : ");
+      nodeReportStr.append(nodeReport.getNodeHealthStatus()
+          .getLastHealthReportTime());
+      nodeReportStr.append("\n\tHealth-Report : ");
+      nodeReportStr
+          .append(nodeReport.getNodeHealthStatus().getHealthReport());
+      nodeReportStr.append("\n\tContainers : ");
+      nodeReportStr.append(nodeReport.getNumContainers());
+      nodeReportStr.append("\n\tMemory-Used : ");
+      nodeReportStr.append((nodeReport.getUsed() == null) ? "0M"
+          : (nodeReport.getUsed().getMemory() + "M"));
+      nodeReportStr.append("\n\tMemory-Capacity : ");
+      nodeReportStr.append(nodeReport.getCapability().getMemory());
+    }
+
+    if (nodeReport == null) {
+      nodeReportStr.append("Could not find the node report for node id : "
+          + nodeIdStr);
+    }
+
+    sysout.println(nodeReportStr.toString());
+  }
+}

+ 63 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/YarnCLI.java

@@ -0,0 +1,63 @@
+/**
+ * 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.cli;
+
+import java.io.PrintStream;
+
+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.conf.YarnConfiguration;
+
+public abstract class YarnCLI extends Configured implements Tool {
+
+  public static final String STATUS_CMD = "status";
+  public static final String LIST_CMD = "list";
+  public static final String KILL_CMD = "kill";
+  protected PrintStream sysout;
+  protected PrintStream syserr;
+  protected YarnClient client;
+
+  public YarnCLI() {
+    super(new YarnConfiguration());
+    client = new YarnClientImpl();
+    client.init(getConf());
+    client.start();
+  }
+
+  public void setSysOutPrintStream(PrintStream sysout) {
+    this.sysout = sysout;
+  }
+
+  public void setSysErrPrintStream(PrintStream syserr) {
+    this.syserr = syserr;
+  }
+
+  public YarnClient getClient() {
+    return client;
+  }
+
+  public void setClient(YarnClient client) {
+    this.client = client;
+  }
+
+  public void stop() {
+    this.client.stop();
+  }
+}

+ 234 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/hadoop/yarn/client/YarnClient.java

@@ -0,0 +1,234 @@
+/**
+* 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.hadoop.yarn.client;
+
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.Text;
+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.DelegationToken;
+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.YarnClusterMetrics;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.service.Service;
+
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface YarnClient extends Service {
+
+  /**
+   * <p>
+   * Obtain a new {@link ApplicationId} for submitting new applications.
+   * </p>
+   * 
+   * <p>
+   * Returns a response which contains {@link ApplicationId} that can be used to
+   * submit a new application. See
+   * {@link #submitApplication(ApplicationSubmissionContext)}.
+   * </p>
+   * 
+   * <p>
+   * See {@link GetNewApplicationResponse} for other information that is
+   * returned.
+   * </p>
+   * 
+   * @return response containing the new <code>ApplicationId</code> to be used
+   *         to submit an application
+   * @throws YarnRemoteException
+   */
+  GetNewApplicationResponse getNewApplication() throws YarnRemoteException;
+
+  /**
+   * <p>
+   * Submit a new application to <code>YARN.</code>
+   * </p>
+   * 
+   * @param appContext
+   *          {@link ApplicationSubmissionContext} containing all the details
+   *          needed to submit a new application
+   * @return {@link ApplicationId} of the accepted application
+   * @throws YarnRemoteException
+   * @see #getNewApplication()
+   */
+  ApplicationId submitApplication(ApplicationSubmissionContext appContext)
+      throws YarnRemoteException;
+
+  /**
+   * <p>
+   * Kill an application identified by given ID.
+   * </p>
+   * 
+   * @param applicationId
+   *          {@link ApplicationId} of the application that needs to be killed
+   * @throws YarnRemoteException
+   *           in case of errors or if YARN rejects the request due to
+   *           access-control restrictions.
+   * @see #getQueueAclsInfo()
+   */
+  void killApplication(ApplicationId applicationId) throws YarnRemoteException;
+
+  /**
+   * <p>
+   * Get a report of the given Application.
+   * </p>
+   * 
+   * <p>
+   * In secure mode, <code>YARN</code> verifies access to the application, queue
+   * etc. before accepting the request.
+   * </p>
+   * 
+   * <p>
+   * If the user does not have <code>VIEW_APP</code> access then the following
+   * fields in the report will be set to stubbed values:
+   * <ul>
+   * <li>host - set to "N/A"</li>
+   * <li>RPC port - set to -1</li>
+   * <li>client token - set to "N/A"</li>
+   * <li>diagnostics - set to "N/A"</li>
+   * <li>tracking URL - set to "N/A"</li>
+   * <li>original tracking URL - set to "N/A"</li>
+   * <li>resource usage report - all values are -1</li>
+   * </ul>
+   * </p>
+   * 
+   * @param appId
+   *          {@link ApplicationId} of the application that needs a report
+   * @return application report
+   * @throws YarnRemoteException
+   */
+  ApplicationReport getApplicationReport(ApplicationId appId)
+      throws YarnRemoteException;
+
+  /**
+   * <p>
+   * Get a report (ApplicationReport) of all Applications in the cluster.
+   * </p>
+   * 
+   * <p>
+   * If the user does not have <code>VIEW_APP</code> access for an application
+   * then the corresponding report will be filtered as described in
+   * {@link #getApplicationReport(ApplicationId)}.
+   * </p>
+   * 
+   * @return a list of reports of all running applications
+   * @throws YarnRemoteException
+   */
+  List<ApplicationReport> getApplicationList() throws YarnRemoteException;
+
+  /**
+   * <p>
+   * Get metrics ({@link YarnClusterMetrics}) about the cluster.
+   * </p>
+   * 
+   * @return cluster metrics
+   * @throws YarnRemoteException
+   */
+  YarnClusterMetrics getYarnClusterMetrics() throws YarnRemoteException;
+
+  /**
+   * <p>
+   * Get a report of all nodes ({@link NodeReport}) in the cluster.
+   * </p>
+   * 
+   * @return A list of report of all nodes
+   * @throws YarnRemoteException
+   */
+  List<NodeReport> getNodeReports() throws YarnRemoteException;
+
+  /**
+   * <p>
+   * 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 DelegationToken}) that can be used to
+   *         talk to YARN
+   * @throws YarnRemoteException
+   */
+  DelegationToken getRMDelegationToken(Text renewer) throws YarnRemoteException;
+
+  /**
+   * <p>
+   * Get information ({@link QueueInfo}) about a given <em>queue</em>.
+   * </p>
+   * 
+   * @param queueName
+   *          Name of the queue whose information is needed
+   * @return queue information
+   * @throws YarnRemoteException
+   *           in case of errors or if YARN rejects the request due to
+   *           access-control restrictions.
+   */
+  QueueInfo getQueueInfo(String queueName) throws YarnRemoteException;
+
+  /**
+   * <p>
+   * Get information ({@link QueueInfo}) about all queues, recursively if there
+   * is a hierarchy
+   * </p>
+   * 
+   * @return a list of queue-information for all queues
+   * @throws YarnRemoteException
+   */
+  List<QueueInfo> getAllQueues() throws YarnRemoteException;
+
+  /**
+   * <p>
+   * Get information ({@link QueueInfo}) about top level queues.
+   * </p>
+   * 
+   * @return a list of queue-information for all the top-level queues
+   * @throws YarnRemoteException
+   */
+  List<QueueInfo> getRootQueueInfos() throws YarnRemoteException;
+
+  /**
+   * <p>
+   * Get information ({@link QueueInfo}) about all the immediate children queues
+   * of the given queue
+   * </p>
+   * 
+   * @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 YarnRemoteException
+   */
+  List<QueueInfo> getChildQueueInfos(String parent) throws YarnRemoteException;
+
+  /**
+   * <p>
+   * Get information about <em>acls</em> for <em>current user</em> on all the
+   * existing queues.
+   * </p>
+   * 
+   * @return a list of queue acls ({@link QueueUserACLInfo}) for
+   *         <em>current user</em>
+   * @throws YarnRemoteException
+   */
+  List<QueueUserACLInfo> getQueueAclsInfo() throws YarnRemoteException;
+}

+ 258 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/hadoop/yarn/client/YarnClientImpl.java

@@ -0,0 +1,258 @@
+/**
+* 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.hadoop.yarn.client;
+
+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.ClientRMProtocol;
+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.DelegationToken;
+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.YarnClusterMetrics;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.service.AbstractService;
+import org.apache.hadoop.yarn.util.Records;
+
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class YarnClientImpl extends AbstractService implements YarnClient {
+
+  private static final Log LOG = LogFactory.getLog(YarnClientImpl.class);
+
+  protected ClientRMProtocol rmClient;
+  protected InetSocketAddress rmAddress;
+
+  private static final String ROOT = "root";
+
+  public YarnClientImpl() {
+    super(YarnClientImpl.class.getName());
+  }
+
+  private static InetSocketAddress getRmAddress(Configuration conf) {
+    return conf.getSocketAddr(YarnConfiguration.RM_ADDRESS,
+      YarnConfiguration.DEFAULT_RM_ADDRESS, YarnConfiguration.DEFAULT_RM_PORT);
+  }
+
+  @Override
+  public synchronized void init(Configuration conf) {
+    this.rmAddress = getRmAddress(conf);
+    super.init(conf);
+  }
+
+  @Override
+  public synchronized void start() {
+    YarnRPC rpc = YarnRPC.create(getConfig());
+
+    this.rmClient =
+        (ClientRMProtocol) rpc.getProxy(ClientRMProtocol.class, rmAddress,
+          getConfig());
+    LOG.debug("Connecting to ResourceManager at " + rmAddress);
+    super.start();
+  }
+
+  @Override
+  public synchronized void stop() {
+    RPC.stopProxy(this.rmClient);
+    super.stop();
+  }
+
+  @Override
+  public GetNewApplicationResponse getNewApplication()
+      throws YarnRemoteException {
+    GetNewApplicationRequest request =
+        Records.newRecord(GetNewApplicationRequest.class);
+    return rmClient.getNewApplication(request);
+  }
+
+  @Override
+  public ApplicationId
+      submitApplication(ApplicationSubmissionContext appContext)
+          throws YarnRemoteException {
+    ApplicationId applicationId = appContext.getApplicationId();
+    appContext.setApplicationId(applicationId);
+    SubmitApplicationRequest request =
+        Records.newRecord(SubmitApplicationRequest.class);
+    request.setApplicationSubmissionContext(appContext);
+    rmClient.submitApplication(request);
+    LOG.info("Submitted application " + applicationId + " to ResourceManager"
+        + " at " + rmAddress);
+    return applicationId;
+  }
+
+  @Override
+  public void killApplication(ApplicationId applicationId)
+      throws YarnRemoteException {
+    LOG.info("Killing application " + applicationId);
+    KillApplicationRequest request =
+        Records.newRecord(KillApplicationRequest.class);
+    request.setApplicationId(applicationId);
+    rmClient.forceKillApplication(request);
+  }
+
+  @Override
+  public ApplicationReport getApplicationReport(ApplicationId appId)
+      throws YarnRemoteException {
+    GetApplicationReportRequest request =
+        Records.newRecord(GetApplicationReportRequest.class);
+    request.setApplicationId(appId);
+    GetApplicationReportResponse response =
+        rmClient.getApplicationReport(request);
+    return response.getApplicationReport();
+  }
+
+  @Override
+  public List<ApplicationReport> getApplicationList()
+      throws YarnRemoteException {
+    GetAllApplicationsRequest request =
+        Records.newRecord(GetAllApplicationsRequest.class);
+    GetAllApplicationsResponse response = rmClient.getAllApplications(request);
+    return response.getApplicationList();
+  }
+
+  @Override
+  public YarnClusterMetrics getYarnClusterMetrics() throws YarnRemoteException {
+    GetClusterMetricsRequest request =
+        Records.newRecord(GetClusterMetricsRequest.class);
+    GetClusterMetricsResponse response = rmClient.getClusterMetrics(request);
+    return response.getClusterMetrics();
+  }
+
+  @Override
+  public List<NodeReport> getNodeReports() throws YarnRemoteException {
+    GetClusterNodesRequest request =
+        Records.newRecord(GetClusterNodesRequest.class);
+    GetClusterNodesResponse response = rmClient.getClusterNodes(request);
+    return response.getNodeReports();
+  }
+
+  @Override
+  public DelegationToken getRMDelegationToken(Text renewer)
+      throws YarnRemoteException {
+    /* 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 YarnRemoteException {
+    GetQueueInfoRequest request =
+        getQueueInfoRequest(queueName, true, false, false);
+    Records.newRecord(GetQueueInfoRequest.class);
+    return rmClient.getQueueInfo(request).getQueueInfo();
+  }
+
+  @Override
+  public List<QueueUserACLInfo> getQueueAclsInfo() throws YarnRemoteException {
+    GetQueueUserAclsInfoRequest request =
+        Records.newRecord(GetQueueUserAclsInfoRequest.class);
+    return rmClient.getQueueUserAcls(request).getUserAclsInfoList();
+  }
+
+  @Override
+  public List<QueueInfo> getAllQueues() throws YarnRemoteException {
+    List<QueueInfo> queues = new ArrayList<QueueInfo>();
+
+    QueueInfo rootQueue =
+        rmClient.getQueueInfo(getQueueInfoRequest(ROOT, false, true, true))
+          .getQueueInfo();
+    getChildQueues(rootQueue, queues, true);
+    return queues;
+  }
+
+  @Override
+  public List<QueueInfo> getRootQueueInfos() throws YarnRemoteException {
+    List<QueueInfo> queues = new ArrayList<QueueInfo>();
+
+    QueueInfo rootQueue =
+        rmClient.getQueueInfo(getQueueInfoRequest(ROOT, false, true, true))
+          .getQueueInfo();
+    getChildQueues(rootQueue, queues, false);
+    return queues;
+  }
+
+  @Override
+  public List<QueueInfo> getChildQueueInfos(String parent)
+      throws YarnRemoteException {
+    List<QueueInfo> queues = new ArrayList<QueueInfo>();
+
+    QueueInfo parentQueue =
+        rmClient.getQueueInfo(getQueueInfoRequest(parent, false, true, false))
+          .getQueueInfo();
+    getChildQueues(parentQueue, queues, true);
+    return queues;
+  }
+
+  private void getChildQueues(QueueInfo parent, List<QueueInfo> queues,
+      boolean recursive) {
+    List<QueueInfo> childQueues = parent.getChildQueues();
+
+    for (QueueInfo child : childQueues) {
+      queues.add(child);
+      if (recursive) {
+        getChildQueues(child, queues, recursive);
+      }
+    }
+  }
+}

+ 47 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestYarnClient.java

@@ -0,0 +1,47 @@
+/**
+* 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.client.YarnClient;
+import org.apache.hadoop.yarn.client.YarnClientImpl;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.junit.Test;
+
+public class TestYarnClient {
+
+  @Test
+  public void test() {
+    // More to come later.
+  }
+
+  @Test
+  public void testClientStop() {
+    Configuration conf = new YarnConfiguration();
+    ResourceManager rm = new ResourceManager(null);
+    rm.init(conf);
+    rm.start();
+
+    YarnClient client = new YarnClientImpl();
+    client.init(conf);
+    client.start();
+    client.stop();
+  }
+}

+ 247 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java

@@ -0,0 +1,247 @@
+/**
+ * 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.cli;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyInt;
+import static org.mockito.Matchers.isA;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.io.ByteArrayOutputStream;
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.List;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.NodeHealthStatus;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+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.factories.RecordFactory;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.util.BuilderUtils;
+import org.apache.hadoop.yarn.util.Records;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestYarnCLI {
+
+ private static final RecordFactory recordFactory = RecordFactoryProvider
+      .getRecordFactory(null);
+
+  private YarnClient client = mock(YarnClient.class);
+  ByteArrayOutputStream sysOutStream;
+  private PrintStream sysOut;
+  ByteArrayOutputStream sysErrStream;
+  private PrintStream sysErr;
+
+  @Before
+  public void setup() {
+    sysOutStream = new ByteArrayOutputStream();
+    sysOut = spy(new PrintStream(sysOutStream));
+    sysErrStream = new ByteArrayOutputStream();
+    sysErr = spy(new PrintStream(sysErrStream));
+  }
+  
+  @Test
+  public void testGetApplicationReport() throws Exception {
+    ApplicationCLI cli = createAndGetAppCLI();
+    ApplicationId applicationId = BuilderUtils.newApplicationId(1234, 5);
+    ApplicationReport newApplicationReport = BuilderUtils.newApplicationReport(
+        applicationId, 
+        "user", "queue", "appname", "host", 124, null,
+        YarnApplicationState.FINISHED, "diagnostics", "url", 0, 0,
+        FinalApplicationStatus.SUCCEEDED, null, "N/A");
+    when(client.getApplicationReport(any(ApplicationId.class))).thenReturn(
+        newApplicationReport);
+    int result = cli.run(new String[] { "-status", applicationId.toString() });
+    assertEquals(0, result);
+    verify(client).getApplicationReport(applicationId);
+    String appReportStr = "Application Report : \n\t"
+        + "Application-Id : application_1234_0005\n\t"
+        + "Application-Name : appname\n\tUser : user\n\t"
+        + "Queue : queue\n\tStart-Time : 0\n\tFinish-Time : 0\n\t"
+        + "State : FINISHED\n\tFinal-State : SUCCEEDED\n\t"
+        + "Tracking-URL : N/A\n\tDiagnostics : diagnostics\n";
+    Assert.assertEquals(appReportStr, sysOutStream.toString());
+    verify(sysOut, times(1)).println(isA(String.class));
+  }
+
+  @Test
+  public void testGetAllApplications() throws Exception {
+    ApplicationCLI cli = createAndGetAppCLI();
+    ApplicationId applicationId = BuilderUtils.newApplicationId(1234, 5);
+    ApplicationReport newApplicationReport = BuilderUtils.newApplicationReport(
+        applicationId, 
+        "user", "queue", "appname", "host", 124, null,
+        YarnApplicationState.FINISHED, "diagnostics", "url", 0, 0,
+        FinalApplicationStatus.SUCCEEDED, null, "N/A");
+    List<ApplicationReport> applicationReports = new ArrayList<ApplicationReport>();
+    applicationReports.add(newApplicationReport);
+    when(client.getApplicationList()).thenReturn(applicationReports);
+    int result = cli.run(new String[] { "-list" });
+    assertEquals(0, result);
+    verify(client).getApplicationList();
+
+    StringBuffer appsReportStrBuf = new StringBuffer();
+    appsReportStrBuf.append("Total Applications:1\n");
+    appsReportStrBuf
+        .append("                Application-Id\t    Application-Name"
+            + "\t      User\t     Queue\t             State\t       "
+            + "Final-State\t                       Tracking-URL\n");
+    appsReportStrBuf.append("         application_1234_0005\t             "
+        + "appname\t      user\t     queue\t          FINISHED\t         "
+        + "SUCCEEDED\t                                N/A\n");
+    Assert.assertEquals(appsReportStrBuf.toString(), sysOutStream.toString());
+    verify(sysOut, times(1)).write(any(byte[].class), anyInt(), anyInt());
+  }
+
+  @Test
+  public void testKillApplication() throws Exception {
+    ApplicationCLI cli = createAndGetAppCLI();
+    ApplicationId applicationId = BuilderUtils.newApplicationId(1234, 5);
+    int result = cli.run(new String[] { "-kill", applicationId.toString() });
+    assertEquals(0, result);
+    verify(client).killApplication(any(ApplicationId.class));
+    verify(sysOut).println("Killing application application_1234_0005");
+  }
+
+  @Test
+  public void testListClusterNodes() throws Exception {
+    NodeCLI cli = new NodeCLI();
+    when(client.getNodeReports()).thenReturn(getNodeReports(3));
+    cli.setClient(client);
+    cli.setSysOutPrintStream(sysOut);
+    int result = cli.run(new String[] { "-list" });
+    assertEquals(0, result);
+    verify(client).getNodeReports();
+    StringBuffer nodesReportStr = new StringBuffer();
+    nodesReportStr.append("Total Nodes:3");
+    nodesReportStr
+        .append("\n         Node-Id\tNode-Http-Address\t"
+            + "Health-Status(isNodeHealthy)\tRunning-Containers");
+    nodesReportStr.append("\n         host0:0\t       host1:8888"
+        + "\t                     false\t                 0");
+    nodesReportStr.append("\n         host1:0\t       host1:8888"
+        + "\t                     false\t                 0");
+    nodesReportStr.append("\n         host2:0\t       host1:8888"
+        + "\t                     false\t                 0\n");
+    Assert.assertEquals(nodesReportStr.toString(), sysOutStream.toString());
+    verify(sysOut, times(1)).write(any(byte[].class), anyInt(), anyInt());
+  }
+
+  @Test
+  public void testNodeStatus() throws Exception {
+    NodeId nodeId = BuilderUtils.newNodeId("host0", 0);
+    NodeCLI cli = new NodeCLI();
+    when(client.getNodeReports()).thenReturn(getNodeReports(3));
+    cli.setClient(client);
+    cli.setSysOutPrintStream(sysOut);
+    cli.setSysErrPrintStream(sysErr);
+    int result = cli.run(new String[] { "-status", nodeId.toString() });
+    assertEquals(0, result);
+    verify(client).getNodeReports();
+    String nodeStatusStr = "Node Report : \n\tNode-Id : host0:0\n\t"
+        + "Rack : rack1\n\t"
+        + "Node-Http-Address : host1:8888\n\tHealth-Status(isNodeHealthy) "
+        + ": false\n\tLast-Last-Health-Update : 0\n\tHealth-Report : null"
+        + "\n\tContainers : 0\n\tMemory-Used : 0M\n\tMemory-Capacity : 0";
+    verify(sysOut, times(1)).println(isA(String.class));
+    verify(sysOut).println(nodeStatusStr);
+  }
+
+  @Test
+  public void testAbsentNodeStatus() throws Exception {
+    NodeId nodeId = BuilderUtils.newNodeId("Absenthost0", 0);
+    NodeCLI cli = new NodeCLI();
+    when(client.getNodeReports()).thenReturn(getNodeReports(0));
+    cli.setClient(client);
+    cli.setSysOutPrintStream(sysOut);
+    cli.setSysErrPrintStream(sysErr);
+    int result = cli.run(new String[] { "-status", nodeId.toString() });
+    assertEquals(0, result);
+    verify(client).getNodeReports();
+    verify(sysOut, times(1)).println(isA(String.class));
+    verify(sysOut).println(
+      "Could not find the node report for node id : " + nodeId.toString());
+  }
+
+  @Test
+  public void testAppCLIUsageInfo() throws Exception {
+    verifyUsageInfo(new ApplicationCLI());
+  }
+
+  @Test
+  public void testNodeCLIUsageInfo() throws Exception {
+    verifyUsageInfo(new NodeCLI());
+  }
+
+  private void verifyUsageInfo(YarnCLI cli) throws Exception {
+    cli.setSysErrPrintStream(sysErr);
+    cli.run(new String[0]);
+    verify(sysErr).println("Invalid Command Usage : ");
+  }
+
+  private List<NodeReport> getNodeReports(int noOfNodes) {
+    List<NodeReport> nodeReports = new ArrayList<NodeReport>();
+ 	
+    for (int i = 0; i < noOfNodes; i++) {
+      NodeReport nodeReport = newNodeReport(BuilderUtils
+      .newNodeId("host" + i, 0), "host" + 1 + ":8888",
+      "rack1", Records.newRecord(Resource.class), Records
+      .newRecord(Resource.class), 0, Records
+      .newRecord(NodeHealthStatus.class));
+      nodeReports.add(nodeReport);
+    }
+    return nodeReports;
+  }
+
+  private static NodeReport newNodeReport(NodeId nodeId, 
+      String httpAddress, String rackName, Resource used, Resource capability,
+      int numContainers, NodeHealthStatus nodeHealthStatus) {
+    NodeReport nodeReport = recordFactory.newRecordInstance(NodeReport.class);
+    nodeReport.setNodeId(nodeId);
+    nodeReport.setHttpAddress(httpAddress);
+    nodeReport.setRackName(rackName);
+    nodeReport.setUsed(used);
+    nodeReport.setCapability(capability);
+    nodeReport.setNumContainers(numContainers);
+    nodeReport.setNodeHealthStatus(nodeHealthStatus);
+    return nodeReport;
+  }     
+
+
+  private ApplicationCLI createAndGetAppCLI() {
+    ApplicationCLI cli = new ApplicationCLI();
+    cli.setClient(client);
+    cli.setSysOutPrintStream(sysOut);
+    return cli;
+  }
+
+}

+ 30 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/hadoop/yarn/client/TestYarnClient.java

@@ -0,0 +1,30 @@
+/**
+* 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.hadoop.yarn.client;
+
+import org.junit.Test;
+
+public class TestYarnClient {
+
+  @Test
+  public void test() {
+    // More to come later.
+  }
+
+}

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/pom.xml

@@ -11,8 +11,7 @@
   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. See accompanying LICENSE file.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+--><project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
   <modelVersion>4.0.0</modelVersion>
   <parent>
     <groupId>org.apache.hadoop</groupId>
@@ -244,5 +243,6 @@
     <module>hadoop-yarn-server</module>
     <module>hadoop-yarn-applications</module>
     <module>hadoop-yarn-site</module>
+    <module>hadoop-yarn-client</module>
   </modules>
-</project>
+</project>