瀏覽代碼

Merge -c 1241205 from trunk to branch-0.23 to fix MAPREDUCE-3810. Performance tweaks - reduced logging in AM and defined hascode/equals for ResourceRequest & Priority.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.23@1241208 13f79535-47bb-0310-9956-ffa450edef68
Arun Murthy 13 年之前
父節點
當前提交
caea6d2e85
共有 12 個文件被更改,包括 329 次插入124 次删除
  1. 3 0
      hadoop-mapreduce-project/CHANGES.txt
  2. 8 10
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
  3. 8 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java
  4. 8 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java
  5. 63 34
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
  6. 39 23
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java
  7. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java
  8. 93 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRAppBenchmark.java
  9. 28 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Priority.java
  10. 76 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java
  11. 1 12
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PriorityPBImpl.java
  12. 1 27
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceRequestPBImpl.java

+ 3 - 0
hadoop-mapreduce-project/CHANGES.txt

@@ -665,6 +665,9 @@ Release 0.23.1 - Unreleased
 
     MAPREDUCE-3814. Fixed MRV1 compilation. (Arun C Murthy via vinodkv)
 
+    MAPREDUCE-3810. Performance tweaks - reduced logging in AM and defined
+    hascode/equals for ResourceRequest & Priority. (vinodkv via acmurthy) 
+
 Release 0.23.0 - 2011-11-01 
 
   INCOMPATIBLE CHANGES

+ 8 - 10
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java

@@ -127,7 +127,7 @@ import org.apache.hadoop.yarn.util.RackResolver;
 /**
  * Implementation of TaskAttempt interface.
  */
-@SuppressWarnings({ "rawtypes", "deprecation" })
+@SuppressWarnings({ "rawtypes" })
 public abstract class TaskAttemptImpl implements
     org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt,
       EventHandler<TaskAttemptEvent> {
@@ -910,8 +910,10 @@ public abstract class TaskAttemptImpl implements
   @SuppressWarnings("unchecked")
   @Override
   public void handle(TaskAttemptEvent event) {
-    LOG.info("Processing " + event.getTaskAttemptID() +
-        " of type " + event.getType());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Processing " + event.getTaskAttemptID() + " of type "
+          + event.getType());
+    }
     writeLock.lock();
     try {
       final TaskAttemptState oldState = getState();
@@ -1278,15 +1280,11 @@ public abstract class TaskAttemptImpl implements
         TaskAttemptEvent event) {
       //set the finish time
       taskAttempt.setFinishTime();
-      String taskType = 
-          TypeConverter.fromYarn(taskAttempt.attemptId.getTaskId().getTaskType()).toString();
-      LOG.info("In TaskAttemptImpl taskType: " + taskType);
       long slotMillis = computeSlotMillis(taskAttempt);
-      JobCounterUpdateEvent jce =
-          new JobCounterUpdateEvent(taskAttempt.attemptId.getTaskId()
-              .getJobId());
+      TaskId taskId = taskAttempt.attemptId.getTaskId();
+      JobCounterUpdateEvent jce = new JobCounterUpdateEvent(taskId.getJobId());
       jce.addCounterUpdate(
-        taskAttempt.attemptId.getTaskId().getTaskType() == TaskType.MAP ? 
+        taskId.getTaskType() == TaskType.MAP ? 
           JobCounter.SLOTS_MILLIS_MAPS : JobCounter.SLOTS_MILLIS_REDUCES,
           slotMillis);
       taskAttempt.eventHandler.handle(jce);

+ 8 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java

@@ -81,7 +81,7 @@ import org.apache.hadoop.yarn.state.StateMachineFactory;
 /**
  * Implementation of Task interface.
  */
-@SuppressWarnings({ "rawtypes", "unchecked", "deprecation" })
+@SuppressWarnings({ "rawtypes", "unchecked" })
 public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
 
   private static final Log LOG = LogFactory.getLog(TaskImpl.class);
@@ -505,7 +505,9 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
   // This is always called in the Write Lock
   private void addAndScheduleAttempt() {
     TaskAttempt attempt = createAttempt();
-    LOG.info("Created attempt " + attempt.getID());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Created attempt " + attempt.getID());
+    }
     switch (attempts.size()) {
       case 0:
         attempts = Collections.singletonMap(attempt.getID(), attempt);
@@ -537,7 +539,10 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
 
   @Override
   public void handle(TaskEvent event) {
-    LOG.debug("Processing " + event.getTaskID() + " of type " + event.getType());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Processing " + event.getTaskID() + " of type "
+          + event.getType());
+    }
     try {
       writeLock.lock();
       TaskState oldState = getState();

+ 8 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java

@@ -30,7 +30,6 @@ import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.TypeConverter;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
-import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
 import org.apache.hadoop.mapreduce.v2.api.records.JobState;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
@@ -46,9 +45,9 @@ import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -149,7 +148,7 @@ public abstract class RMCommunicator extends AbstractService  {
       LOG.info("minContainerCapability: " + minContainerCapability.getMemory());
       LOG.info("maxContainerCapability: " + maxContainerCapability.getMemory());
     } catch (Exception are) {
-      LOG.info("Exception while registering", are);
+      LOG.error("Exception while registering", are);
       throw new YarnException(are);
     }
   }
@@ -183,7 +182,7 @@ public abstract class RMCommunicator extends AbstractService  {
       request.setTrackingUrl(historyUrl);
       scheduler.finishApplicationMaster(request);
     } catch(Exception are) {
-      LOG.info("Exception while unregistering ", are);
+      LOG.error("Exception while unregistering ", are);
     }
   }
 
@@ -205,7 +204,7 @@ public abstract class RMCommunicator extends AbstractService  {
     try {
       allocatorThread.join();
     } catch (InterruptedException ie) {
-      LOG.info("InterruptedException while stopping", ie);
+      LOG.warn("InterruptedException while stopping", ie);
     }
     unregister();
     super.stop();
@@ -228,7 +227,7 @@ public abstract class RMCommunicator extends AbstractService  {
               // TODO: for other exceptions
             }
           } catch (InterruptedException e) {
-            LOG.info("Allocated thread interrupted. Returning.");
+            LOG.warn("Allocated thread interrupted. Returning.");
             return;
           }
         }
@@ -255,7 +254,9 @@ public abstract class RMCommunicator extends AbstractService  {
     if (UserGroupInformation.isSecurityEnabled()) {
       String tokenURLEncodedStr = System.getenv().get(
           ApplicationConstants.APPLICATION_MASTER_TOKEN_ENV_NAME);
-      LOG.debug("AppMasterToken is " + tokenURLEncodedStr);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("AppMasterToken is " + tokenURLEncodedStr);
+      }
       Token<? extends TokenIdentifier> token = new Token<TokenIdentifier>();
 
       try {

+ 63 - 34
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java

@@ -41,6 +41,7 @@ import org.apache.hadoop.mapreduce.JobCounter;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent;
 import org.apache.hadoop.mapreduce.jobhistory.NormalizedResourceEvent;
+import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
@@ -254,28 +255,30 @@ public class RMContainerAllocator extends RMContainerRequestor
 
   @SuppressWarnings({ "unchecked" })
   protected synchronized void handleEvent(ContainerAllocatorEvent event) {
-    LOG.info("Processing the event " + event.toString());
     recalculateReduceSchedule = true;
     if (event.getType() == ContainerAllocator.EventType.CONTAINER_REQ) {
       ContainerRequestEvent reqEvent = (ContainerRequestEvent) event;
+      JobId jobId = getJob().getID();
+      int supportedMaxContainerCapability =
+          getMaxContainerCapability().getMemory();
       if (reqEvent.getAttemptID().getTaskId().getTaskType().equals(TaskType.MAP)) {
         if (mapResourceReqt == 0) {
           mapResourceReqt = reqEvent.getCapability().getMemory();
           int minSlotMemSize = getMinContainerCapability().getMemory();
           mapResourceReqt = (int) Math.ceil((float) mapResourceReqt/minSlotMemSize)
               * minSlotMemSize;
-          eventHandler.handle(new JobHistoryEvent(getJob().getID(), 
+          eventHandler.handle(new JobHistoryEvent(jobId, 
               new NormalizedResourceEvent(org.apache.hadoop.mapreduce.TaskType.MAP,
               mapResourceReqt)));
           LOG.info("mapResourceReqt:"+mapResourceReqt);
-          if (mapResourceReqt > getMaxContainerCapability().getMemory()) {
+          if (mapResourceReqt > supportedMaxContainerCapability) {
             String diagMsg = "MAP capability required is more than the supported " +
             "max container capability in the cluster. Killing the Job. mapResourceReqt: " + 
-            mapResourceReqt + " maxContainerCapability:" + getMaxContainerCapability().getMemory();
+            mapResourceReqt + " maxContainerCapability:" + supportedMaxContainerCapability;
             LOG.info(diagMsg);
             eventHandler.handle(new JobDiagnosticsUpdateEvent(
-                getJob().getID(), diagMsg));
-            eventHandler.handle(new JobEvent(getJob().getID(), JobEventType.JOB_KILL));
+                jobId, diagMsg));
+            eventHandler.handle(new JobEvent(jobId, JobEventType.JOB_KILL));
           }
         }
         //set the rounded off memory
@@ -288,20 +291,20 @@ public class RMContainerAllocator extends RMContainerRequestor
           //round off on slotsize
           reduceResourceReqt = (int) Math.ceil((float) 
               reduceResourceReqt/minSlotMemSize) * minSlotMemSize;
-          eventHandler.handle(new JobHistoryEvent(getJob().getID(), 
+          eventHandler.handle(new JobHistoryEvent(jobId, 
               new NormalizedResourceEvent(
                   org.apache.hadoop.mapreduce.TaskType.REDUCE,
               reduceResourceReqt)));
           LOG.info("reduceResourceReqt:"+reduceResourceReqt);
-          if (reduceResourceReqt > getMaxContainerCapability().getMemory()) {
+          if (reduceResourceReqt > supportedMaxContainerCapability) {
             String diagMsg = "REDUCE capability required is more than the " +
             		"supported max container capability in the cluster. Killing the " +
             		"Job. reduceResourceReqt: " + reduceResourceReqt +
-            		" maxContainerCapability:" + getMaxContainerCapability().getMemory();
+            		" maxContainerCapability:" + supportedMaxContainerCapability;
             LOG.info(diagMsg);
             eventHandler.handle(new JobDiagnosticsUpdateEvent(
-                getJob().getID(), diagMsg));
-            eventHandler.handle(new JobEvent(getJob().getID(), JobEventType.JOB_KILL));
+                jobId, diagMsg));
+            eventHandler.handle(new JobEvent(jobId, JobEventType.JOB_KILL));
           }
         }
         //set the rounded off memory
@@ -317,6 +320,9 @@ public class RMContainerAllocator extends RMContainerRequestor
       
     } else if (
         event.getType() == ContainerAllocator.EventType.CONTAINER_DEALLOCATE) {
+  
+      LOG.info("Processing the event " + event.toString());
+
       TaskAttemptId aId = event.getAttemptID();
       
       boolean removed = scheduledRequests.remove(aId);
@@ -579,7 +585,7 @@ public class RMContainerAllocator extends RMContainerRequestor
     computeIgnoreBlacklisting();
     
     for (ContainerStatus cont : finishedContainers) {
-      LOG.info("Received completed container " + cont);
+      LOG.info("Received completed container " + cont.getContainerId());
       TaskAttemptId attemptID = assignedRequests.get(cont.getContainerId());
       if (attemptID == null) {
         LOG.error("Container complete event for unknown container id "
@@ -664,7 +670,9 @@ public class RMContainerAllocator extends RMContainerRequestor
             mapsHostMapping.put(host, list);
           }
           list.add(event.getAttemptID());
-          LOG.info("Added attempt req to host " + host);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Added attempt req to host " + host);
+          }
        }
        for (String rack: event.getRacks()) {
          LinkedList<TaskAttemptId> list = mapsRackMapping.get(rack);
@@ -673,7 +681,9 @@ public class RMContainerAllocator extends RMContainerRequestor
            mapsRackMapping.put(rack, list);
          }
          list.add(event.getAttemptID());
-         LOG.info("Added attempt req to rack " + rack);
+         if (LOG.isDebugEnabled()) {
+            LOG.debug("Added attempt req to rack " + rack);
+         }
        }
        request = new ContainerRequest(event, PRIORITY_MAP);
       }
@@ -694,18 +704,21 @@ public class RMContainerAllocator extends RMContainerRequestor
       containersAllocated += allocatedContainers.size();
       while (it.hasNext()) {
         Container allocated = it.next();
-        LOG.info("Assigning container " + allocated.getId() +
-            " with priority " + allocated.getPriority() +
-            " to NM " + allocated.getNodeId());
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Assigning container " + allocated.getId()
+              + " with priority " + allocated.getPriority() + " to NM "
+              + allocated.getNodeId());
+        }
         
         // check if allocated container meets memory requirements 
         // and whether we have any scheduled tasks that need 
         // a container to be assigned
         boolean isAssignable = true;
         Priority priority = allocated.getPriority();
+        int allocatedMemory = allocated.getResource().getMemory();
         if (PRIORITY_FAST_FAIL_MAP.equals(priority) 
             || PRIORITY_MAP.equals(priority)) {
-          if (allocated.getResource().getMemory() < mapResourceReqt
+          if (allocatedMemory < mapResourceReqt
               || maps.isEmpty()) {
             LOG.info("Cannot assign container " + allocated 
                 + " for a map as either "
@@ -716,7 +729,7 @@ public class RMContainerAllocator extends RMContainerRequestor
           }
         } 
         else if (PRIORITY_REDUCE.equals(priority)) {
-          if (allocated.getResource().getMemory() < reduceResourceReqt
+          if (allocatedMemory < reduceResourceReqt
               || reduces.isEmpty()) {
             LOG.info("Cannot assign container " + allocated 
                 + " for a reduce as either "
@@ -730,15 +743,17 @@ public class RMContainerAllocator extends RMContainerRequestor
         boolean blackListed = false;         
         ContainerRequest assigned = null;
         
+        ContainerId allocatedContainerId = allocated.getId();
         if (isAssignable) {
           // do not assign if allocated container is on a  
           // blacklisted host
-          blackListed = isNodeBlacklisted(allocated.getNodeId().getHost());
+          String allocatedHost = allocated.getNodeId().getHost();
+          blackListed = isNodeBlacklisted(allocatedHost);
           if (blackListed) {
             // we need to request for a new container 
             // and release the current one
             LOG.info("Got allocated container on a blacklisted "
-                + " host "+allocated.getNodeId().getHost()
+                + " host "+allocatedHost
                 +". Releasing container " + allocated);
 
             // find the request matching this allocated container 
@@ -775,11 +790,13 @@ public class RMContainerAllocator extends RMContainerRequestor
               eventHandler.handle(new TaskAttemptContainerAssignedEvent(
                   assigned.attemptID, allocated, applicationACLs));
 
-              assignedRequests.add(allocated.getId(), assigned.attemptID);
+              assignedRequests.add(allocatedContainerId, assigned.attemptID);
 
-              LOG.info("Assigned container (" + allocated + ") " +
-                  " to task " + assigned.attemptID +
-                  " on node " + allocated.getNodeId().toString());
+              if (LOG.isDebugEnabled()) {
+                LOG.info("Assigned container (" + allocated + ") "
+                    + " to task " + assigned.attemptID + " on node "
+                    + allocated.getNodeId().toString());
+              }
             }
             else {
               //not assigned to any request, release the container
@@ -794,7 +811,7 @@ public class RMContainerAllocator extends RMContainerRequestor
         // or if we could not assign it 
         if (blackListed || assigned == null) {
           containersReleased++;
-          release(allocated.getId());
+          release(allocatedContainerId);
         }
       }
     }
@@ -807,10 +824,14 @@ public class RMContainerAllocator extends RMContainerRequestor
         LOG.info("Assigning container " + allocated + " to fast fail map");
         assigned = assignToFailedMap(allocated);
       } else if (PRIORITY_REDUCE.equals(priority)) {
-        LOG.info("Assigning container " + allocated + " to reduce");
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Assigning container " + allocated + " to reduce");
+        }
         assigned = assignToReduce(allocated);
       } else if (PRIORITY_MAP.equals(priority)) {
-        LOG.info("Assigning container " + allocated + " to map");
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Assigning container " + allocated + " to map");
+        }
         assigned = assignToMap(allocated);
       } else {
         LOG.warn("Container allocated at unwanted priority: " + priority + 
@@ -897,7 +918,9 @@ public class RMContainerAllocator extends RMContainerRequestor
         String host = allocated.getNodeId().getHost();
         LinkedList<TaskAttemptId> list = mapsHostMapping.get(host);
         while (list != null && list.size() > 0) {
-          LOG.info("Host matched to the request list " + host);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Host matched to the request list " + host);
+          }
           TaskAttemptId tId = list.removeFirst();
           if (maps.containsKey(tId)) {
             assigned = maps.remove(tId);
@@ -906,7 +929,9 @@ public class RMContainerAllocator extends RMContainerRequestor
             jce.addCounterUpdate(JobCounter.DATA_LOCAL_MAPS, 1);
             eventHandler.handle(jce);
             hostLocalAssigned++;
-            LOG.info("Assigned based on host match " + host);
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Assigned based on host match " + host);
+            }
             break;
           }
         }
@@ -922,7 +947,9 @@ public class RMContainerAllocator extends RMContainerRequestor
               jce.addCounterUpdate(JobCounter.RACK_LOCAL_MAPS, 1);
               eventHandler.handle(jce);
               rackLocalAssigned++;
-              LOG.info("Assigned based on rack match " + rack);
+              if (LOG.isDebugEnabled()) {
+                LOG.debug("Assigned based on rack match " + rack);
+              }
               break;
             }
           }
@@ -933,7 +960,9 @@ public class RMContainerAllocator extends RMContainerRequestor
               new JobCounterUpdateEvent(assigned.attemptID.getTaskId().getJobId());
             jce.addCounterUpdate(JobCounter.OTHER_LOCAL_MAPS, 1);
             eventHandler.handle(jce);
-            LOG.info("Assigned based on * match");
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Assigned based on * match");
+            }
             break;
           }
         }
@@ -953,8 +982,7 @@ public class RMContainerAllocator extends RMContainerRequestor
       new HashSet<TaskAttemptId>();
     
     void add(ContainerId containerId, TaskAttemptId tId) {
-      LOG.info("Assigned container " + containerId.toString()
-          + " to " + tId);
+      LOG.info("Assigned container " + containerId.toString() + " to " + tId);
       containerToAttemptMap.put(containerId, tId);
       if (tId.getTaskId().getTaskType().equals(TaskType.MAP)) {
         maps.put(tId, containerId);
@@ -963,6 +991,7 @@ public class RMContainerAllocator extends RMContainerRequestor
       }
     }
 
+    @SuppressWarnings("unchecked")
     void preemptReduce(int toPreempt) {
       List<TaskAttemptId> reduceList = new ArrayList<TaskAttemptId>
         (reduces.keySet());

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

@@ -155,13 +155,14 @@ public abstract class RMContainerRequestor extends RMCommunicator {
     lastClusterNmCount = clusterNmCount;
     clusterNmCount = allocateResponse.getNumClusterNodes();
 
-    LOG.info("getResources() for " + applicationId + ":" + " ask="
-        + ask.size() + " release= " + release.size() + 
-        " newContainers=" + response.getAllocatedContainers().size() + 
-        " finishedContainers=" + 
-        response.getCompletedContainersStatuses().size() + 
-        " resourcelimit=" + availableResources + 
-        " knownNMs=" + clusterNmCount);
+    if (ask.size() > 0 || release.size() > 0) {
+      LOG.info("getResources() for " + applicationId + ":" + " ask="
+          + ask.size() + " release= " + release.size() + " newContainers="
+          + response.getAllocatedContainers().size() + " finishedContainers="
+          + response.getCompletedContainersStatuses().size()
+          + " resourcelimit=" + availableResources + " knownNMs="
+          + clusterNmCount);
+    }
 
     ask.clear();
     release.clear();
@@ -172,6 +173,9 @@ public abstract class RMContainerRequestor extends RMCommunicator {
   // knownNodeCount is based on node managers, not hosts. blacklisting is
   // currently based on hosts.
   protected void computeIgnoreBlacklisting() {
+    if (!nodeBlacklistingEnabled) {
+      return;
+    }
     if (blacklistDisablePercent != -1
         && (blacklistedNodeCount != blacklistedNodes.size() ||
             clusterNmCount != lastClusterNmCount)) {
@@ -200,7 +204,9 @@ public abstract class RMContainerRequestor extends RMCommunicator {
       return;
     }
     if (blacklistedNodes.contains(hostName)) {
-      LOG.info("Host " + hostName + " is already blacklisted.");
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Host " + hostName + " is already blacklisted.");
+      }
       return; //already blacklisted
     }
     Integer failures = nodeFailures.remove(hostName);
@@ -293,7 +299,9 @@ public abstract class RMContainerRequestor extends RMCommunicator {
     if (remoteRequests == null) {
       remoteRequests = new HashMap<String, Map<Resource, ResourceRequest>>();
       this.remoteRequestsTable.put(priority, remoteRequests);
-      LOG.info("Added priority=" + priority);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Added priority=" + priority);
+      }
     }
     Map<Resource, ResourceRequest> reqMap = remoteRequests.get(resourceName);
     if (reqMap == null) {
@@ -313,10 +321,12 @@ public abstract class RMContainerRequestor extends RMCommunicator {
 
     // Note this down for next interaction with ResourceManager
     ask.add(remoteRequest);
-    LOG.info("addResourceRequest:" + " applicationId=" + applicationId.getId()
-        + " priority=" + priority.getPriority() + " resourceName=" + resourceName
-        + " numContainers=" + remoteRequest.getNumContainers() + " #asks="
-        + ask.size());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("addResourceRequest:" + " applicationId="
+          + applicationId.getId() + " priority=" + priority.getPriority()
+          + " resourceName=" + resourceName + " numContainers="
+          + remoteRequest.getNumContainers() + " #asks=" + ask.size());
+    }
   }
 
   private void decResourceRequest(Priority priority, String resourceName,
@@ -328,16 +338,20 @@ public abstract class RMContainerRequestor extends RMCommunicator {
       // as we modify the resource requests by filtering out blacklisted hosts 
       // when they are added, this value may be null when being 
       // decremented
-      LOG.debug("Not decrementing resource as " + resourceName
-          + " is not present in request table");
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Not decrementing resource as " + resourceName
+            + " is not present in request table");
+      }
       return;
     }
     ResourceRequest remoteRequest = reqMap.get(capability);
 
-    LOG.info("BEFORE decResourceRequest:" + " applicationId=" + applicationId.getId()
-        + " priority=" + priority.getPriority() + " resourceName=" + resourceName
-        + " numContainers=" + remoteRequest.getNumContainers() + " #asks="
-        + ask.size());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("BEFORE decResourceRequest:" + " applicationId="
+          + applicationId.getId() + " priority=" + priority.getPriority()
+          + " resourceName=" + resourceName + " numContainers="
+          + remoteRequest.getNumContainers() + " #asks=" + ask.size());
+    }
 
     remoteRequest.setNumContainers(remoteRequest.getNumContainers() -1);
     if (remoteRequest.getNumContainers() == 0) {
@@ -355,10 +369,12 @@ public abstract class RMContainerRequestor extends RMCommunicator {
       //already have it.
     }
 
-    LOG.info("AFTER decResourceRequest:" + " applicationId="
-             + applicationId.getId() + " priority=" + priority.getPriority()
-             + " resourceName=" + resourceName + " numContainers="
-             + remoteRequest.getNumContainers() + " #asks=" + ask.size());
+    if (LOG.isDebugEnabled()) {
+      LOG.info("AFTER decResourceRequest:" + " applicationId="
+          + applicationId.getId() + " priority=" + priority.getPriority()
+          + " resourceName=" + resourceName + " numContainers="
+          + remoteRequest.getNumContainers() + " #asks=" + ask.size());
+    }
   }
 
   protected void release(ContainerId containerId) {

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

@@ -436,7 +436,7 @@ public class MRApp extends MRAppMaster {
     return new ClientService(){
       @Override
       public InetSocketAddress getBindAddress() {
-        return null;
+        return NetUtils.createSocketAddr("localhost:9876");
       }
 
       @Override

+ 93 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRAppBenchmark.java

@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.mapreduce.v2.app;
 
+import java.util.ArrayList;
+import java.util.List;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
 
@@ -29,16 +31,30 @@ import org.apache.hadoop.mapreduce.v2.app.job.Job;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerAssignedEvent;
 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.RMContainerAllocator;
 import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.api.AMRMProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.records.AMResponse;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.service.AbstractService;
+import org.apache.hadoop.yarn.util.BuilderUtils;
+import org.apache.hadoop.yarn.util.Records;
 import org.apache.log4j.Level;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
+import org.junit.Test;
 
 public class MRAppBenchmark {
 
@@ -167,17 +183,89 @@ public class MRAppBenchmark {
     }
   }
 
+  @Test
   public void benchmark1() throws Exception {
-    int maps = 100000;
-    int reduces = 100;
+    int maps = 100; // Adjust for benchmarking. Start with thousands.
+    int reduces = 0;
     System.out.println("Running benchmark with maps:"+maps +
         " reduces:"+reduces);
-    run(new MRApp(maps, reduces, true, this.getClass().getName(), true));
+    run(new MRApp(maps, reduces, true, this.getClass().getName(), true) {
+
+      @Override
+      protected ContainerAllocator createContainerAllocator(
+          ClientService clientService, AppContext context) {
+        return new RMContainerAllocator(clientService, context) {
+          @Override
+          protected AMRMProtocol createSchedulerProxy() {
+            return new AMRMProtocol() {
+
+              @Override
+              public RegisterApplicationMasterResponse
+                  registerApplicationMaster(
+                      RegisterApplicationMasterRequest request)
+                      throws YarnRemoteException {
+                RegisterApplicationMasterResponse response =
+                    Records.newRecord(RegisterApplicationMasterResponse.class);
+                response.setMinimumResourceCapability(BuilderUtils
+                  .newResource(1024));
+                response.setMaximumResourceCapability(BuilderUtils
+                  .newResource(10240));
+                return response;
+              }
+
+              @Override
+              public FinishApplicationMasterResponse finishApplicationMaster(
+                  FinishApplicationMasterRequest request)
+                  throws YarnRemoteException {
+                FinishApplicationMasterResponse response =
+                    Records.newRecord(FinishApplicationMasterResponse.class);
+                return response;
+              }
+
+              @Override
+              public AllocateResponse allocate(AllocateRequest request)
+                  throws YarnRemoteException {
+
+                AllocateResponse response =
+                    Records.newRecord(AllocateResponse.class);
+                List<ResourceRequest> askList = request.getAskList();
+                List<Container> containers = new ArrayList<Container>();
+                for (ResourceRequest req : askList) {
+                  if (req.getHostName() != "*") {
+                    continue;
+                  }
+                  int numContainers = req.getNumContainers();
+                  for (int i = 0; i < numContainers; i++) {
+                    ContainerId containerId =
+                        BuilderUtils.newContainerId(
+                          request.getApplicationAttemptId(),
+                          request.getResponseId() + i);
+                    containers.add(BuilderUtils
+                      .newContainer(containerId, BuilderUtils.newNodeId("host"
+                          + containerId.getId(), 2345),
+                        "host" + containerId.getId() + ":5678", req
+                          .getCapability(), req.getPriority(), null));
+                  }
+                }
+
+                AMResponse amResponse = Records.newRecord(AMResponse.class);
+                amResponse.setAllocatedContainers(containers);
+                amResponse.setResponseId(request.getResponseId() + 1);
+                response.setAMResponse(amResponse);
+                response.setNumClusterNodes(350);
+                return response;
+              }
+            };
+          }
+        };
+      }
+    });
   }
 
+  @Test
   public void benchmark2() throws Exception {
-    int maps = 4000;
-    int reduces = 1000;
+    int maps = 100; // Adjust for benchmarking, start with a couple of thousands
+    int reduces = 50;
     int maxConcurrentRunningTasks = 500;
     
     System.out.println("Running benchmark with throttled running tasks with " +

+ 28 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Priority.java

@@ -23,7 +23,7 @@ package org.apache.hadoop.yarn.api.records;
  * allocation 
  *
  */
-public interface Priority extends Comparable<Priority> {
+public abstract class Priority implements Comparable<Priority> {
   
   /**
    * Get the assigned priority
@@ -37,4 +37,31 @@ public interface Priority extends Comparable<Priority> {
    */
   public abstract void setPriority(int priority);
   
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + getPriority();
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    Priority other = (Priority) obj;
+    if (getPriority() != other.getPriority())
+      return false;
+    return true;
+  }
+
+  @Override
+  public int compareTo(Priority other) {
+    return this.getPriority() - other.getPriority();
+  }
+
 }

+ 76 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java

@@ -48,7 +48,7 @@ import org.apache.hadoop.yarn.api.AMRMProtocol;
  */
 @Public
 @Stable
-public interface ResourceRequest extends Comparable<ResourceRequest> {
+public abstract class ResourceRequest implements Comparable<ResourceRequest> {
   /**
    * Get the <code>Priority</code> of the request.
    * @return <code>Priority</code> of the request
@@ -121,4 +121,79 @@ public interface ResourceRequest extends Comparable<ResourceRequest> {
   @Public
   @Stable
   public abstract void setNumContainers(int numContainers);
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    Resource capability = getCapability();
+    String hostName = getHostName();
+    Priority priority = getPriority();
+    result =
+        prime * result + ((capability == null) ? 0 : capability.hashCode());
+    result = prime * result + ((hostName == null) ? 0 : hostName.hashCode());
+    result = prime * result + getNumContainers();
+    result = prime * result + ((priority == null) ? 0 : priority.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    ResourceRequest other = (ResourceRequest) obj;
+    Resource capability = getCapability();
+    if (capability == null) {
+      if (other.getCapability() != null)
+        return false;
+    } else if (!capability.equals(other.getCapability()))
+      return false;
+    String hostName = getHostName();
+    if (hostName == null) {
+      if (other.getHostName() != null)
+        return false;
+    } else if (!hostName.equals(other.getHostName()))
+      return false;
+    if (getNumContainers() != other.getNumContainers())
+      return false;
+    Priority priority = getPriority();
+    if (priority == null) {
+      if (other.getPriority() != null)
+        return false;
+    } else if (!priority.equals(other.getPriority()))
+      return false;
+    return true;
+  }
+
+  @Override
+  public int compareTo(ResourceRequest other) {
+    int priorityComparison = this.getPriority().compareTo(other.getPriority());
+    if (priorityComparison == 0) {
+      int hostNameComparison =
+          this.getHostName().compareTo(other.getHostName());
+      if (hostNameComparison == 0) {
+        int capabilityComparison =
+            this.getCapability().compareTo(other.getCapability());
+        if (capabilityComparison == 0) {
+          int numContainersComparison =
+              this.getNumContainers() - other.getNumContainers();
+          if (numContainersComparison == 0) {
+            return 0;
+          } else {
+            return numContainersComparison;
+          }
+        } else {
+          return capabilityComparison;
+        }
+      } else {
+        return hostNameComparison;
+      }
+    } else {
+      return priorityComparison;
+    }
+  }
 }

+ 1 - 12
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PriorityPBImpl.java

@@ -18,15 +18,11 @@
 
 package org.apache.hadoop.yarn.api.records.impl.pb;
 
-
 import org.apache.hadoop.yarn.api.records.Priority;
-import org.apache.hadoop.yarn.api.records.ProtoBase;
 import org.apache.hadoop.yarn.proto.YarnProtos.PriorityProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.PriorityProtoOrBuilder;
-
-
     
-public class PriorityPBImpl extends ProtoBase<PriorityProto> implements Priority {
+public class PriorityPBImpl extends Priority {
   PriorityProto proto = PriorityProto.getDefaultInstance();
   PriorityProto.Builder builder = null;
   boolean viaProto = false;
@@ -66,11 +62,4 @@ public class PriorityPBImpl extends ProtoBase<PriorityProto> implements Priority
     builder.setPriority((priority));
   }
 
-  
-  @Override
-  public int compareTo(Priority other) {
-    return this.getPriority() - other.getPriority();
-  }
-
-
 }  

+ 1 - 27
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceRequestPBImpl.java

@@ -20,19 +20,14 @@ package org.apache.hadoop.yarn.api.records.impl.pb;
 
 
 import org.apache.hadoop.yarn.api.records.Priority;
-import org.apache.hadoop.yarn.api.records.ProtoBase;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
-import org.apache.hadoop.yarn.api.records.impl.pb.PriorityPBImpl;
-import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl;
 import org.apache.hadoop.yarn.proto.YarnProtos.PriorityProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceRequestProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceRequestProtoOrBuilder;
 
-
-    
-public class ResourceRequestPBImpl extends ProtoBase<ResourceRequestProto> implements  ResourceRequest {
+public class ResourceRequestPBImpl extends  ResourceRequest {
   ResourceRequestProto proto = ResourceRequestProto.getDefaultInstance();
   ResourceRequestProto.Builder builder = null;
   boolean viaProto = false;
@@ -168,25 +163,4 @@ public class ResourceRequestPBImpl extends ProtoBase<ResourceRequestProto> imple
     return ((ResourcePBImpl)t).getProto();
   }
 
-  @Override
-  public int compareTo(ResourceRequest other) {
-    if (this.getPriority().compareTo(other.getPriority()) == 0) {
-      if (this.getHostName().equals(other.getHostName())) {
-        if (this.getCapability().equals(other.getCapability())) {
-          if (this.getNumContainers() == other.getNumContainers()) {
-            return 0;
-          } else {
-            return this.getNumContainers() - other.getNumContainers();
-          }
-        } else {
-          return this.getCapability().compareTo(other.getCapability());
-        }
-      } else {
-        return this.getHostName().compareTo(other.getHostName());
-      }
-    } else {
-      return this.getPriority().compareTo(other.getPriority());
-    }
-  }
-
 }