Преглед на файлове

MAPREDUCE-5171. Expose blacklisted nodes from the MR AM REST API. (sandyr via tucu)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1493603 13f79535-47bb-0310-9956-ffa450edef68
Alejandro Abdelnur преди 12 години
родител
ревизия
c71d86cd7c
променени са 9 файла, в които са добавени 148 реда и са изтрити 2 реда
  1. 3 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/AppContext.java
  2. 7 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
  3. 4 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java
  4. 9 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AMWebServices.java
  5. 42 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/BlacklistedNodesInfo.java
  6. 12 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockAppContext.java
  7. 6 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java
  8. 58 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServices.java
  9. 7 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java

+ 3 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/AppContext.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.mapreduce.v2.app;
 package org.apache.hadoop.mapreduce.v2.app;
 
 
 import java.util.Map;
 import java.util.Map;
+import java.util.Set;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
@@ -56,4 +57,6 @@ public interface AppContext {
   Clock getClock();
   Clock getClock();
   
   
   ClusterInfo getClusterInfo();
   ClusterInfo getClusterInfo();
+  
+  Set<String> getBlacklistedNodes();
 }
 }

+ 7 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java

@@ -29,6 +29,7 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Map.Entry;
+import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentHashMap;
 
 
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.io.IOUtils;
@@ -95,6 +96,7 @@ import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocator;
 import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocatorEvent;
 import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocatorEvent;
 import org.apache.hadoop.mapreduce.v2.app.rm.RMCommunicator;
 import org.apache.hadoop.mapreduce.v2.app.rm.RMCommunicator;
 import org.apache.hadoop.mapreduce.v2.app.rm.RMContainerAllocator;
 import org.apache.hadoop.mapreduce.v2.app.rm.RMContainerAllocator;
+import org.apache.hadoop.mapreduce.v2.app.rm.RMContainerRequestor;
 import org.apache.hadoop.mapreduce.v2.app.rm.RMHeartbeatHandler;
 import org.apache.hadoop.mapreduce.v2.app.rm.RMHeartbeatHandler;
 import org.apache.hadoop.mapreduce.v2.app.speculate.DefaultSpeculator;
 import org.apache.hadoop.mapreduce.v2.app.speculate.DefaultSpeculator;
 import org.apache.hadoop.mapreduce.v2.app.speculate.Speculator;
 import org.apache.hadoop.mapreduce.v2.app.speculate.Speculator;
@@ -937,6 +939,11 @@ public class MRAppMaster extends CompositeService {
     public ClusterInfo getClusterInfo() {
     public ClusterInfo getClusterInfo() {
       return this.clusterInfo;
       return this.clusterInfo;
     }
     }
+
+    @Override
+    public Set<String> getBlacklistedNodes() {
+      return ((RMContainerRequestor) containerAllocator).getBlacklistedNodes();
+    }
   }
   }
 
 
   @SuppressWarnings("unchecked")
   @SuppressWarnings("unchecked")

+ 4 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java

@@ -424,4 +424,8 @@ public abstract class RMContainerRequestor extends RMCommunicator {
         hosts, orig.racks, orig.priority); 
         hosts, orig.racks, orig.priority); 
     return newReq;
     return newReq;
   }
   }
+  
+  public Set<String> getBlacklistedNodes() {
+    return blacklistedNodes;
+  }
 }
 }

+ 9 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AMWebServices.java

@@ -45,6 +45,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
 import org.apache.hadoop.mapreduce.v2.app.webapp.dao.AppInfo;
 import org.apache.hadoop.mapreduce.v2.app.webapp.dao.AppInfo;
 import org.apache.hadoop.mapreduce.v2.app.webapp.dao.AMAttemptInfo;
 import org.apache.hadoop.mapreduce.v2.app.webapp.dao.AMAttemptInfo;
 import org.apache.hadoop.mapreduce.v2.app.webapp.dao.AMAttemptsInfo;
 import org.apache.hadoop.mapreduce.v2.app.webapp.dao.AMAttemptsInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.BlacklistedNodesInfo;
 import org.apache.hadoop.mapreduce.v2.app.webapp.dao.ConfInfo;
 import org.apache.hadoop.mapreduce.v2.app.webapp.dao.ConfInfo;
 import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobCounterInfo;
 import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobCounterInfo;
 import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobInfo;
 import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobInfo;
@@ -216,6 +217,14 @@ public class AMWebServices {
     init();
     init();
     return new AppInfo(this.app, this.app.context);
     return new AppInfo(this.app, this.app.context);
   }
   }
+  
+  @GET
+  @Path("/blacklistednodes")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public BlacklistedNodesInfo getBlacklistedNodes() {
+    init();
+    return new BlacklistedNodesInfo(this.app.context);
+  }
 
 
   @GET
   @GET
   @Path("/jobs")
   @Path("/jobs")

+ 42 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/BlacklistedNodesInfo.java

@@ -0,0 +1,42 @@
+/**
+ * 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.mapreduce.v2.app.webapp.dao;
+
+import java.util.Set;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.hadoop.mapreduce.v2.app.AppContext;
+
+@XmlRootElement(name = "blacklistednodesinfo")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class BlacklistedNodesInfo {
+  private Set<String> blacklistedNodes;
+  
+  public BlacklistedNodesInfo() { }
+  
+  public BlacklistedNodesInfo(AppContext appContext) {
+    blacklistedNodes = appContext.getBlacklistedNodes();
+  }
+  
+  public Set<String> getBlacklistedNodes() {
+    return blacklistedNodes;
+  }
+}

+ 12 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockAppContext.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.mapreduce.v2.app;
 package org.apache.hadoop.mapreduce.v2.app;
 
 
 import java.util.Map;
 import java.util.Map;
+import java.util.Set;
 
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
@@ -37,7 +38,8 @@ public class MockAppContext implements AppContext {
   final String user = MockJobs.newUserName();
   final String user = MockJobs.newUserName();
   final Map<JobId, Job> jobs;
   final Map<JobId, Job> jobs;
   final long startTime = System.currentTimeMillis();
   final long startTime = System.currentTimeMillis();
-
+  Set<String> blacklistedNodes;
+  
   public MockAppContext(int appid) {
   public MockAppContext(int appid) {
     appID = MockJobs.newAppID(appid);
     appID = MockJobs.newAppID(appid);
     appAttemptID = ApplicationAttemptId.newInstance(appID, 0);
     appAttemptID = ApplicationAttemptId.newInstance(appID, 0);
@@ -115,4 +117,13 @@ public class MockAppContext implements AppContext {
     return null;
     return null;
   }
   }
 
 
+  @Override
+  public Set<String> getBlacklistedNodes() {
+    return blacklistedNodes;
+  }
+  
+  public void setBlacklistedNodes(Set<String> blacklistedNodes) {
+    this.blacklistedNodes = blacklistedNodes;
+  }
+
 }
 }

+ 6 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java

@@ -24,6 +24,7 @@ import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicLong;
@@ -850,5 +851,10 @@ public class TestRuntimeEstimators {
     public ClusterInfo getClusterInfo() {
     public ClusterInfo getClusterInfo() {
       return new ClusterInfo();
       return new ClusterInfo();
     }
     }
+
+    @Override
+    public Set<String> getBlacklistedNodes() {
+      return null;
+    }
   }
   }
 }
 }

+ 58 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServices.java

@@ -23,6 +23,7 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.junit.Assert.fail;
 
 
 import java.io.StringReader;
 import java.io.StringReader;
+import java.util.Set;
 
 
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.MediaType;
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilder;
@@ -33,6 +34,7 @@ import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.MockAppContext;
 import org.apache.hadoop.mapreduce.v2.app.MockAppContext;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
 import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
 import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
+import org.codehaus.jettison.json.JSONArray;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
 import org.codehaus.jettison.json.JSONObject;
 import org.junit.Before;
 import org.junit.Before;
@@ -42,6 +44,7 @@ import org.w3c.dom.Element;
 import org.w3c.dom.NodeList;
 import org.w3c.dom.NodeList;
 import org.xml.sax.InputSource;
 import org.xml.sax.InputSource;
 
 
+import com.google.common.collect.Sets;
 import com.google.inject.Guice;
 import com.google.inject.Guice;
 import com.google.inject.Injector;
 import com.google.inject.Injector;
 import com.google.inject.servlet.GuiceServletContextListener;
 import com.google.inject.servlet.GuiceServletContextListener;
@@ -64,13 +67,15 @@ import com.sun.jersey.test.framework.WebAppDescriptor;
 public class TestAMWebServices extends JerseyTest {
 public class TestAMWebServices extends JerseyTest {
 
 
   private static Configuration conf = new Configuration();
   private static Configuration conf = new Configuration();
-  private static AppContext appContext;
+  private static MockAppContext appContext;
 
 
   private Injector injector = Guice.createInjector(new ServletModule() {
   private Injector injector = Guice.createInjector(new ServletModule() {
     @Override
     @Override
     protected void configureServlets() {
     protected void configureServlets() {
 
 
       appContext = new MockAppContext(0, 1, 1, 1);
       appContext = new MockAppContext(0, 1, 1, 1);
+      appContext.setBlacklistedNodes(Sets.newHashSet("badnode1", "badnode2"));
+      
       bind(JAXBContextResolver.class);
       bind(JAXBContextResolver.class);
       bind(AMWebServices.class);
       bind(AMWebServices.class);
       bind(GenericExceptionHandler.class);
       bind(GenericExceptionHandler.class);
@@ -240,6 +245,29 @@ public class TestAMWebServices extends JerseyTest {
           "error string exists and shouldn't", "", responseStr);
           "error string exists and shouldn't", "", responseStr);
     }
     }
   }
   }
+  
+  @Test
+  public void testBlacklistedNodes() throws JSONException, Exception {
+    WebResource r = resource();
+    ClientResponse response = r.path("ws").path("v1").path("mapreduce")
+        .path("blacklistednodes").accept(MediaType.APPLICATION_JSON)
+        .get(ClientResponse.class);
+    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    JSONObject json = response.getEntity(JSONObject.class);
+    assertEquals("incorrect number of elements", 1, json.length());
+    verifyBlacklistedNodesInfo(json, appContext);
+  }
+  
+  @Test
+  public void testBlacklistedNodesXML() throws Exception {
+    WebResource r = resource();
+    ClientResponse response = r.path("ws").path("v1").path("mapreduce")
+        .path("blacklistednodes").accept(MediaType.APPLICATION_XML)
+        .get(ClientResponse.class);
+    assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+    String xml = response.getEntity(String.class);
+    verifyBlacklistedNodesInfoXML(xml, appContext);
+  }
 
 
   public void verifyAMInfo(JSONObject info, AppContext ctx)
   public void verifyAMInfo(JSONObject info, AppContext ctx)
       throws JSONException {
       throws JSONException {
@@ -285,4 +313,33 @@ public class TestAMWebServices extends JerseyTest {
     assertTrue("elapsedTime not greater then 0", (elapsedTime > 0));
     assertTrue("elapsedTime not greater then 0", (elapsedTime > 0));
 
 
   }
   }
+  
+  public void verifyBlacklistedNodesInfo(JSONObject blacklist, AppContext ctx)
+    throws JSONException, Exception{
+    JSONArray array = blacklist.getJSONArray("blacklistedNodes");
+    assertEquals(array.length(), ctx.getBlacklistedNodes().size());
+    for (int i = 0; i < array.length(); i++) {
+      assertTrue(ctx.getBlacklistedNodes().contains(array.getString(i)));
+    }
+  }
+  
+  public void verifyBlacklistedNodesInfoXML(String xml, AppContext ctx)
+      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 infonodes = dom.getElementsByTagName("blacklistednodesinfo");
+    assertEquals("incorrect number of elements", 1, infonodes.getLength());
+    NodeList nodes = dom.getElementsByTagName("blacklistedNodes");
+    Set<String> blacklistedNodes = ctx.getBlacklistedNodes();
+    assertEquals("incorrect number of elements", blacklistedNodes.size(),
+        nodes.getLength());
+    for (int i = 0; i < nodes.getLength(); i++) {
+      Element element = (Element) nodes.item(i);
+      assertTrue(
+          blacklistedNodes.contains(element.getFirstChild().getNodeValue()));
+    }
+  }
 }
 }

+ 7 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.mapreduce.v2.hs;
 import java.io.IOException;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeUnit;
 import java.util.regex.Pattern;
 import java.util.regex.Pattern;
@@ -301,4 +302,10 @@ public class JobHistory extends AbstractService implements HistoryContext {
   public ClusterInfo getClusterInfo() {
   public ClusterInfo getClusterInfo() {
     return null;
     return null;
   }
   }
+
+  // TODO AppContext - Not Required
+  @Override
+  public Set<String> getBlacklistedNodes() {
+    return null;
+  }
 }
 }