You are viewing a plain text version of this content. The canonical link for it is here.
Posted to mapreduce-commits@hadoop.apache.org by ac...@apache.org on 2011/12/29 09:06:05 UTC

svn commit: r1225463 [2/3] - in /hadoop/common/trunk/hadoop-mapreduce-project: ./ hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/ hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/...

Modified: hadoop/common/trunk/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java?rev=1225463&r1=1225462&r2=1225463&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java (original)
+++ hadoop/common/trunk/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java Thu Dec 29 08:06:04 2011
@@ -24,6 +24,7 @@ import javax.xml.bind.annotation.XmlAcce
 import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 
+import org.apache.hadoop.yarn.api.records.QueueState;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
 
 @XmlRootElement
@@ -43,14 +44,14 @@ public class CapacitySchedulerQueueInfo 
   protected float usedCapacity;
   protected float maxCapacity;
   protected String queueName;
-  protected String state;
+  protected QueueState state;
   protected ArrayList<CapacitySchedulerQueueInfo> subQueues;
 
   CapacitySchedulerQueueInfo() {
   };
 
   CapacitySchedulerQueueInfo(float cap, float used, float max, String name,
-      String state, String path) {
+      QueueState state, String path) {
     this.capacity = cap;
     this.usedCapacity = used;
     this.maxCapacity = max;
@@ -84,7 +85,7 @@ public class CapacitySchedulerQueueInfo 
   }
 
   public String getQueueState() {
-    return this.state;
+    return this.state.toString();
   }
 
   public String getQueuePath() {

Modified: hadoop/common/trunk/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ClusterInfo.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ClusterInfo.java?rev=1225463&r1=1225462&r2=1225463&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ClusterInfo.java (original)
+++ hadoop/common/trunk/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ClusterInfo.java Thu Dec 29 08:06:04 2011
@@ -23,6 +23,7 @@ import javax.xml.bind.annotation.XmlRoot
 
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.service.Service.STATE;
 import org.apache.hadoop.yarn.util.YarnVersionInfo;
 
 @XmlRootElement
@@ -31,7 +32,7 @@ public class ClusterInfo {
 
   protected long id;
   protected long startedOn;
-  protected String state;
+  protected STATE state;
   protected String resourceManagerVersion;
   protected String resourceManagerBuildVersion;
   protected String resourceManagerVersionBuiltOn;
@@ -46,7 +47,7 @@ public class ClusterInfo {
     long ts = ResourceManager.clusterTimeStamp;
 
     this.id = ts;
-    this.state = rm.getServiceState().toString();
+    this.state = rm.getServiceState();
     this.startedOn = ts;
     this.resourceManagerVersion = YarnVersionInfo.getVersion();
     this.resourceManagerBuildVersion = YarnVersionInfo.getBuildVersion();
@@ -57,7 +58,7 @@ public class ClusterInfo {
   }
 
   public String getState() {
-    return this.state;
+    return this.state.toString();
   }
 
   public String getRMVersion() {

Modified: hadoop/common/trunk/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java?rev=1225463&r1=1225462&r2=1225463&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java (original)
+++ hadoop/common/trunk/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java Thu Dec 29 08:06:04 2011
@@ -87,8 +87,12 @@ public class MockRM extends ResourceMana
         .newRecord(GetNewApplicationRequest.class));
   }
 
-  // client
   public RMApp submitApp(int masterMemory) throws Exception {
+    return submitApp(masterMemory, "", "");
+  }
+
+  // client
+  public RMApp submitApp(int masterMemory, String name, String user) throws Exception {
     ClientRMProtocol client = getClientRMService();
     GetNewApplicationResponse resp = client.getNewApplication(Records
         .newRecord(GetNewApplicationRequest.class));
@@ -99,8 +103,8 @@ public class MockRM extends ResourceMana
     ApplicationSubmissionContext sub = Records
         .newRecord(ApplicationSubmissionContext.class);
     sub.setApplicationId(appId);
-    sub.setApplicationName("");
-    sub.setUser("");
+    sub.setApplicationName(name);
+    sub.setUser(user);
     ContainerLaunchContext clc = Records
         .newRecord(ContainerLaunchContext.class);
     Resource capability = Records.newRecord(Resource.class);

Modified: hadoop/common/trunk/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java?rev=1225463&r1=1225462&r2=1225463&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java (original)
+++ hadoop/common/trunk/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java Thu Dec 29 08:06:04 2011
@@ -22,34 +22,34 @@ import static org.junit.Assert.assertEqu
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-import java.util.ArrayList;
+import java.io.StringReader;
 
 import javax.ws.rs.core.MediaType;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
-import org.apache.hadoop.yarn.api.records.NodeHealthStatus;
+import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.yarn.api.records.QueueState;
-import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
-import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
+import org.apache.hadoop.yarn.server.resourcemanager.ClusterMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
-import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
-import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
-import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeState;
-import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeStatusEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
+import org.apache.hadoop.yarn.service.Service.STATE;
+import org.apache.hadoop.yarn.util.YarnVersionInfo;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
-import org.codehaus.jettison.json.JSONArray;
+import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
 import org.junit.Before;
 import org.junit.Test;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.NodeList;
+import org.xml.sax.InputSource;
 
 import com.google.inject.Guice;
 import com.google.inject.Injector;
@@ -105,997 +105,439 @@ public class TestRMWebServices extends J
   }
 
   @Test
-  public void testCluster() throws JSONException, Exception {
-    WebResource r = resource();
-    JSONObject json = r.path("ws").path("v1").path("cluster")
-        .accept(MediaType.APPLICATION_JSON).get(JSONObject.class);
-    verifyClusterInfo(json);
-  }
-
-  @Test
-  public void testClusterSlash() throws JSONException, Exception {
-    WebResource r = resource();
-    // test with trailing "/" to make sure acts same as without slash
-    JSONObject json = r.path("ws").path("v1").path("cluster/")
-        .accept(MediaType.APPLICATION_JSON).get(JSONObject.class);
-    verifyClusterInfo(json);
-  }
-
-  @Test
-  public void testInfo() throws JSONException, Exception {
-    WebResource r = resource();
-    JSONObject json = r.path("ws").path("v1").path("cluster").path("info")
-        .accept(MediaType.APPLICATION_JSON).get(JSONObject.class);
-    verifyClusterInfo(json);
-  }
-
-  @Test
-  public void testInfoSlash() throws JSONException, Exception {
-    WebResource r = resource();
-    // test with trailing "/" to make sure acts same as without slash
-    JSONObject json = r.path("ws").path("v1").path("cluster").path("info/")
-        .accept(MediaType.APPLICATION_JSON).get(JSONObject.class);
-    verifyClusterInfo(json);
-  }
-
-  public void verifyClusterInfo(JSONObject json) throws JSONException,
-      Exception {
-    assertEquals("correct number of elements", 1, json.length());
-    JSONObject clusterinfo = json.getJSONObject("clusterInfo");
-    assertEquals("correct number of elements", 9, clusterinfo.length());
-    String clusterid = clusterinfo.get("id").toString();
-    assertTrue("clusterId doesn't match: " + clusterid, clusterid.toString()
-        .matches("^\\d+"));
-    String startedon = clusterinfo.get("startedOn").toString();
-    assertTrue("startedOn doesn't match: " + startedon,
-        startedon.matches("^\\d+"));
-    String state = clusterinfo.get("state").toString();
-    assertTrue("stated doesn't match: " + state, state.matches("INITED"));
-    String rmVersion = clusterinfo.get("resourceManagerVersion").toString();
-    assertTrue("rm version doesn't match: " + rmVersion,
-        rmVersion.matches(".*"));
-    String rmBuildVersion = clusterinfo.get("resourceManagerBuildVersion")
-        .toString();
-    assertTrue("rm Build version doesn't match: " + rmBuildVersion,
-        rmBuildVersion.matches(".*"));
-    String rmVersionBuiltOn = clusterinfo.get("resourceManagerVersionBuiltOn")
-        .toString();
-    assertTrue(
-        "rm version built on doesn't match: " + rmVersionBuiltOn,
-        rmVersionBuiltOn
-            .matches("^\\w+\\s+\\w+\\s+\\d+\\s+\\d\\d:\\d\\d:\\d\\d\\s+\\w+\\s+\\d\\d\\d\\d"));
-    String hadoopVersion = clusterinfo.get("hadoopVersion").toString();
-    assertTrue("hadoop version doesn't match: " + hadoopVersion,
-        hadoopVersion.matches(".*"));
-    String hadoopBuildVersion = clusterinfo.get("hadoopBuildVersion")
-        .toString();
-    assertTrue("hadoop Build version doesn't match: " + hadoopBuildVersion,
-        hadoopBuildVersion.matches(".*"));
-    String hadoopVersionBuiltOn = clusterinfo.get("hadoopVersionBuiltOn")
-        .toString();
-    assertTrue(
-        "hadoop version built on doesn't match: " + hadoopVersionBuiltOn,
-        hadoopVersionBuiltOn
-            .matches("^\\w+\\s+\\w+\\s+\\d+\\s+\\d\\d:\\d\\d:\\d\\d\\s+\\w+\\s+\\d\\d\\d\\d"));
-  }
-
-  @Test
-  public void testClusterMetrics() throws JSONException, Exception {
-    WebResource r = resource();
-    JSONObject json = r.path("ws").path("v1").path("cluster").path("metrics")
-        .accept(MediaType.APPLICATION_JSON).get(JSONObject.class);
-    verifyClusterMetrics(json);
-  }
-
-  @Test
-  public void testClusterMetricsSlash() throws JSONException, Exception {
-    WebResource r = resource();
-    JSONObject json = r.path("ws").path("v1").path("cluster").path("metrics/")
-        .accept(MediaType.APPLICATION_JSON).get(JSONObject.class);
-    verifyClusterMetrics(json);
-  }
-
-  public void verifyClusterMetrics(JSONObject json) throws JSONException,
-      Exception {
-    assertEquals("correct number of elements", 1, json.length());
-    JSONObject clusterinfo = json.getJSONObject("clusterMetrics");
-    assertEquals("correct number of elements", 11, clusterinfo.length());
-    assertEquals("appsSubmitted doesn't match", 0,
-        clusterinfo.getInt("appsSubmitted"));
-    assertEquals("reservedMB doesn't match", 0,
-        clusterinfo.getInt("reservedMB"));
-    assertEquals("availableMB doesn't match", 0,
-        clusterinfo.getInt("availableMB"));
-    assertEquals("allocatedMB doesn't match", 0,
-        clusterinfo.getInt("allocatedMB"));
-    assertEquals("containersAllocated doesn't match", 0,
-        clusterinfo.getInt("containersAllocated"));
-    assertEquals("totalMB doesn't match", 0, clusterinfo.getInt("totalMB"));
-    assertEquals("totalNodes doesn't match", 0,
-        clusterinfo.getInt("totalNodes"));
-    assertEquals("lostNodes doesn't match", 0, clusterinfo.getInt("lostNodes"));
-    assertEquals("unhealthyNodes doesn't match", 0,
-        clusterinfo.getInt("unhealthyNodes"));
-    assertEquals("decommissionedNodes doesn't match", 0,
-        clusterinfo.getInt("decommissionedNodes"));
-    assertEquals("rebootedNodes doesn't match", 0,
-        clusterinfo.getInt("rebootedNodes"));
-  }
-
-  @Test
-  public void testClusterSchedulerFifo() throws JSONException, Exception {
-    WebResource r = resource();
-    JSONObject json = r.path("ws").path("v1").path("cluster").path("scheduler")
-        .accept(MediaType.APPLICATION_JSON).get(JSONObject.class);
-    verifyClusterSchedulerFifo(json);
-  }
-
-  @Test
-  public void testClusterSchedulerFifoSlash() throws JSONException, Exception {
-    WebResource r = resource();
-    JSONObject json = r.path("ws").path("v1").path("cluster")
-        .path("scheduler/").accept(MediaType.APPLICATION_JSON)
-        .get(JSONObject.class);
-    verifyClusterSchedulerFifo(json);
-  }
-
-  public void verifyClusterSchedulerFifo(JSONObject json) throws JSONException,
-      Exception {
-    assertEquals("correct number of elements", 1, json.length());
-    JSONObject info = json.getJSONObject("scheduler");
-    assertEquals("correct number of elements", 1, info.length());
-    info = info.getJSONObject("schedulerInfo");
-    assertEquals("correct number of elements", 11, info.length());
-    assertEquals("type doesn't match", "fifoScheduler", info.getString("type"));
-    assertEquals("qstate doesn't match", QueueState.RUNNING.toString(),
-        info.getString("qstate"));
-    assertEquals("capacity doesn't match", 1.0, info.getDouble("capacity"), 0.0);
-    assertEquals("usedCapacity doesn't match", Float.NaN,
-        info.getDouble("usedCapacity"), 0.0);
-    assertEquals("minQueueMemoryCapacity doesn't match", 1024,
-        info.getInt("minQueueMemoryCapacity"));
-    assertEquals("maxQueueMemoryCapacity doesn't match", 10240,
-        info.getInt("maxQueueMemoryCapacity"));
-    assertEquals("maxQueueMemoryCapacity doesn't match", 10240,
-        info.getInt("maxQueueMemoryCapacity"));
-
-  }
-
-  @Test
-  public void testNodes() throws JSONException, Exception {
-    testNodesHelper("nodes");
-  }
-
-  @Test
-  public void testNodesSlash() throws JSONException, Exception {
-    testNodesHelper("nodes/");
-  }
-
-  @Test
-  public void testNodesQueryState() throws JSONException, Exception {
+  public void testInfoXML() throws JSONException, Exception {
     WebResource r = resource();
-    MockNM nm1 = rm.registerNode("h1:1234", 5120);
-    MockNM nm2 = rm.registerNode("h2:1235", 5121);
-    rm.sendNodeStarted(nm1);
-    rm.NMwaitForState(nm1.getNodeId(), RMNodeState.RUNNING);
-    rm.NMwaitForState(nm2.getNodeId(), RMNodeState.NEW);
-
-    JSONObject json = r.path("ws").path("v1").path("cluster").path("nodes")
-        .queryParam("state", RMNodeState.RUNNING.toString())
-        .accept("application/json").get(JSONObject.class);
-
-    assertEquals("correct number of elements", 1, json.length());
-    JSONObject nodes = json.getJSONObject("nodes");
-    assertEquals("correct number of elements", 1, nodes.length());
-    JSONArray nodeArray = nodes.getJSONArray("node");
-    assertEquals("correct number of elements", 1, nodeArray.length());
-    JSONObject info = nodeArray.getJSONObject(0);
-
-    verifyNodeInfo(info, nm1, RMNodeState.RUNNING);
+    ClientResponse response = r.path("ws").path("v1").path("cluster")
+        .path("info").accept("application/xml").get(ClientResponse.class);
+    assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+    String xml = response.getEntity(String.class);
+    verifyClusterInfoXML(xml);
   }
 
   @Test
-  public void testNodesQueryStateNone() throws JSONException, Exception {
+  public void testInvalidUri() throws JSONException, Exception {
     WebResource r = resource();
-    rm.registerNode("h1:1234", 5120);
-    rm.registerNode("h2:1235", 5121);
-
-    JSONObject json = r.path("ws").path("v1").path("cluster").path("nodes")
-        .queryParam("state", RMNodeState.DECOMMISSIONED.toString())
-        .accept("application/json").get(JSONObject.class);
-    assertEquals("correct number of elements", 1, json.length());
-    assertEquals("nodes is not null", JSONObject.NULL, json.get("nodes"));
-  }
-
-  @Test
-  public void testNodesQueryStateInvalid() throws JSONException, Exception {
-    WebResource r = resource();
-    rm.registerNode("h1:1234", 5120);
-    rm.registerNode("h2:1235", 5121);
-
+    String responseStr = "";
     try {
-      r.path("ws").path("v1").path("cluster").path("nodes")
-          .queryParam("state", "BOGUSSTATE").accept("application/json")
-          .get(JSONObject.class);
-
-      fail("should have thrown exception querying invalid state");
+      responseStr = r.path("ws").path("v1").path("cluster").path("bogus")
+          .accept(MediaType.APPLICATION_JSON).get(String.class);
+      fail("should have thrown exception on invalid uri");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
+      assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
 
-      assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
-      JSONObject msg = response.getEntity(JSONObject.class);
-      JSONObject exception = msg.getJSONObject("RemoteException");
-      assertEquals("correct number of elements", 3, exception.length());
-      String message = exception.getString("message");
-      String type = exception.getString("exception");
-      String classname = exception.getString("javaClassName");
-      checkStringMatch(
-          "exception message",
-          "No enum const class org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeState.BOGUSSTATE",
-          message);
-      checkStringMatch("exception type", "IllegalArgumentException", type);
-      checkStringMatch("exception classname",
-          "java.lang.IllegalArgumentException", classname);
-
-    } finally {
-      rm.stop();
+      WebServicesTestUtils.checkStringMatch(
+          "error string exists and shouldn't", "", responseStr);
     }
   }
 
   @Test
-  public void testNodesQueryHealthy() throws JSONException, Exception {
-    WebResource r = resource();
-    MockNM nm1 = rm.registerNode("h1:1234", 5120);
-    MockNM nm2 = rm.registerNode("h2:1235", 5121);
-    rm.sendNodeStarted(nm1);
-    rm.NMwaitForState(nm1.getNodeId(), RMNodeState.RUNNING);
-    rm.NMwaitForState(nm2.getNodeId(), RMNodeState.NEW);
-    JSONObject json = r.path("ws").path("v1").path("cluster").path("nodes")
-        .queryParam("healthy", "true").accept("application/json")
-        .get(JSONObject.class);
-
-    assertEquals("correct number of elements", 1, json.length());
-    JSONObject nodes = json.getJSONObject("nodes");
-    assertEquals("correct number of elements", 1, nodes.length());
-    JSONArray nodeArray = nodes.getJSONArray("node");
-    assertEquals("correct number of elements", 2, nodeArray.length());
-  }
-
-  @Test
-  public void testNodesQueryHealthyCase() throws JSONException, Exception {
-    WebResource r = resource();
-    MockNM nm1 = rm.registerNode("h1:1234", 5120);
-    MockNM nm2 = rm.registerNode("h2:1235", 5121);
-    rm.sendNodeStarted(nm1);
-    rm.NMwaitForState(nm1.getNodeId(), RMNodeState.RUNNING);
-    rm.NMwaitForState(nm2.getNodeId(), RMNodeState.NEW);
-    JSONObject json = r.path("ws").path("v1").path("cluster").path("nodes")
-        .queryParam("healthy", "TRUe").accept("application/json")
-        .get(JSONObject.class);
-
-    assertEquals("correct number of elements", 1, json.length());
-    JSONObject nodes = json.getJSONObject("nodes");
-    assertEquals("correct number of elements", 1, nodes.length());
-    JSONArray nodeArray = nodes.getJSONArray("node");
-    assertEquals("correct number of elements", 2, nodeArray.length());
-
-  }
-
-  @Test
-  public void testNodesQueryHealthyAndState() throws JSONException, Exception {
-    WebResource r = resource();
-    MockNM nm1 = rm.registerNode("h1:1234", 5120);
-    MockNM nm2 = rm.registerNode("h2:1235", 5121);
-    rm.sendNodeStarted(nm1);
-    rm.NMwaitForState(nm2.getNodeId(), RMNodeState.NEW);
-    rm.NMwaitForState(nm1.getNodeId(), RMNodeState.RUNNING);
-    RMNodeImpl node = (RMNodeImpl) rm.getRMContext().getRMNodes()
-        .get(nm1.getNodeId());
-    NodeHealthStatus nodeHealth = node.getNodeHealthStatus();
-    nodeHealth.setHealthReport("test health report");
-    nodeHealth.setIsNodeHealthy(false);
-    node.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeHealth,
-        new ArrayList<ContainerStatus>(), null, null));
-    rm.NMwaitForState(nm1.getNodeId(), RMNodeState.UNHEALTHY);
-
-    JSONObject json = r.path("ws").path("v1").path("cluster").path("nodes")
-        .queryParam("healthy", "true")
-        .queryParam("state", RMNodeState.RUNNING.toString())
-        .accept("application/json").get(JSONObject.class);
-    assertEquals("correct number of elements", 1, json.length());
-    assertEquals("nodes is not null", JSONObject.NULL, json.get("nodes"));
-  }
-
-  @Test
-  public void testNodesQueryHealthyFalse() throws JSONException, Exception {
+  public void testInvalidUri2() throws JSONException, Exception {
     WebResource r = resource();
-    MockNM nm1 = rm.registerNode("h1:1234", 5120);
-    MockNM nm2 = rm.registerNode("h2:1235", 5121);
-    rm.sendNodeStarted(nm1);
-    rm.NMwaitForState(nm1.getNodeId(), RMNodeState.RUNNING);
-    rm.NMwaitForState(nm2.getNodeId(), RMNodeState.NEW);
-    JSONObject json = r.path("ws").path("v1").path("cluster").path("nodes")
-        .queryParam("healthy", "false").accept("application/json")
-        .get(JSONObject.class);
-
-    assertEquals("correct number of elements", 1, json.length());
-    assertEquals("nodes is not null", JSONObject.NULL, json.get("nodes"));
-  }
-
-  @Test
-  public void testNodesQueryHealthyInvalid() throws JSONException, Exception {
-    WebResource r = resource();
-    rm.registerNode("h1:1234", 5120);
-    rm.registerNode("h2:1235", 5121);
-
+    String responseStr = "";
     try {
-      r.path("ws").path("v1").path("cluster").path("nodes")
-          .queryParam("healthy", "tr").accept("application/json")
-          .get(JSONObject.class);
-      fail("should have thrown exception querying invalid healthy string");
+      responseStr = r.accept(MediaType.APPLICATION_JSON).get(String.class);
+      fail("should have thrown exception on invalid uri");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
-      JSONObject msg = response.getEntity(JSONObject.class);
-      JSONObject exception = msg.getJSONObject("RemoteException");
-      assertEquals("correct number of elements", 3, exception.length());
-      String message = exception.getString("message");
-      String type = exception.getString("exception");
-      String classname = exception.getString("javaClassName");
-      checkStringMatch(
-          "exception message",
-          "java.lang.Exception: Error: You must specify either true or false to query on health",
-          message);
-      checkStringMatch("exception type", "BadRequestException", type);
-      checkStringMatch("exception classname",
-          "org.apache.hadoop.yarn.webapp.BadRequestException", classname);
-
-    } finally {
-      rm.stop();
-    }
-  }
-
-  public void testNodesHelper(String path) throws JSONException, Exception {
-    WebResource r = resource();
-    MockNM nm1 = rm.registerNode("h1:1234", 5120);
-    MockNM nm2 = rm.registerNode("h2:1235", 5121);
-    JSONObject json = r.path("ws").path("v1").path("cluster").path(path)
-        .accept("application/json").get(JSONObject.class);
-    assertEquals("correct number of elements", 1, json.length());
-    JSONObject nodes = json.getJSONObject("nodes");
-    assertEquals("correct number of elements", 1, nodes.length());
-    JSONArray nodeArray = nodes.getJSONArray("node");
-    assertEquals("correct number of elements", 2, nodeArray.length());
-    JSONObject info = nodeArray.getJSONObject(0);
-    String id = info.get("id").toString();
-
-    if (id.matches("h1:1234")) {
-      verifyNodeInfo(info, nm1, RMNodeState.NEW);
-      verifyNodeInfo(nodeArray.getJSONObject(1), nm2, RMNodeState.NEW);
-    } else {
-      verifyNodeInfo(info, nm2, RMNodeState.NEW);
-      verifyNodeInfo(nodeArray.getJSONObject(1), nm1, RMNodeState.NEW);
+      assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
+      WebServicesTestUtils.checkStringMatch(
+          "error string exists and shouldn't", "", responseStr);
     }
   }
 
   @Test
-  public void testSingleNode() throws JSONException, Exception {
-    rm.registerNode("h1:1234", 5120);
-    MockNM nm2 = rm.registerNode("h2:1235", 5121);
-    testSingleNodeHelper("h2:1235", nm2);
-  }
-
-  @Test
-  public void testSingleNodeSlash() throws JSONException, Exception {
-    MockNM nm1 = rm.registerNode("h1:1234", 5120);
-    rm.registerNode("h2:1235", 5121);
-    testSingleNodeHelper("h1:1234/", nm1);
-  }
-
-  public void testSingleNodeHelper(String nodeid, MockNM nm)
-      throws JSONException, Exception {
-    WebResource r = resource();
-    JSONObject json = r.path("ws").path("v1").path("cluster").path("nodes")
-        .path(nodeid).accept("application/json").get(JSONObject.class);
-    assertEquals("correct number of elements", 1, json.length());
-    JSONObject info = json.getJSONObject("node");
-    verifyNodeInfo(info, nm, RMNodeState.NEW);
-  }
-
-  @Test
-  public void testNonexistNode() throws JSONException, Exception {
-    rm.registerNode("h1:1234", 5120);
-    rm.registerNode("h2:1235", 5121);
+  public void testInvalidAccept() throws JSONException, Exception {
     WebResource r = resource();
+    String responseStr = "";
     try {
-      r.path("ws").path("v1").path("cluster").path("nodes")
-          .path("node_invalid:99").accept("application/json")
-          .get(JSONObject.class);
-
-      fail("should have thrown exception on non-existent nodeid");
+      responseStr = r.path("ws").path("v1").path("cluster")
+          .accept(MediaType.TEXT_PLAIN).get(String.class);
+      fail("should have thrown exception on invalid uri");
     } catch (UniformInterfaceException ue) {
       ClientResponse response = ue.getResponse();
-      assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
-
-      JSONObject msg = response.getEntity(JSONObject.class);
-      JSONObject exception = msg.getJSONObject("RemoteException");
-      assertEquals("correct number of elements", 3, exception.length());
-      String message = exception.getString("message");
-      String type = exception.getString("exception");
-      String classname = exception.getString("javaClassName");
-      checkStringMatch("exception message",
-          "java.lang.Exception: nodeId, node_invalid:99, is not found", message);
-      checkStringMatch("exception type", "NotFoundException", type);
-      checkStringMatch("exception classname",
-          "org.apache.hadoop.yarn.webapp.NotFoundException", classname);
-
-    } finally {
-      rm.stop();
+      assertEquals(Status.INTERNAL_SERVER_ERROR,
+          response.getClientResponseStatus());
+      WebServicesTestUtils.checkStringMatch(
+          "error string exists and shouldn't", "", responseStr);
     }
   }
 
   @Test
-  public void testInvalidNode() throws JSONException, Exception {
-    rm.registerNode("h1:1234", 5120);
-    rm.registerNode("h2:1235", 5121);
-
+  public void testCluster() throws JSONException, Exception {
     WebResource r = resource();
-    try {
-      r.path("ws").path("v1").path("cluster").path("nodes")
-          .path("node_invalid_foo").accept("application/json")
-          .get(JSONObject.class);
+    ClientResponse response = r.path("ws").path("v1").path("cluster")
+        .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
 
-      fail("should have thrown exception on non-existent nodeid");
-    } catch (UniformInterfaceException ue) {
-      ClientResponse response = ue.getResponse();
-
-      assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
-      JSONObject msg = response.getEntity(JSONObject.class);
-      JSONObject exception = msg.getJSONObject("RemoteException");
-      assertEquals("correct number of elements", 3, exception.length());
-      String message = exception.getString("message");
-      String type = exception.getString("exception");
-      String classname = exception.getString("javaClassName");
-      checkStringMatch("exception message",
-          "Invalid NodeId \\[node_invalid_foo\\]. Expected host:port", message);
-      checkStringMatch("exception type", "IllegalArgumentException", type);
-      checkStringMatch("exception classname",
-          "java.lang.IllegalArgumentException", classname);
-    } finally {
-      rm.stop();
-    }
-  }
-
-  public void verifyNodeInfo(JSONObject nodeInfo, MockNM nm,
-      RMNodeState expectedState) throws JSONException, Exception {
-    assertEquals("correct number of elements", 11, nodeInfo.length());
-    String state = nodeInfo.get("state").toString();
-    assertTrue("stated doesn't match: " + state,
-        state.matches(expectedState.toString()));
-    String rack = nodeInfo.get("rack").toString();
-    assertTrue("rack doesn't match: " + rack, rack.matches("/default-rack"));
-    String healthStatus = nodeInfo.get("healthStatus").toString();
-    assertTrue("healthStatus doesn't match: " + healthStatus,
-        healthStatus.matches("Healthy"));
-    String id = nodeInfo.get("id").toString();
-    assertTrue("id doesn't match, got: " + id + " expected: "
-        + nm.getNodeId().toString(), id.matches(nm.getNodeId().toString()));
-    String nodeHostName = nodeInfo.get("nodeHostName").toString();
-    assertTrue("hostname doesn't match, got: " + nodeHostName + " expected: "
-        + nm.getNodeId().getHost(),
-        nodeHostName.matches(nm.getNodeId().getHost()));
-
-    String nodeHTTPAddress = nodeInfo.get("nodeHTTPAddress").toString();
-    String expectedHttpAddress = nm.getNodeId().getHost() + ":"
-        + nm.getHttpPort();
-    assertTrue("nodeHTTPAddress doesn't match, got: " + nodeHTTPAddress
-        + " expected: " + expectedHttpAddress,
-        nodeHTTPAddress.matches(expectedHttpAddress));
-    // could use this for other checks
-    RMNode node = rm.getRMContext().getRMNodes().get(nm.getNodeId());
-    long lastHealthUpdate = nodeInfo.getLong("lastHealthUpdate");
-    long expectedHealthUpdate = node.getNodeHealthStatus()
-        .getLastHealthReportTime();
-    assertEquals("lastHealthUpdate doesn't match, got: " + lastHealthUpdate
-        + " expected: " + expectedHealthUpdate, expectedHealthUpdate,
-        lastHealthUpdate);
-    String healthReport = nodeInfo.get("healthReport").toString();
-    assertTrue("healthReport doesn't match: " + healthReport,
-        healthReport.matches("Healthy"));
-
-    int numContainers = nodeInfo.getInt("numContainers");
-    assertEquals("numContainers doesn't match: " + numContainers, 0,
-        numContainers);
-
-    long usedMemoryMB = nodeInfo.getLong("usedMemoryMB");
-    assertEquals("usedMemoryMB doesn't match: " + usedMemoryMB, 0, usedMemoryMB);
-
-    long availMemoryMB = nodeInfo.getLong("availMemoryMB");
-    assertEquals("availMemoryMB doesn't match: " + availMemoryMB, 0,
-        availMemoryMB);
+    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    JSONObject json = response.getEntity(JSONObject.class);
+    verifyClusterInfo(json);
   }
 
   @Test
-  public void testApps() throws JSONException, Exception {
-    rm.start();
-    MockNM amNodeManager = rm.registerNode("amNM:1234", 2048);
-    RMApp app1 = rm.submitApp(1024);
-    amNodeManager.nodeHeartbeat(true);
-    testAppsHelper("apps", app1);
-    rm.stop();
+  public void testClusterSlash() throws JSONException, Exception {
+    WebResource r = resource();
+    // test with trailing "/" to make sure acts same as without slash
+    ClientResponse response = r.path("ws").path("v1").path("cluster/")
+        .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
 
+    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    JSONObject json = response.getEntity(JSONObject.class);
+    verifyClusterInfo(json);
   }
 
   @Test
-  public void testAppsSlash() throws JSONException, Exception {
-    rm.start();
-    MockNM amNodeManager = rm.registerNode("amNM:1234", 2048);
-    RMApp app1 = rm.submitApp(1024);
-    amNodeManager.nodeHeartbeat(true);
-    testAppsHelper("apps/", app1);
-    rm.stop();
-
-  }
-
-  public void testAppsHelper(String path, RMApp app) throws JSONException,
-      Exception {
+  public void testClusterDefault() throws JSONException, Exception {
     WebResource r = resource();
-    JSONObject json = r.path("ws").path("v1").path("cluster").path(path)
-        .accept("application/json").get(JSONObject.class);
-    assertEquals("correct number of elements", 1, json.length());
-    JSONObject apps = json.getJSONObject("apps");
-    assertEquals("correct number of elements", 1, apps.length());
-    JSONArray array = apps.getJSONArray("app");
-    assertEquals("correct number of elements", 1, array.length());
-    verifyAppInfo(array.getJSONObject(0), app);
+    // test with trailing "/" to make sure acts same as without slash
+    ClientResponse response = r.path("ws").path("v1").path("cluster")
+        .get(ClientResponse.class);
 
+    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    JSONObject json = response.getEntity(JSONObject.class);
+    verifyClusterInfo(json);
   }
 
   @Test
-  public void testAppsQueryState() throws JSONException, Exception {
-    rm.start();
-    MockNM amNodeManager = rm.registerNode("amNM:1234", 2048);
-    RMApp app1 = rm.submitApp(1024);
-    amNodeManager.nodeHeartbeat(true);
+  public void testInfo() throws JSONException, Exception {
     WebResource r = resource();
-    JSONObject json = r.path("ws").path("v1").path("cluster").path("apps")
-        .queryParam("state", RMAppState.ACCEPTED.toString())
-        .accept("application/json").get(JSONObject.class);
-    assertEquals("correct number of elements", 1, json.length());
-    JSONObject apps = json.getJSONObject("apps");
-    assertEquals("correct number of elements", 1, apps.length());
-    JSONArray array = apps.getJSONArray("app");
-    assertEquals("correct number of elements", 1, array.length());
-    verifyAppInfo(array.getJSONObject(0), app1);
-    rm.stop();
+    ClientResponse response = r.path("ws").path("v1").path("cluster")
+        .path("info").accept(MediaType.APPLICATION_JSON)
+        .get(ClientResponse.class);
+
+    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    JSONObject json = response.getEntity(JSONObject.class);
+    verifyClusterInfo(json);
   }
 
   @Test
-  public void testAppsQueryStateNone() throws JSONException, Exception {
-    rm.start();
-    MockNM amNodeManager = rm.registerNode("amNM:1234", 2048);
-    rm.submitApp(1024);
-    amNodeManager.nodeHeartbeat(true);
+  public void testInfoSlash() throws JSONException, Exception {
+    // test with trailing "/" to make sure acts same as without slash
     WebResource r = resource();
-    JSONObject json = r.path("ws").path("v1").path("cluster").path("apps")
-        .queryParam("state", RMAppState.RUNNING.toString())
-        .accept("application/json").get(JSONObject.class);
-    assertEquals("correct number of elements", 1, json.length());
-    assertEquals("apps is not null", JSONObject.NULL, json.get("apps"));
-    rm.stop();
+    ClientResponse response = r.path("ws").path("v1").path("cluster")
+        .path("info/").accept(MediaType.APPLICATION_JSON)
+        .get(ClientResponse.class);
+
+    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    JSONObject json = response.getEntity(JSONObject.class);
+    verifyClusterInfo(json);
   }
 
   @Test
-  public void testAppsQueryStateInvalid() throws JSONException, Exception {
-    rm.start();
-    MockNM amNodeManager = rm.registerNode("amNM:1234", 2048);
-    rm.submitApp(1024);
-    amNodeManager.nodeHeartbeat(true);
+  public void testInfoDefault() throws JSONException, Exception {
     WebResource r = resource();
+    ClientResponse response = r.path("ws").path("v1").path("cluster")
+        .path("info").get(ClientResponse.class);
 
-    try {
-      r.path("ws").path("v1").path("cluster").path("apps")
-          .queryParam("state", "INVALID_test").accept("application/json")
-          .get(JSONObject.class);
-      fail("should have thrown exception on invalid state query");
-    } catch (UniformInterfaceException ue) {
-      ClientResponse response = ue.getResponse();
-
-      assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
-      JSONObject msg = response.getEntity(JSONObject.class);
-      JSONObject exception = msg.getJSONObject("RemoteException");
-      assertEquals("correct number of elements", 3, exception.length());
-      String message = exception.getString("message");
-      String type = exception.getString("exception");
-      String classname = exception.getString("javaClassName");
-      checkStringMatch(
-          "exception message",
-          "No enum const class org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState.INVALID_test",
-          message);
-      checkStringMatch("exception type", "IllegalArgumentException", type);
-      checkStringMatch("exception classname",
-          "java.lang.IllegalArgumentException", classname);
+    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    JSONObject json = response.getEntity(JSONObject.class);
+    verifyClusterInfo(json);
+  }
 
-    } finally {
-      rm.stop();
+  public void verifyClusterInfoXML(String xml) throws JSONException, Exception {
+    DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
+    DocumentBuilder db = dbf.newDocumentBuilder();
+    InputSource is = new InputSource();
+    is.setCharacterStream(new StringReader(xml));
+    Document dom = db.parse(is);
+    NodeList nodes = dom.getElementsByTagName("clusterInfo");
+    assertEquals("incorrect number of elements", 1, nodes.getLength());
+
+    for (int i = 0; i < nodes.getLength(); i++) {
+      Element element = (Element) nodes.item(i);
+
+      verifyClusterGeneric(WebServicesTestUtils.getXmlLong(element, "id"),
+          WebServicesTestUtils.getXmlLong(element, "startedOn"),
+          WebServicesTestUtils.getXmlString(element, "state"),
+          WebServicesTestUtils.getXmlString(element, "hadoopVersionBuiltOn"),
+          WebServicesTestUtils.getXmlString(element, "hadoopBuildVersion"),
+          WebServicesTestUtils.getXmlString(element, "hadoopVersion"),
+          WebServicesTestUtils.getXmlString(element,
+              "resourceManagerVersionBuiltOn"),
+          WebServicesTestUtils.getXmlString(element,
+              "resourceManagerBuildVersion"),
+          WebServicesTestUtils.getXmlString(element, "resourceManagerVersion"));
     }
   }
 
-  @Test
-  public void testAppsQueryUser() throws JSONException, Exception {
-    rm.start();
-    MockNM amNodeManager = rm.registerNode("amNM:1234", 2048);
-    rm.submitApp(1024);
-    rm.submitApp(1024);
+  public void verifyClusterInfo(JSONObject json) throws JSONException,
+      Exception {
+    assertEquals("incorrect number of elements", 1, json.length());
+    JSONObject info = json.getJSONObject("clusterInfo");
+    assertEquals("incorrect number of elements", 9, info.length());
+    verifyClusterGeneric(info.getLong("id"), info.getLong("startedOn"),
+        info.getString("state"), info.getString("hadoopVersionBuiltOn"),
+        info.getString("hadoopBuildVersion"), info.getString("hadoopVersion"),
+        info.getString("resourceManagerVersionBuiltOn"),
+        info.getString("resourceManagerBuildVersion"),
+        info.getString("resourceManagerVersion"));
 
-    amNodeManager.nodeHeartbeat(true);
-    WebResource r = resource();
-    JSONObject json = r
-        .path("ws")
-        .path("v1")
-        .path("cluster")
-        .path("apps")
-        .queryParam("user",
-            UserGroupInformation.getCurrentUser().getShortUserName())
-        .accept("application/json").get(JSONObject.class);
-    assertEquals("correct number of elements", 1, json.length());
-    JSONObject apps = json.getJSONObject("apps");
-    assertEquals("correct number of elements", 1, apps.length());
-    JSONArray array = apps.getJSONArray("app");
-    assertEquals("correct number of elements", 2, array.length());
-    rm.stop();
   }
 
-  @Test
-  public void testAppsQueryQueue() throws JSONException, Exception {
-    rm.start();
-    MockNM amNodeManager = rm.registerNode("amNM:1234", 2048);
-    rm.submitApp(1024);
-    rm.submitApp(1024);
+  public void verifyClusterGeneric(long clusterid, long startedon,
+      String state, String hadoopVersionBuiltOn, String hadoopBuildVersion,
+      String hadoopVersion, String resourceManagerVersionBuiltOn,
+      String resourceManagerBuildVersion, String resourceManagerVersion) {
+
+    assertEquals("clusterId doesn't match: ", ResourceManager.clusterTimeStamp,
+        clusterid);
+    assertEquals("startedOn doesn't match: ", ResourceManager.clusterTimeStamp,
+        startedon);
+    assertTrue("stated doesn't match: " + state,
+        state.matches(STATE.INITED.toString()));
 
-    amNodeManager.nodeHeartbeat(true);
-    WebResource r = resource();
-    JSONObject json = r.path("ws").path("v1").path("cluster").path("apps")
-        .queryParam("queue", "default").accept("application/json")
-        .get(JSONObject.class);
-    assertEquals("correct number of elements", 1, json.length());
-    JSONObject apps = json.getJSONObject("apps");
-    assertEquals("correct number of elements", 1, apps.length());
-    JSONArray array = apps.getJSONArray("app");
-    assertEquals("correct number of elements", 2, array.length());
-    rm.stop();
+    WebServicesTestUtils.checkStringMatch("hadoopVersionBuiltOn",
+        VersionInfo.getDate(), hadoopVersionBuiltOn);
+    WebServicesTestUtils.checkStringMatch("hadoopBuildVersion",
+        VersionInfo.getBuildVersion(), hadoopBuildVersion);
+    WebServicesTestUtils.checkStringMatch("hadoopVersion",
+        VersionInfo.getVersion(), hadoopVersion);
+
+    WebServicesTestUtils.checkStringMatch("resourceManagerVersionBuiltOn",
+        YarnVersionInfo.getDate(), resourceManagerVersionBuiltOn);
+    WebServicesTestUtils.checkStringMatch("resourceManagerBuildVersion",
+        YarnVersionInfo.getBuildVersion(), resourceManagerBuildVersion);
+    WebServicesTestUtils.checkStringMatch("resourceManagerVersion",
+        YarnVersionInfo.getVersion(), resourceManagerVersion);
   }
 
   @Test
-  public void testAppsQueryLimit() throws JSONException, Exception {
-    rm.start();
-    rm.registerNode("amNM:1234", 2048);
-    rm.submitApp(1024);
-    rm.submitApp(1024);
-    rm.submitApp(1024);
+  public void testClusterMetrics() throws JSONException, Exception {
     WebResource r = resource();
-    JSONObject json = r.path("ws").path("v1").path("cluster").path("apps")
-        .queryParam("limit", "2").accept("application/json")
-        .get(JSONObject.class);
-    assertEquals("correct number of elements", 1, json.length());
-    JSONObject apps = json.getJSONObject("apps");
-    assertEquals("correct number of elements", 1, apps.length());
-    JSONArray array = apps.getJSONArray("app");
-    assertEquals("correct number of elements", 2, array.length());
-    rm.stop();
+    ClientResponse response = r.path("ws").path("v1").path("cluster")
+        .path("metrics").accept(MediaType.APPLICATION_JSON)
+        .get(ClientResponse.class);
+
+    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    JSONObject json = response.getEntity(JSONObject.class);
+    verifyClusterMetricsJSON(json);
   }
 
   @Test
-  public void testAppsQueryStartBegin() throws JSONException, Exception {
-    rm.start();
-    long start = System.currentTimeMillis();
-    Thread.sleep(1);
-    rm.registerNode("amNM:1234", 2048);
-    rm.submitApp(1024);
-    rm.submitApp(1024);
-    rm.submitApp(1024);
+  public void testClusterMetricsSlash() throws JSONException, Exception {
     WebResource r = resource();
-    JSONObject json = r.path("ws").path("v1").path("cluster").path("apps")
-        .queryParam("startedTimeBegin", String.valueOf(start))
-        .accept("application/json").get(JSONObject.class);
-    assertEquals("correct number of elements", 1, json.length());
-    JSONObject apps = json.getJSONObject("apps");
-    assertEquals("correct number of elements", 1, apps.length());
-    JSONArray array = apps.getJSONArray("app");
-    assertEquals("correct number of elements", 3, array.length());
-    rm.stop();
+    ClientResponse response = r.path("ws").path("v1").path("cluster")
+        .path("metrics/").accept(MediaType.APPLICATION_JSON)
+        .get(ClientResponse.class);
+
+    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    JSONObject json = response.getEntity(JSONObject.class);
+    verifyClusterMetricsJSON(json);
   }
 
   @Test
-  public void testAppsQueryStartBeginSome() throws JSONException, Exception {
-    rm.start();
-    rm.registerNode("amNM:1234", 2048);
-    rm.submitApp(1024);
-    rm.submitApp(1024);
-    long start = System.currentTimeMillis();
-    Thread.sleep(1);
-    rm.submitApp(1024);
+  public void testClusterMetricsDefault() throws JSONException, Exception {
     WebResource r = resource();
-    JSONObject json = r.path("ws").path("v1").path("cluster").path("apps")
-        .queryParam("startedTimeBegin", String.valueOf(start))
-        .accept("application/json").get(JSONObject.class);
-    assertEquals("correct number of elements", 1, json.length());
-    JSONObject apps = json.getJSONObject("apps");
-    assertEquals("correct number of elements", 1, apps.length());
-    JSONArray array = apps.getJSONArray("app");
-    assertEquals("correct number of elements", 1, array.length());
-    rm.stop();
+    ClientResponse response = r.path("ws").path("v1").path("cluster")
+        .path("metrics").get(ClientResponse.class);
+
+    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    JSONObject json = response.getEntity(JSONObject.class);
+    verifyClusterMetricsJSON(json);
   }
 
   @Test
-  public void testAppsQueryStartEnd() throws JSONException, Exception {
-    rm.start();
-    rm.registerNode("amNM:1234", 2048);
-    long end = System.currentTimeMillis();
-    Thread.sleep(1);
-    rm.submitApp(1024);
-    rm.submitApp(1024);
-    rm.submitApp(1024);
+  public void testClusterMetricsXML() throws JSONException, Exception {
     WebResource r = resource();
-    JSONObject json = r.path("ws").path("v1").path("cluster").path("apps")
-        .queryParam("startedTimeEnd", String.valueOf(end))
-        .accept("application/json").get(JSONObject.class);
-    assertEquals("correct number of elements", 1, json.length());
-    assertEquals("apps is not null", JSONObject.NULL, json.get("apps"));
-    rm.stop();
+    ClientResponse response = r.path("ws").path("v1").path("cluster")
+        .path("metrics").accept("application/xml").get(ClientResponse.class);
+    assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+    String xml = response.getEntity(String.class);
+    verifyClusterMetricsXML(xml);
   }
 
-  @Test
-  public void testAppsQueryStartBeginEnd() throws JSONException, Exception {
-    rm.start();
-    rm.registerNode("amNM:1234", 2048);
-    long start = System.currentTimeMillis();
-    Thread.sleep(1);
-    rm.submitApp(1024);
-    rm.submitApp(1024);
-    long end = System.currentTimeMillis();
-    Thread.sleep(1);
-    rm.submitApp(1024);
-
-    WebResource r = resource();
-    JSONObject json = r.path("ws").path("v1").path("cluster").path("apps")
-        .queryParam("startedTimeBegin", String.valueOf(start))
-        .queryParam("startedTimeEnd", String.valueOf(end))
-        .accept("application/json").get(JSONObject.class);
-    assertEquals("correct number of elements", 1, json.length());
-    JSONObject apps = json.getJSONObject("apps");
-    assertEquals("correct number of elements", 1, apps.length());
-    JSONArray array = apps.getJSONArray("app");
-    assertEquals("correct number of elements", 2, array.length());
-    rm.stop();
+  public void verifyClusterMetricsXML(String xml) throws JSONException,
+      Exception {
+    DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
+    DocumentBuilder db = dbf.newDocumentBuilder();
+    InputSource is = new InputSource();
+    is.setCharacterStream(new StringReader(xml));
+    Document dom = db.parse(is);
+    NodeList nodes = dom.getElementsByTagName("clusterMetrics");
+    assertEquals("incorrect number of elements", 1, nodes.getLength());
+
+    for (int i = 0; i < nodes.getLength(); i++) {
+      Element element = (Element) nodes.item(i);
+
+      verifyClusterMetrics(
+          WebServicesTestUtils.getXmlInt(element, "appsSubmitted"),
+          WebServicesTestUtils.getXmlInt(element, "reservedMB"),
+          WebServicesTestUtils.getXmlInt(element, "availableMB"),
+          WebServicesTestUtils.getXmlInt(element, "allocatedMB"),
+          WebServicesTestUtils.getXmlInt(element, "containersAllocated"),
+          WebServicesTestUtils.getXmlInt(element, "totalMB"),
+          WebServicesTestUtils.getXmlInt(element, "totalNodes"),
+          WebServicesTestUtils.getXmlInt(element, "lostNodes"),
+          WebServicesTestUtils.getXmlInt(element, "unhealthyNodes"),
+          WebServicesTestUtils.getXmlInt(element, "decommissionedNodes"),
+          WebServicesTestUtils.getXmlInt(element, "rebootedNodes"));
+    }
   }
 
-  @Test
-  public void testAppsQueryFinishBegin() throws JSONException, Exception {
-    rm.start();
-    MockNM amNodeManager = rm.registerNode("amNM:1234", 2048);
-    long start = System.currentTimeMillis();
-    Thread.sleep(1);
-    RMApp app1 = rm.submitApp(1024);
-    amNodeManager.nodeHeartbeat(true);
-    // finish App
-    MockAM am = rm
-        .sendAMLaunched(app1.getCurrentAppAttempt().getAppAttemptId());
-    am.registerAppAttempt();
-    am.unregisterAppAttempt();
-    rm.submitApp(1024);
-    rm.submitApp(1024);
+  public void verifyClusterMetricsJSON(JSONObject json) throws JSONException,
+      Exception {
+    assertEquals("incorrect number of elements", 1, json.length());
+    JSONObject clusterinfo = json.getJSONObject("clusterMetrics");
+    assertEquals("incorrect number of elements", 11, clusterinfo.length());
+    verifyClusterMetrics(clusterinfo.getInt("appsSubmitted"),
+        clusterinfo.getInt("reservedMB"), clusterinfo.getInt("availableMB"),
+        clusterinfo.getInt("allocatedMB"),
+        clusterinfo.getInt("containersAllocated"),
+        clusterinfo.getInt("totalMB"), clusterinfo.getInt("totalNodes"),
+        clusterinfo.getInt("lostNodes"), clusterinfo.getInt("unhealthyNodes"),
+        clusterinfo.getInt("decommissionedNodes"),
+        clusterinfo.getInt("rebootedNodes"));
+  }
 
-    WebResource r = resource();
-    JSONObject json = r.path("ws").path("v1").path("cluster").path("apps")
-        .queryParam("finishedTimeBegin", String.valueOf(start))
-        .accept("application/json").get(JSONObject.class);
-    assertEquals("correct number of elements", 1, json.length());
-    JSONObject apps = json.getJSONObject("apps");
-    assertEquals("correct number of elements", 1, apps.length());
-    JSONArray array = apps.getJSONArray("app");
-    assertEquals("correct number of elements", 1, array.length());
-    rm.stop();
+  public void verifyClusterMetrics(int sub, int reservedMB, int availableMB,
+      int allocMB, int containersAlloc, int totalMB, int totalNodes,
+      int lostNodes, int unhealthyNodes, int decommissionedNodes,
+      int rebootedNodes) throws JSONException, Exception {
+
+    ResourceScheduler rs = rm.getResourceScheduler();
+    QueueMetrics metrics = rs.getRootQueueMetrics();
+    ClusterMetrics clusterMetrics = ClusterMetrics.getMetrics();
+    final long MB_IN_GB = 1024;
+
+    long totalMBExpect = (metrics.getReservedGB() * MB_IN_GB)
+        + (metrics.getAvailableGB() * MB_IN_GB)
+        + (metrics.getAllocatedGB() * MB_IN_GB);
+
+    assertEquals("appsSubmitted doesn't match", metrics.getAppsSubmitted(), sub);
+    assertEquals("reservedMB doesn't match",
+        metrics.getReservedGB() * MB_IN_GB, reservedMB);
+    assertEquals("availableMB doesn't match", metrics.getAvailableGB()
+        * MB_IN_GB, availableMB);
+    assertEquals("allocatedMB doesn't match", metrics.getAllocatedGB()
+        * MB_IN_GB, allocMB);
+    assertEquals("containersAllocated doesn't match", 0, containersAlloc);
+    assertEquals("totalMB doesn't match", totalMBExpect, totalMB);
+    assertEquals("totalNodes doesn't match", clusterMetrics.getNumNMs(),
+        totalNodes);
+    assertEquals("lostNodes doesn't match", clusterMetrics.getNumLostNMs(),
+        lostNodes);
+    assertEquals("unhealthyNodes doesn't match",
+        clusterMetrics.getUnhealthyNMs(), unhealthyNodes);
+    assertEquals("decommissionedNodes doesn't match",
+        clusterMetrics.getNumDecommisionedNMs(), decommissionedNodes);
+    assertEquals("rebootedNodes doesn't match",
+        clusterMetrics.getNumRebootedNMs(), rebootedNodes);
   }
 
   @Test
-  public void testAppsQueryFinishEnd() throws JSONException, Exception {
-    rm.start();
-    MockNM amNodeManager = rm.registerNode("amNM:1234", 2048);
-    RMApp app1 = rm.submitApp(1024);
-    amNodeManager.nodeHeartbeat(true);
-    // finish App
-    MockAM am = rm
-        .sendAMLaunched(app1.getCurrentAppAttempt().getAppAttemptId());
-    am.registerAppAttempt();
-    am.unregisterAppAttempt();
-
-    rm.submitApp(1024);
-    rm.submitApp(1024);
-    long end = System.currentTimeMillis();
-
+  public void testClusterSchedulerFifo() throws JSONException, Exception {
     WebResource r = resource();
-    JSONObject json = r.path("ws").path("v1").path("cluster").path("apps")
-        .queryParam("finishedTimeEnd", String.valueOf(end))
-        .accept("application/json").get(JSONObject.class);
-    assertEquals("correct number of elements", 1, json.length());
-    JSONObject apps = json.getJSONObject("apps");
-    assertEquals("correct number of elements", 1, apps.length());
-    JSONArray array = apps.getJSONArray("app");
-    assertEquals("correct number of elements", 3, array.length());
-    rm.stop();
+    ClientResponse response = r.path("ws").path("v1").path("cluster")
+        .path("scheduler").accept(MediaType.APPLICATION_JSON)
+        .get(ClientResponse.class);
+
+    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    JSONObject json = response.getEntity(JSONObject.class);
+    verifyClusterSchedulerFifo(json);
   }
 
   @Test
-  public void testAppsQueryFinishBeginEnd() throws JSONException, Exception {
-    rm.start();
-    MockNM amNodeManager = rm.registerNode("amNM:1234", 2048);
-    long start = System.currentTimeMillis();
-    Thread.sleep(1);
-    RMApp app1 = rm.submitApp(1024);
-    amNodeManager.nodeHeartbeat(true);
-    // finish App
-    MockAM am = rm
-        .sendAMLaunched(app1.getCurrentAppAttempt().getAppAttemptId());
-    am.registerAppAttempt();
-    am.unregisterAppAttempt();
-
-    rm.submitApp(1024);
-    rm.submitApp(1024);
-    long end = System.currentTimeMillis();
-
+  public void testClusterSchedulerFifoSlash() throws JSONException, Exception {
     WebResource r = resource();
-    JSONObject json = r.path("ws").path("v1").path("cluster").path("apps")
-        .queryParam("finishedTimeBegin", String.valueOf(start))
-        .queryParam("finishedTimeEnd", String.valueOf(end))
-        .accept("application/json").get(JSONObject.class);
-    assertEquals("correct number of elements", 1, json.length());
-    JSONObject apps = json.getJSONObject("apps");
-    assertEquals("correct number of elements", 1, apps.length());
-    JSONArray array = apps.getJSONArray("app");
-    assertEquals("correct number of elements", 1, array.length());
-    rm.stop();
-  }
+    ClientResponse response = r.path("ws").path("v1").path("cluster")
+        .path("scheduler/").accept(MediaType.APPLICATION_JSON)
+        .get(ClientResponse.class);
 
-  @Test
-  public void testSingleApp() throws JSONException, Exception {
-    rm.start();
-    MockNM amNodeManager = rm.registerNode("amNM:1234", 2048);
-    RMApp app1 = rm.submitApp(1024);
-    amNodeManager.nodeHeartbeat(true);
-    testSingleAppsHelper(app1.getApplicationId().toString(), app1);
-    rm.stop();
+    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    JSONObject json = response.getEntity(JSONObject.class);
+    verifyClusterSchedulerFifo(json);
   }
 
   @Test
-  public void testSingleAppsSlash() throws JSONException, Exception {
-    rm.start();
-    MockNM amNodeManager = rm.registerNode("amNM:1234", 2048);
-    RMApp app1 = rm.submitApp(1024);
-    amNodeManager.nodeHeartbeat(true);
-    testSingleAppsHelper(app1.getApplicationId().toString() + "/", app1);
-    rm.stop();
+  public void testClusterSchedulerFifoDefault() throws JSONException, Exception {
+    WebResource r = resource();
+    ClientResponse response = r.path("ws").path("v1").path("cluster")
+        .path("scheduler").get(ClientResponse.class);
+
+    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    JSONObject json = response.getEntity(JSONObject.class);
+    verifyClusterSchedulerFifo(json);
   }
 
   @Test
-  public void testInvalidApp() throws JSONException, Exception {
-    rm.start();
-    MockNM amNodeManager = rm.registerNode("amNM:1234", 2048);
-    rm.submitApp(1024);
-    amNodeManager.nodeHeartbeat(true);
+  public void testClusterSchedulerFifoXML() throws JSONException, Exception {
     WebResource r = resource();
+    ClientResponse response = r.path("ws").path("v1").path("cluster")
+        .path("scheduler").accept(MediaType.APPLICATION_XML)
+        .get(ClientResponse.class);
 
-    try {
-      r.path("ws").path("v1").path("cluster").path("apps")
-          .path("application_invalid_12").accept("application/json")
-          .get(JSONObject.class);
-      fail("should have thrown exception on invalid appid");
-    } catch (UniformInterfaceException ue) {
-      ClientResponse response = ue.getResponse();
-
-      assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
-      JSONObject msg = response.getEntity(JSONObject.class);
-      JSONObject exception = msg.getJSONObject("RemoteException");
-      assertEquals("correct number of elements", 3, exception.length());
-      String message = exception.getString("message");
-      String type = exception.getString("exception");
-      String classname = exception.getString("javaClassName");
-      checkStringMatch("exception message", "For input string: \"invalid\"",
-          message);
-      checkStringMatch("exception type", "NumberFormatException", type);
-      checkStringMatch("exception classname",
-          "java.lang.NumberFormatException", classname);
+    assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+    String xml = response.getEntity(String.class);
+    verifySchedulerFifoXML(xml);
+  }
 
-    } finally {
-      rm.stop();
+  public void verifySchedulerFifoXML(String xml) throws JSONException,
+      Exception {
+    DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
+    DocumentBuilder db = dbf.newDocumentBuilder();
+    InputSource is = new InputSource();
+    is.setCharacterStream(new StringReader(xml));
+    Document dom = db.parse(is);
+    NodeList nodesSched = dom.getElementsByTagName("scheduler");
+    assertEquals("incorrect number of elements", 1, nodesSched.getLength());
+    NodeList nodes = dom.getElementsByTagName("schedulerInfo");
+    assertEquals("incorrect number of elements", 1, nodes.getLength());
+
+    for (int i = 0; i < nodes.getLength(); i++) {
+      Element element = (Element) nodes.item(i);
+
+      verifyClusterSchedulerFifoGeneric(
+          WebServicesTestUtils.getXmlAttrString(element, "xsi:type"),
+          WebServicesTestUtils.getXmlString(element, "qstate"),
+          WebServicesTestUtils.getXmlFloat(element, "capacity"),
+          WebServicesTestUtils.getXmlFloat(element, "usedCapacity"),
+          WebServicesTestUtils.getXmlInt(element, "minQueueMemoryCapacity"),
+          WebServicesTestUtils.getXmlInt(element, "maxQueueMemoryCapacity"),
+          WebServicesTestUtils.getXmlInt(element, "numNodes"),
+          WebServicesTestUtils.getXmlInt(element, "usedNodeCapacity"),
+          WebServicesTestUtils.getXmlInt(element, "availNodeCapacity"),
+          WebServicesTestUtils.getXmlInt(element, "totalNodeCapacity"),
+          WebServicesTestUtils.getXmlInt(element, "numContainers"));
     }
   }
 
-  @Test
-  public void testNonexistApp() throws JSONException, Exception {
-    rm.start();
-    MockNM amNodeManager = rm.registerNode("amNM:1234", 2048);
-    rm.submitApp(1024);
-    amNodeManager.nodeHeartbeat(true);
-    WebResource r = resource();
+  public void verifyClusterSchedulerFifo(JSONObject json) throws JSONException,
+      Exception {
+    assertEquals("incorrect number of elements", 1, json.length());
+    JSONObject info = json.getJSONObject("scheduler");
+    assertEquals("incorrect number of elements", 1, info.length());
+    info = info.getJSONObject("schedulerInfo");
+    assertEquals("incorrect number of elements", 11, info.length());
 
-    try {
-      r.path("ws").path("v1").path("cluster").path("apps")
-          .path("application_00000_0099").accept("application/json")
-          .get(JSONObject.class);
-      fail("should have thrown exception on invalid appid");
-    } catch (UniformInterfaceException ue) {
-      ClientResponse response = ue.getResponse();
+    verifyClusterSchedulerFifoGeneric(info.getString("type"),
+        info.getString("qstate"), (float) info.getDouble("capacity"),
+        (float) info.getDouble("usedCapacity"),
+        info.getInt("minQueueMemoryCapacity"),
+        info.getInt("maxQueueMemoryCapacity"), info.getInt("numNodes"),
+        info.getInt("usedNodeCapacity"), info.getInt("availNodeCapacity"),
+        info.getInt("totalNodeCapacity"), info.getInt("numContainers"));
 
-      assertEquals(Status.NOT_FOUND, response.getClientResponseStatus());
-      JSONObject msg = response.getEntity(JSONObject.class);
-      JSONObject exception = msg.getJSONObject("RemoteException");
-      assertEquals("correct number of elements", 3, exception.length());
-      String message = exception.getString("message");
-      String type = exception.getString("exception");
-      String classname = exception.getString("javaClassName");
-      checkStringMatch("exception message",
-          "java.lang.Exception: app with id: application_00000_0099 not found",
-          message);
-      checkStringMatch("exception type", "NotFoundException", type);
-      checkStringMatch("exception classname",
-          "org.apache.hadoop.yarn.webapp.NotFoundException", classname);
-    } finally {
-      rm.stop();
-    }
   }
 
-  public void testSingleAppsHelper(String path, RMApp app)
+  public void verifyClusterSchedulerFifoGeneric(String type, String state,
+      float capacity, float usedCapacity, int minQueueCapacity,
+      int maxQueueCapacity, int numNodes, int usedNodeCapacity,
+      int availNodeCapacity, int totalNodeCapacity, int numContainers)
       throws JSONException, Exception {
-    WebResource r = resource();
-    JSONObject json = r.path("ws").path("v1").path("cluster").path("apps")
-        .path(path).accept("application/json").get(JSONObject.class);
-    assertEquals("correct number of elements", 1, json.length());
-    verifyAppInfo(json.getJSONObject("app"), app);
-  }
 
-  public void verifyAppInfo(JSONObject info, RMApp app) throws JSONException,
-      Exception {
+    assertEquals("type doesn't match", "fifoScheduler", type);
+    assertEquals("qstate doesn't match", QueueState.RUNNING.toString(), state);
+    assertEquals("capacity doesn't match", 1.0, capacity, 0.0);
+    assertEquals("usedCapacity doesn't match", Float.NaN, usedCapacity, 0.0);
+    assertEquals("minQueueMemoryCapacity doesn't match", 1024, minQueueCapacity);
+    assertEquals("maxQueueMemoryCapacity doesn't match", 10240,
+        maxQueueCapacity);
+    assertEquals("numNodes doesn't match", 0, numNodes);
+    assertEquals("usedNodeCapacity doesn't match", 0, usedNodeCapacity);
+    assertEquals("availNodeCapacity doesn't match", 0, availNodeCapacity);
+    assertEquals("totalNodeCapacity doesn't match", 0, totalNodeCapacity);
+    assertEquals("numContainers doesn't match", 0, numContainers);
 
-    // 15 because trackingUrl not assigned yet
-    assertEquals("correct number of elements", 15, info.length());
-    String id = info.getString("id");
-    String expectedId = app.getApplicationId().toString();
-    checkStringMatch("id", expectedId, id);
-
-    String user = info.getString("user");
-    String expectedUser = app.getUser();
-    checkStringMatch("user", expectedUser, user);
-
-    checkStringMatch("name", "", info.getString("name"));
-    checkStringMatch("queue", "default", info.getString("queue"));
-    checkStringMatch("state", RMAppState.ACCEPTED.toString(),
-        info.getString("state"));
-    checkStringMatch("finalStatus",
-        FinalApplicationStatus.UNDEFINED.toString(),
-        info.getString("finalStatus"));
-    assertEquals("progress doesn't match", 0, info.getDouble("progress"), 0.0);
-    checkStringMatch("trackingUI", "UNASSIGNED", info.getString("trackingUI"));
-    checkStringMatch("diagnostics", "", info.getString("diagnostics"));
-    assertEquals("clusterId doesn't match", ResourceManager.clusterTimeStamp,
-        info.getLong("clusterId"));
-    assertEquals("startedTime doesn't match", app.getStartTime(),
-        info.getLong("startedTime"));
-    assertEquals("finishedTime doesn't match", app.getFinishTime(),
-        info.getLong("finishedTime"));
-    assertTrue("elapsed time not greater than 0",
-        info.getLong("elapsedTime") > 0);
-    checkStringMatch("amHostHttpAddress", app.getCurrentAppAttempt()
-        .getMasterContainer().getNodeHttpAddress(),
-        info.getString("amHostHttpAddress"));
-    assertTrue("amContainerLogs doesn't match",
-        info.getString("amContainerLogs").startsWith("http://"));
-  }
-
-  private void checkStringMatch(String print, String expected, String got) {
-    assertTrue(
-        print + " doesn't match, got: " + got + " expected: " + expected,
-        got.matches(expected));
   }
 
 }