瀏覽代碼

YARN-9152. Improved AuxServices REST API output.
Contributed by Billie Rinaldi

Eric Yang 6 年之前
父節點
當前提交
a80d321074

+ 3 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/JAXBContextResolver.java

@@ -32,6 +32,8 @@ import javax.xml.bind.JAXBContext;
 
 import org.apache.hadoop.yarn.server.nodemanager.webapp.dao.AppInfo;
 import org.apache.hadoop.yarn.server.nodemanager.webapp.dao.AppsInfo;
+import org.apache.hadoop.yarn.server.nodemanager.webapp.dao.AuxiliaryServiceInfo;
+import org.apache.hadoop.yarn.server.nodemanager.webapp.dao.AuxiliaryServicesInfo;
 import org.apache.hadoop.yarn.server.nodemanager.webapp.dao.ContainerInfo;
 import org.apache.hadoop.yarn.server.nodemanager.webapp.dao.ContainersInfo;
 import org.apache.hadoop.yarn.server.nodemanager.webapp.dao.NodeInfo;
@@ -46,6 +48,7 @@ public class JAXBContextResolver implements ContextResolver<JAXBContext> {
 
   // you have to specify all the dao classes here
   private final Class[] cTypes = {AppInfo.class, AppsInfo.class,
+      AuxiliaryServicesInfo.class, AuxiliaryServiceInfo.class,
       ContainerInfo.class, ContainersInfo.class, NodeInfo.class,
       RemoteExceptionData.class};
 

+ 0 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java

@@ -567,9 +567,6 @@ public class NMWebServices {
       HttpServletRequest hsr) {
     init();
     AuxiliaryServicesInfo auxiliaryServices = new AuxiliaryServicesInfo();
-    if (!hasAdminAccess(hsr)) {
-      return auxiliaryServices;
-    }
     Collection<AuxServiceRecord> loadedServices = nmContext.getAuxServices()
         .getServiceRecords();
     if (loadedServices != null) {

+ 5 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/AuxiliaryServicesInfo.java

@@ -31,7 +31,7 @@ import java.util.Collection;
 @XmlRootElement(name = "services")
 @XmlAccessorType(XmlAccessType.FIELD)
 public class AuxiliaryServicesInfo {
-  private ArrayList<AuxiliaryServiceInfo> services = new
+  private ArrayList<AuxiliaryServiceInfo> service = new
       ArrayList<>();
 
   public AuxiliaryServicesInfo() {
@@ -39,17 +39,17 @@ public class AuxiliaryServicesInfo {
   }
 
   public void add(AuxServiceRecord s) {
-    services.add(new AuxiliaryServiceInfo(s.getName(), s.getVersion(), s
+    service.add(new AuxiliaryServiceInfo(s.getName(), s.getVersion(), s
         .getLaunchTime()));
   }
 
   public void addAll(Collection<AuxServiceRecord> serviceList) {
-    for (AuxServiceRecord service : serviceList) {
-      add(service);
+    for (AuxServiceRecord s : serviceList) {
+      add(s);
     }
   }
 
   public ArrayList<AuxiliaryServiceInfo> getServices() {
-    return services;
+    return service;
   }
 }

+ 296 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServicesAuxServices.java

@@ -0,0 +1,296 @@
+/**
+ * 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.server.nodemanager.webapp;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.File;
+import java.io.StringReader;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.Arrays;
+import java.util.Date;
+
+import javax.ws.rs.core.MediaType;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+
+import com.sun.jersey.api.client.filter.LoggingFilter;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.http.JettyUtils;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.nodemanager.Context;
+import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService;
+import org.apache.hadoop.yarn.server.nodemanager.NodeHealthCheckerService;
+import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
+import org.apache.hadoop.yarn.server.nodemanager.ResourceView;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServices;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.records.AuxServiceRecord;
+import org.apache.hadoop.yarn.server.nodemanager.webapp.WebServer.NMWebApp;
+import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
+import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
+import org.apache.hadoop.yarn.webapp.JerseyTestBase;
+import org.apache.hadoop.yarn.webapp.WebApp;
+import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
+import org.codehaus.jettison.json.JSONArray;
+import org.codehaus.jettison.json.JSONObject;
+import org.junit.AfterClass;
+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.servlet.ServletModule;
+import com.sun.jersey.api.client.ClientResponse;
+import com.sun.jersey.api.client.WebResource;
+import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
+import com.sun.jersey.test.framework.WebAppDescriptor;
+
+/**
+ * Basic sanity Tests for AuxServices.
+ *
+ */
+public class TestNMWebServicesAuxServices extends JerseyTestBase {
+  private static final String AUX_SERVICES_PATH = "auxiliaryservices";
+  private static Context nmContext;
+  private static Configuration conf = new Configuration();
+  private DateFormat dateFormat =
+      new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+
+  private static final File testRootDir = new File("target",
+      TestNMWebServicesContainers.class.getSimpleName());
+  private static final File testLogDir = new File("target",
+      TestNMWebServicesContainers.class.getSimpleName() + "LogDir");
+
+  private static class WebServletModule extends ServletModule {
+    @Override
+    protected void configureServlets() {
+      ResourceView resourceView = new ResourceView() {
+        @Override
+        public long getVmemAllocatedForContainers() {
+          // 15.5G in bytes
+          return new Long("16642998272");
+        }
+
+        @Override
+        public long getPmemAllocatedForContainers() {
+          // 16G in bytes
+          return new Long("17179869184");
+        }
+
+        @Override
+        public long getVCoresAllocatedForContainers() {
+          return new Long("4000");
+        }
+
+        @Override
+        public boolean isVmemCheckEnabled() {
+          return true;
+        }
+
+        @Override
+        public boolean isPmemCheckEnabled() {
+          return true;
+        }
+      };
+      conf.set(YarnConfiguration.NM_LOCAL_DIRS, testRootDir.getAbsolutePath());
+      conf.set(YarnConfiguration.NM_LOG_DIRS, testLogDir.getAbsolutePath());
+      LocalDirsHandlerService dirsHandler = new LocalDirsHandlerService();
+      NodeHealthCheckerService healthChecker = new NodeHealthCheckerService(
+          NodeManager.getNodeHealthScriptRunner(conf), dirsHandler);
+      healthChecker.init(conf);
+      dirsHandler = healthChecker.getDiskHandler();
+      ApplicationACLsManager aclsManager = new ApplicationACLsManager(conf);
+      nmContext = new NodeManager.NMContext(null, null, dirsHandler,
+          aclsManager, null, false, conf) {
+        public NodeId getNodeId() {
+          return NodeId.newInstance("testhost.foo.com", 8042);
+        };
+
+        public int getHttpPort() {
+          return 1234;
+        };
+      };
+      WebApp nmWebApp = new NMWebApp(resourceView, aclsManager, dirsHandler);
+      bind(JAXBContextResolver.class);
+      bind(NMWebServices.class);
+      bind(GenericExceptionHandler.class);
+      bind(Context.class).toInstance(nmContext);
+      bind(WebApp.class).toInstance(nmWebApp);
+      bind(ResourceView.class).toInstance(resourceView);
+      bind(ApplicationACLsManager.class).toInstance(aclsManager);
+      bind(LocalDirsHandlerService.class).toInstance(dirsHandler);
+
+      serve("/*").with(GuiceContainer.class);
+    }
+  }
+
+  static {
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
+  }
+
+  @Before
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
+    testRootDir.mkdirs();
+    testLogDir.mkdir();
+  }
+
+  @AfterClass
+  static public void cleanup() {
+    FileUtil.fullyDelete(testRootDir);
+    FileUtil.fullyDelete(testLogDir);
+  }
+
+  public TestNMWebServicesAuxServices() {
+    super(new WebAppDescriptor.Builder(
+        "org.apache.hadoop.yarn.server.nodemanager.webapp")
+        .contextListenerClass(GuiceServletConfig.class)
+        .filterClass(com.google.inject.servlet.GuiceFilter.class)
+        .contextPath("jersey-guice-filter").servletPath("/").build());
+  }
+
+  @Test
+  public void testNodeAuxServicesNone() throws Exception {
+    addAuxServices();
+    WebResource r = resource();
+    ClientResponse response = r.path("ws").path("v1").path("node")
+        .path(AUX_SERVICES_PATH).accept(MediaType.APPLICATION_JSON)
+        .get(ClientResponse.class);
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
+    JSONObject json = response.getEntity(JSONObject.class);
+    assertEquals("aux services isn't empty",
+        new JSONObject().toString(), json.get("services").toString());
+  }
+
+  private void addAuxServices(AuxServiceRecord... records) {
+    AuxServices auxServices = mock(AuxServices.class);
+    when(auxServices.getServiceRecords()).thenReturn(Arrays.asList(records));
+    nmContext.setAuxServices(auxServices);
+  }
+
+  @Test
+  public void testNodeAuxServices() throws Exception {
+    testNodeHelper(AUX_SERVICES_PATH, MediaType.APPLICATION_JSON);
+  }
+
+  @Test
+  public void testNodeAuxServicesSlash() throws Exception {
+    testNodeHelper(AUX_SERVICES_PATH + "/", MediaType.APPLICATION_JSON);
+  }
+
+  // make sure default is json output
+  @Test
+  public void testNodeAuxServicesDefault() throws Exception {
+    testNodeHelper(AUX_SERVICES_PATH + "/", "");
+  }
+
+  public void testNodeHelper(String path, String media) throws Exception {
+    AuxServiceRecord r1 = new AuxServiceRecord().name("name1").launchTime(new
+        Date(123L)).version("1");
+    AuxServiceRecord r2 = new AuxServiceRecord().name("name2").launchTime(new
+        Date(456L));
+    addAuxServices(r1, r2);
+    WebResource r = resource();
+    client().addFilter(new LoggingFilter());
+
+    ClientResponse response = r.path("ws").path("v1").path("node").path(path)
+        .accept(media).get(ClientResponse.class);
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
+    JSONObject json = response.getEntity(JSONObject.class);
+    JSONObject info = json.getJSONObject("services");
+    assertEquals("incorrect number of elements", 1, info.length());
+    JSONArray auxInfo = info.getJSONArray("service");
+    assertEquals("incorrect number of elements", 2, auxInfo.length());
+
+    verifyNodeAuxServiceInfo(auxInfo.getJSONObject(0), r1);
+    verifyNodeAuxServiceInfo(auxInfo.getJSONObject(1), r2);
+  }
+
+  @Test
+  public void testNodeContainerXML() throws Exception {
+    AuxServiceRecord r1 = new AuxServiceRecord().name("name1").launchTime(new
+        Date(123L)).version("1");
+    AuxServiceRecord r2 = new AuxServiceRecord().name("name2").launchTime(new
+        Date(456L));
+    addAuxServices(r1, r2);
+    WebResource r = resource();
+
+    ClientResponse response = r.path("ws").path("v1").path("node")
+        .path(AUX_SERVICES_PATH).accept(MediaType.APPLICATION_XML)
+        .get(ClientResponse.class);
+    assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
+    String xml = response.getEntity(String.class);
+    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("service");
+    assertEquals("incorrect number of elements", 2, nodes.getLength());
+    verifyContainersInfoXML(nodes, r1, r2);
+  }
+
+  public void verifyContainersInfoXML(NodeList nodes, AuxServiceRecord...
+      records) {
+    for (int i = 0; i < nodes.getLength(); i++) {
+      Element element = (Element) nodes.item(i);
+
+      verifyNodeAuxServiceInfoGeneric(records[i],
+          WebServicesTestUtils.getXmlString(element, "name"),
+          WebServicesTestUtils.getXmlString(element, "version"),
+          WebServicesTestUtils.getXmlString(element, "startTime"));
+    }
+  }
+
+  public void verifyNodeAuxServiceInfo(JSONObject info, AuxServiceRecord r)
+      throws Exception {
+    String version = null;
+    if (info.has("version")) {
+      version = info.getString("version");
+    }
+    assertEquals("incorrect number of elements",
+        version == null ? 2 : 3, info.length());
+    verifyNodeAuxServiceInfoGeneric(r, info.getString("name"),
+        version, info.getString("startTime"));
+  }
+
+  public void verifyNodeAuxServiceInfoGeneric(AuxServiceRecord r, String name,
+      String version, String startTime) {
+    assertEquals(r.getName(), name);
+    assertEquals(r.getVersion(), version);
+    assertEquals("startTime", dateFormat.format(r.getLaunchTime()),
+        startTime);
+  }
+}