You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ji...@apache.org on 2015/05/01 09:01:18 UTC

[01/50] [abbrv] hadoop git commit: YARN-3406. Display count of running containers in the RM's Web UI. Contributed by Ryu Kobayashi.

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7240 9d8952f97 -> 1b3b9e5c3


YARN-3406. Display count of running containers in the RM's Web UI. Contributed by Ryu Kobayashi.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/4a3dabd9
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/4a3dabd9
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/4a3dabd9

Branch: refs/heads/HDFS-7240
Commit: 4a3dabd94fc3dddd9d7604b826065c23859d565f
Parents: 78fe6e5
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Sat Apr 25 07:17:11 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Sat Apr 25 07:17:11 2015 +0900

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +++
 .../hadoop/yarn/server/webapp/WebPageUtils.java | 25 +++++++++++++-------
 .../hadoop/yarn/server/webapp/dao/AppInfo.java  |  9 +++++++
 .../webapp/FairSchedulerAppsBlock.java          |  2 ++
 .../webapp/FairSchedulerPage.java               |  2 +-
 .../resourcemanager/webapp/RMAppsBlock.java     |  6 ++++-
 6 files changed, 37 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a3dabd9/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 6da55b5..44b87e5 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -168,6 +168,9 @@ Release 2.8.0 - UNRELEASED
 
     YARN-3511. Add errors and warnings page to ATS. (Varun Vasudev via xgong)
 
+    YARN-3406. Display count of running containers in the RM's Web UI.
+    (Ryu Kobayashi via ozawa)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a3dabd9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebPageUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebPageUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebPageUtils.java
index 5acabf5..6ca5011 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebPageUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebPageUtils.java
@@ -24,10 +24,11 @@ import static org.apache.hadoop.yarn.webapp.view.JQueryUI.tableInit;
 public class WebPageUtils {
 
   public static String appsTableInit() {
-    return appsTableInit(false);
+    return appsTableInit(false, true);
   }
 
-  public static String appsTableInit(boolean isFairSchedulerPage) {
+  public static String appsTableInit(
+      boolean isFairSchedulerPage, boolean isResourceManager) {
     // id, user, name, queue, starttime, finishtime, state, status, progress, ui
     // FairSchedulerPage's table is a bit different
     return tableInit()
@@ -35,22 +36,30 @@ public class WebPageUtils {
       .append(", bDeferRender: true")
       .append(", bProcessing: true")
       .append("\n, aoColumnDefs: ")
-      .append(getAppsTableColumnDefs(isFairSchedulerPage))
+      .append(getAppsTableColumnDefs(isFairSchedulerPage, isResourceManager))
       // Sort by id upon page load
       .append(", aaSorting: [[0, 'desc']]}").toString();
   }
 
-  private static String getAppsTableColumnDefs(boolean isFairSchedulerPage) {
+  private static String getAppsTableColumnDefs(
+      boolean isFairSchedulerPage, boolean isResourceManager) {
     StringBuilder sb = new StringBuilder();
-    return sb
-      .append("[\n")
+    sb.append("[\n")
       .append("{'sType':'string', 'aTargets': [0]")
       .append(", 'mRender': parseHadoopID }")
       .append("\n, {'sType':'numeric', 'aTargets': " +
           (isFairSchedulerPage ? "[6, 7]": "[5, 6]"))
       .append(", 'mRender': renderHadoopDate }")
-      .append("\n, {'sType':'numeric', bSearchable:false, 'aTargets': [9]")
-      .append(", 'mRender': parseHadoopProgress }]").toString();
+      .append("\n, {'sType':'numeric', bSearchable:false, 'aTargets':");
+    if (isFairSchedulerPage) {
+      sb.append("[11]");
+    } else if (isResourceManager) {
+      sb.append("[10]");
+    } else {
+      sb.append("[9]");
+    }
+    sb.append(", 'mRender': parseHadoopProgress }]");
+    return sb.toString();
   }
 
   public static String attemptsTableInit() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a3dabd9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppInfo.java
index e8b1acc..3eafb8c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppInfo.java
@@ -42,6 +42,7 @@ public class AppInfo {
   protected String host;
   protected int rpcPort;
   protected YarnApplicationState appState;
+  protected int runningContainers;
   protected float progress;
   protected String diagnosticsInfo;
   protected String originalTrackingUrl;
@@ -77,6 +78,10 @@ public class AppInfo {
     finishedTime = app.getFinishTime();
     elapsedTime = Times.elapsed(startedTime, finishedTime);
     finalAppStatus = app.getFinalApplicationStatus();
+    if (app.getApplicationResourceUsageReport() != null) {
+      runningContainers =
+          app.getApplicationResourceUsageReport().getNumUsedContainers();
+    }
     progress = app.getProgress() * 100; // in percent
     if (app.getApplicationTags() != null && !app.getApplicationTags().isEmpty()) {
       this.applicationTags = CSV_JOINER.join(app.getApplicationTags());
@@ -119,6 +124,10 @@ public class AppInfo {
     return appState;
   }
 
+  public int getRunningContainers() {
+    return runningContainers;
+  }
+
   public float getProgress() {
     return progress;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a3dabd9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerAppsBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerAppsBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerAppsBlock.java
index 8a5f4bd..9650dfe 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerAppsBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerAppsBlock.java
@@ -91,6 +91,7 @@ public class FairSchedulerAppsBlock extends HtmlBlock {
             th(".finishtime", "FinishTime").
             th(".state", "State").
             th(".finalstatus", "FinalStatus").
+            th(".runningcontainer", "Running Containers").
             th(".progress", "Progress").
             th(".ui", "Tracking UI")._()._().
         tbody();
@@ -132,6 +133,7 @@ public class FairSchedulerAppsBlock extends HtmlBlock {
       .append(appInfo.getFinishTime()).append("\",\"")
       .append(appInfo.getState()).append("\",\"")
       .append(appInfo.getFinalStatus()).append("\",\"")
+      .append(appInfo.getRunningContainers()).append("\",\"")
       // Progress bar
       .append("<br title='").append(percent)
       .append("'> <div class='").append(C_PROGRESSBAR).append("' title='")

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a3dabd9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerPage.java
index 97ab872..76d7e6f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerPage.java
@@ -237,7 +237,7 @@ public class FairSchedulerPage extends RmView {
 
   @Override
   protected String initAppsTable() {
-    return WebPageUtils.appsTableInit(true);
+    return WebPageUtils.appsTableInit(true, false);
   }
 
   static String percent(float f) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a3dabd9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java
index 29565e4..715d9d9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java
@@ -57,7 +57,9 @@ public class RMAppsBlock extends AppsBlock {
           .th(".name", "Name").th(".type", "Application Type")
           .th(".queue", "Queue").th(".starttime", "StartTime")
           .th(".finishtime", "FinishTime").th(".state", "State")
-          .th(".finalstatus", "FinalStatus").th(".progress", "Progress")
+          .th(".finalstatus", "FinalStatus")
+          .th(".runningcontainer", "Running Containers")
+          .th(".progress", "Progress")
           .th(".ui", "Tracking UI").th(".blacklisted", "Blacklisted Nodes")._()
           ._().tbody();
 
@@ -108,6 +110,8 @@ public class RMAppsBlock extends AppsBlock {
         .append("\",\"")
         .append(app.getFinalAppStatus())
         .append("\",\"")
+        .append(String.valueOf(app.getRunningContainers()))
+        .append("\",\"")
         // Progress bar
         .append("<br title='").append(percent).append("'> <div class='")
         .append(C_PROGRESSBAR).append("' title='").append(join(percent, '%'))


[11/50] [abbrv] hadoop git commit: YARN-3464. Race condition in LocalizerRunner kills localizer before localizing all resources. (Zhihai Xu via kasha)

Posted by ji...@apache.org.
YARN-3464. Race condition in LocalizerRunner kills localizer before localizing all resources. (Zhihai Xu via kasha)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/47279c32
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/47279c32
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/47279c32

Branch: refs/heads/HDFS-7240
Commit: 47279c3228185548ed09c36579b420225e4894f5
Parents: 22b70e7
Author: Karthik Kambatla <ka...@apache.org>
Authored: Sun Apr 26 09:13:46 2015 -0700
Committer: Karthik Kambatla <ka...@apache.org>
Committed: Sun Apr 26 09:13:46 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../container/ContainerImpl.java                |  8 ++-
 .../localizer/ResourceLocalizationService.java  | 53 +++++++++++++++-----
 .../localizer/event/LocalizationEventType.java  |  1 +
 .../TestResourceLocalizationService.java        | 12 ++++-
 5 files changed, 62 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/47279c32/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index a626f82..87db291 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -268,6 +268,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3537. NPE when NodeManager.serviceInit fails and stopRecoveryStore
     invoked (Brahma Reddy Battula via jlowe)
 
+    YARN-3464. Race condition in LocalizerRunner kills localizer before 
+    localizing all resources. (Zhihai Xu via kasha)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47279c32/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
index c9874a6..68669aa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
@@ -59,7 +59,9 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.Conta
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainersLauncherEventType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.LocalResourceRequest;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ContainerLocalizationCleanupEvent;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ContainerLocalizationEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ContainerLocalizationRequestEvent;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizationEventType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.sharedcache.SharedCacheUploadEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.sharedcache.SharedCacheUploadEventType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerContainerFinishedEvent;
@@ -716,7 +718,12 @@ public class ContainerImpl implements Container {
         return ContainerState.LOCALIZING;
       }
 
+      container.dispatcher.getEventHandler().handle(
+          new ContainerLocalizationEvent(LocalizationEventType.
+              CONTAINER_RESOURCES_LOCALIZED, container));
+
       container.sendLaunchEvent();
+      container.metrics.endInitingContainer();
 
       // If this is a recovered container that has already launched, skip
       // uploading resources to the shared cache. We do this to avoid uploading
@@ -734,7 +741,6 @@ public class ContainerImpl implements Container {
                 SharedCacheUploadEventType.UPLOAD));
       }
 
-      container.metrics.endInitingContainer();
       return ContainerState.LOCALIZED;
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47279c32/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
index 611fe80..cdd252c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
@@ -35,6 +35,7 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.CancellationException;
 import java.util.concurrent.CompletionService;
 import java.util.concurrent.ConcurrentHashMap;
@@ -108,6 +109,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Cont
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerResourceFailedEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ApplicationLocalizationEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ContainerLocalizationCleanupEvent;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ContainerLocalizationEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ContainerLocalizationRequestEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizationEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizationEventType;
@@ -389,6 +391,9 @@ public class ResourceLocalizationService extends CompositeService
     case INIT_CONTAINER_RESOURCES:
       handleInitContainerResources((ContainerLocalizationRequestEvent) event);
       break;
+    case CONTAINER_RESOURCES_LOCALIZED:
+      handleContainerResourcesLocalized((ContainerLocalizationEvent) event);
+      break;
     case CACHE_CLEANUP:
       handleCacheCleanup(event);
       break;
@@ -455,7 +460,18 @@ public class ResourceLocalizationService extends CompositeService
       }
     }
   }
-  
+
+  /**
+   * Once a container's resources are localized, kill the corresponding
+   * {@link ContainerLocalizer}
+   */
+  private void handleContainerResourcesLocalized(
+      ContainerLocalizationEvent event) {
+    Container c = event.getContainer();
+    String locId = ConverterUtils.toString(c.getContainerId());
+    localizerTracker.endContainerLocalization(locId);
+  }
+
   private void handleCacheCleanup(LocalizationEvent event) {
     ResourceRetentionSet retain =
       new ResourceRetentionSet(delService, cacheTargetSize);
@@ -670,7 +686,7 @@ public class ResourceLocalizationService extends CompositeService
           response.setLocalizerAction(LocalizerAction.DIE);
           return response;
         }
-        return localizer.update(status.getResources());
+        return localizer.processHeartbeat(status.getResources());
       }
     }
     
@@ -724,6 +740,17 @@ public class ResourceLocalizationService extends CompositeService
         localizer.interrupt();
       }
     }
+
+    public void endContainerLocalization(String locId) {
+      LocalizerRunner localizer;
+      synchronized (privLocalizers) {
+        localizer = privLocalizers.get(locId);
+        if (null == localizer) {
+          return; // ignore
+        }
+      }
+      localizer.endContainerLocalization();
+    }
   }
   
 
@@ -878,6 +905,7 @@ public class ResourceLocalizationService extends CompositeService
     final Map<LocalResourceRequest,LocalizerResourceRequestEvent> scheduled;
     // Its a shared list between Private Localizer and dispatcher thread.
     final List<LocalizerResourceRequestEvent> pending;
+    private AtomicBoolean killContainerLocalizer = new AtomicBoolean(false);
 
     // TODO: threadsafe, use outer?
     private final RecordFactory recordFactory =
@@ -898,6 +926,10 @@ public class ResourceLocalizationService extends CompositeService
       pending.add(request);
     }
 
+    public void endContainerLocalization() {
+      killContainerLocalizer.set(true);
+    }
+
     /**
      * Find next resource to be given to a spawned localizer.
      * 
@@ -944,7 +976,7 @@ public class ResourceLocalizationService extends CompositeService
       }
     }
 
-    LocalizerHeartbeatResponse update(
+    LocalizerHeartbeatResponse processHeartbeat(
         List<LocalResourceStatus> remoteResourceStatuses) {
       LocalizerHeartbeatResponse response =
         recordFactory.newRecordInstance(LocalizerHeartbeatResponse.class);
@@ -953,7 +985,7 @@ public class ResourceLocalizationService extends CompositeService
       ApplicationId applicationId =
           context.getContainerId().getApplicationAttemptId().getApplicationId();
 
-      LocalizerAction action = LocalizerAction.LIVE;
+      boolean fetchFailed = false;
       // Update resource statuses.
       for (LocalResourceStatus stat : remoteResourceStatuses) {
         LocalResource rsrc = stat.getResource();
@@ -989,7 +1021,7 @@ public class ResourceLocalizationService extends CompositeService
           case FETCH_FAILURE:
             final String diagnostics = stat.getException().toString();
             LOG.warn(req + " failed: " + diagnostics);
-            action = LocalizerAction.DIE;
+            fetchFailed = true;
             getLocalResourcesTracker(req.getVisibility(), user, applicationId)
               .handle(new ResourceFailedLocalizationEvent(
                   req, diagnostics));
@@ -1001,15 +1033,15 @@ public class ResourceLocalizationService extends CompositeService
             break;
           default:
             LOG.info("Unknown status: " + stat.getStatus());
-            action = LocalizerAction.DIE;
+            fetchFailed = true;
             getLocalResourcesTracker(req.getVisibility(), user, applicationId)
               .handle(new ResourceFailedLocalizationEvent(
                   req, stat.getException().getMessage()));
             break;
         }
       }
-      if (action == LocalizerAction.DIE) {
-        response.setLocalizerAction(action);
+      if (fetchFailed || killContainerLocalizer.get()) {
+        response.setLocalizerAction(LocalizerAction.DIE);
         return response;
       }
 
@@ -1037,12 +1069,9 @@ public class ResourceLocalizationService extends CompositeService
         } catch (URISyntaxException e) {
             //TODO fail? Already translated several times...
         }
-      } else if (pending.isEmpty()) {
-        // TODO: Synchronization
-        action = LocalizerAction.DIE;
       }
 
-      response.setLocalizerAction(action);
+      response.setLocalizerAction(LocalizerAction.LIVE);
       response.setResourceSpecs(rsrcs);
       return response;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47279c32/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/event/LocalizationEventType.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/event/LocalizationEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/event/LocalizationEventType.java
index 5134349..4785fba 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/event/LocalizationEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/event/LocalizationEventType.java
@@ -23,4 +23,5 @@ public enum LocalizationEventType {
   CACHE_CLEANUP,
   CLEANUP_CONTAINER_RESOURCES,
   DESTROY_APPLICATION_RESOURCES,
+  CONTAINER_RESOURCES_LOCALIZED,
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47279c32/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
index d3c3521..2edaf45 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
@@ -125,6 +125,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.Reso
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService.PublicLocalizer;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ApplicationLocalizationEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ContainerLocalizationCleanupEvent;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ContainerLocalizationEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ContainerLocalizationRequestEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizationEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizationEventType;
@@ -975,7 +976,8 @@ public class TestResourceLocalizationService {
         .thenReturn(Collections.<LocalResourceStatus>emptyList())
         .thenReturn(Collections.singletonList(rsrc1success))
         .thenReturn(Collections.singletonList(rsrc2pending))
-        .thenReturn(rsrcs4);
+        .thenReturn(rsrcs4)
+        .thenReturn(Collections.<LocalResourceStatus>emptyList());
 
       String localPath = Path.SEPARATOR + ContainerLocalizer.USERCACHE +
           Path.SEPARATOR + "user0" + Path.SEPARATOR +
@@ -1019,7 +1021,13 @@ public class TestResourceLocalizationService {
       assertTrue(localizedPath.getFile().endsWith(
           localPath + Path.SEPARATOR + "1" + Path.SEPARATOR + "12"));
 
-      // get shutdown
+      response = spyService.heartbeat(stat);
+      assertEquals(LocalizerAction.LIVE, response.getLocalizerAction());
+
+      spyService.handle(new ContainerLocalizationEvent(
+          LocalizationEventType.CONTAINER_RESOURCES_LOCALIZED, c));
+
+      // get shutdown after receive CONTAINER_RESOURCES_LOCALIZED event
       response = spyService.heartbeat(stat);
       assertEquals(LocalizerAction.DIE, response.getLocalizerAction());
 


[33/50] [abbrv] hadoop git commit: YARN-3485. FairScheduler headroom calculation doesn't consider maxResources for Fifo and FairShare policies. (kasha)

Posted by ji...@apache.org.
YARN-3485. FairScheduler headroom calculation doesn't consider maxResources for Fifo and FairShare policies. (kasha)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/8f82970e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8f82970e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8f82970e

Branch: refs/heads/HDFS-7240
Commit: 8f82970e0c247b37b2bf33aa21f6a39afa07efde
Parents: 439614b
Author: Karthik Kambatla <ka...@apache.org>
Authored: Tue Apr 28 21:00:35 2015 -0700
Committer: Karthik Kambatla <ka...@apache.org>
Committed: Tue Apr 28 21:00:35 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../scheduler/fair/FSAppAttempt.java            | 20 ++++--
 .../scheduler/fair/SchedulingPolicy.java        |  4 +-
 .../DominantResourceFairnessPolicy.java         |  6 +-
 .../fair/policies/FairSharePolicy.java          |  6 +-
 .../scheduler/fair/policies/FifoPolicy.java     |  6 +-
 .../scheduler/fair/TestFSAppAttempt.java        | 66 ++++++++++++++------
 7 files changed, 76 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8f82970e/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 9039460..b5581d6 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -308,6 +308,9 @@ Release 2.7.1 - UNRELEASED
 
     YARN-3464. Race condition in LocalizerRunner kills localizer before 
     localizing all resources. (Zhihai Xu via kasha)
+    
+    YARN-3485. FairScheduler headroom calculation doesn't consider 
+    maxResources for Fifo and FairShare policies. (kasha)
 
 Release 2.7.0 - 2015-04-20
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8f82970e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
index f0d1ed1..6287deb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
@@ -172,6 +172,10 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
         + this.attemptResourceUsage.getReserved());
   }
 
+  /**
+   * Headroom depends on resources in the cluster, current usage of the
+   * queue, queue's fair-share and queue's max-resources.
+   */
   @Override
   public Resource getHeadroom() {
     final FSQueue queue = (FSQueue) this.queue;
@@ -182,18 +186,22 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
     Resource clusterResource = this.scheduler.getClusterResource();
     Resource clusterUsage = this.scheduler.getRootQueueMetrics()
         .getAllocatedResources();
-    Resource clusterAvailableResource = Resources.subtract(clusterResource,
-        clusterUsage);
+
+    Resource clusterAvailableResources =
+        Resources.subtract(clusterResource, clusterUsage);
+    Resource queueMaxAvailableResources =
+        Resources.subtract(queue.getMaxShare(), queueUsage);
+    Resource maxAvailableResource = Resources.componentwiseMin(
+        clusterAvailableResources, queueMaxAvailableResources);
+
     Resource headroom = policy.getHeadroom(queueFairShare,
-        queueUsage, clusterAvailableResource);
+        queueUsage, maxAvailableResource);
     if (LOG.isDebugEnabled()) {
       LOG.debug("Headroom calculation for " + this.getName() + ":" +
           "Min(" +
           "(queueFairShare=" + queueFairShare +
           " - queueUsage=" + queueUsage + ")," +
-          " clusterAvailableResource=" + clusterAvailableResource +
-          "(clusterResource=" + clusterResource +
-          " - clusterUsage=" + clusterUsage + ")" +
+          " maxAvailableResource=" + maxAvailableResource +
           "Headroom=" + headroom);
     }
     return headroom;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8f82970e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingPolicy.java
index bf2a25b..abdc834 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingPolicy.java
@@ -185,10 +185,10 @@ public abstract class SchedulingPolicy {
    *
    * @param queueFairShare fairshare in the queue
    * @param queueUsage resources used in the queue
-   * @param clusterAvailable available resource in cluster
+   * @param maxAvailable available resource in cluster for this queue
    * @return calculated headroom
    */
   public abstract Resource getHeadroom(Resource queueFairShare,
-      Resource queueUsage, Resource clusterAvailable);
+      Resource queueUsage, Resource maxAvailable);
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8f82970e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java
index 3f6cbd1..86d503b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java
@@ -90,15 +90,15 @@ public class DominantResourceFairnessPolicy extends SchedulingPolicy {
 
   @Override
   public Resource getHeadroom(Resource queueFairShare, Resource queueUsage,
-                              Resource clusterAvailable) {
+                              Resource maxAvailable) {
     int queueAvailableMemory =
         Math.max(queueFairShare.getMemory() - queueUsage.getMemory(), 0);
     int queueAvailableCPU =
         Math.max(queueFairShare.getVirtualCores() - queueUsage
             .getVirtualCores(), 0);
     Resource headroom = Resources.createResource(
-        Math.min(clusterAvailable.getMemory(), queueAvailableMemory),
-        Math.min(clusterAvailable.getVirtualCores(),
+        Math.min(maxAvailable.getMemory(), queueAvailableMemory),
+        Math.min(maxAvailable.getVirtualCores(),
             queueAvailableCPU));
     return headroom;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8f82970e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java
index 97669cb..918db9d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java
@@ -116,12 +116,12 @@ public class FairSharePolicy extends SchedulingPolicy {
 
   @Override
   public Resource getHeadroom(Resource queueFairShare,
-                              Resource queueUsage, Resource clusterAvailable) {
+                              Resource queueUsage, Resource maxAvailable) {
     int queueAvailableMemory = Math.max(
         queueFairShare.getMemory() - queueUsage.getMemory(), 0);
     Resource headroom = Resources.createResource(
-        Math.min(clusterAvailable.getMemory(), queueAvailableMemory),
-        clusterAvailable.getVirtualCores());
+        Math.min(maxAvailable.getMemory(), queueAvailableMemory),
+        maxAvailable.getVirtualCores());
     return headroom;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8f82970e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FifoPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FifoPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FifoPolicy.java
index a2e17ec..7d88933 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FifoPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FifoPolicy.java
@@ -109,12 +109,12 @@ public class FifoPolicy extends SchedulingPolicy {
 
   @Override
   public Resource getHeadroom(Resource queueFairShare,
-                              Resource queueUsage, Resource clusterAvailable) {
+                              Resource queueUsage, Resource maxAvailable) {
     int queueAvailableMemory = Math.max(
         queueFairShare.getMemory() - queueUsage.getMemory(), 0);
     Resource headroom = Resources.createResource(
-        Math.min(clusterAvailable.getMemory(), queueAvailableMemory),
-        clusterAvailable.getVirtualCores());
+        Math.min(maxAvailable.getMemory(), queueAvailableMemory),
+        maxAvailable.getVirtualCores());
     return headroom;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8f82970e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppAttempt.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppAttempt.java
index f560690..43fe186 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppAttempt.java
@@ -198,18 +198,24 @@ public class TestFSAppAttempt extends FairSchedulerTestBase {
     Mockito.when(mockScheduler.getClock()).thenReturn(scheduler.getClock());
 
     final FSLeafQueue mockQueue = Mockito.mock(FSLeafQueue.class);
-    final Resource queueFairShare = Resources.createResource(4096, 4);
-    final Resource queueUsage = Resource.newInstance(1024, 1);
+
+    final Resource queueMaxResources = Resource.newInstance(5 * 1024, 3);
+    final Resource queueFairShare = Resources.createResource(4096, 2);
+    final Resource queueUsage = Resource.newInstance(2048, 2);
+
+    final Resource queueStarvation =
+        Resources.subtract(queueFairShare, queueUsage);
+    final Resource queueMaxResourcesAvailable =
+        Resources.subtract(queueMaxResources, queueUsage);
+
     final Resource clusterResource = Resources.createResource(8192, 8);
-    final Resource clusterUsage = Resources.createResource(6144, 2);
-    final QueueMetrics fakeRootQueueMetrics = Mockito.mock(QueueMetrics.class);
+    final Resource clusterUsage = Resources.createResource(2048, 2);
+    final Resource clusterAvailable =
+        Resources.subtract(clusterResource, clusterUsage);
 
-    ApplicationAttemptId applicationAttemptId = createAppAttemptId(1, 1);
-    RMContext rmContext = resourceManager.getRMContext();
-    FSAppAttempt schedulerApp =
-        new FSAppAttempt(mockScheduler, applicationAttemptId, "user1", mockQueue ,
-            null, rmContext);
+    final QueueMetrics fakeRootQueueMetrics = Mockito.mock(QueueMetrics.class);
 
+    Mockito.when(mockQueue.getMaxShare()).thenReturn(queueMaxResources);
     Mockito.when(mockQueue.getFairShare()).thenReturn(queueFairShare);
     Mockito.when(mockQueue.getResourceUsage()).thenReturn(queueUsage);
     Mockito.when(mockScheduler.getClusterResource()).thenReturn
@@ -219,27 +225,51 @@ public class TestFSAppAttempt extends FairSchedulerTestBase {
     Mockito.when(mockScheduler.getRootQueueMetrics()).thenReturn
         (fakeRootQueueMetrics);
 
-    int minClusterAvailableMemory = 2048;
-    int minClusterAvailableCPU = 6;
-    int minQueueAvailableCPU = 3;
+    ApplicationAttemptId applicationAttemptId = createAppAttemptId(1, 1);
+    RMContext rmContext = resourceManager.getRMContext();
+    FSAppAttempt schedulerApp =
+        new FSAppAttempt(mockScheduler, applicationAttemptId, "user1", mockQueue ,
+            null, rmContext);
 
     // Min of Memory and CPU across cluster and queue is used in
     // DominantResourceFairnessPolicy
     Mockito.when(mockQueue.getPolicy()).thenReturn(SchedulingPolicy
         .getInstance(DominantResourceFairnessPolicy.class));
-    verifyHeadroom(schedulerApp, minClusterAvailableMemory,
-        minQueueAvailableCPU);
+    verifyHeadroom(schedulerApp,
+        min(queueStarvation.getMemory(),
+            clusterAvailable.getMemory(),
+            queueMaxResourcesAvailable.getMemory()),
+        min(queueStarvation.getVirtualCores(),
+            clusterAvailable.getVirtualCores(),
+            queueMaxResourcesAvailable.getVirtualCores())
+    );
 
     // Fair and Fifo ignore CPU of queue, so use cluster available CPU
     Mockito.when(mockQueue.getPolicy()).thenReturn(SchedulingPolicy
         .getInstance(FairSharePolicy.class));
-    verifyHeadroom(schedulerApp, minClusterAvailableMemory,
-        minClusterAvailableCPU);
+    verifyHeadroom(schedulerApp,
+        min(queueStarvation.getMemory(),
+            clusterAvailable.getMemory(),
+            queueMaxResourcesAvailable.getMemory()),
+        Math.min(
+            clusterAvailable.getVirtualCores(),
+            queueMaxResourcesAvailable.getVirtualCores())
+    );
 
     Mockito.when(mockQueue.getPolicy()).thenReturn(SchedulingPolicy
         .getInstance(FifoPolicy.class));
-    verifyHeadroom(schedulerApp, minClusterAvailableMemory,
-        minClusterAvailableCPU);
+    verifyHeadroom(schedulerApp,
+        min(queueStarvation.getMemory(),
+            clusterAvailable.getMemory(),
+            queueMaxResourcesAvailable.getMemory()),
+        Math.min(
+            clusterAvailable.getVirtualCores(),
+            queueMaxResourcesAvailable.getVirtualCores())
+    );
+  }
+
+  private static int min(int value1, int value2, int value3) {
+    return Math.min(Math.min(value1, value2), value3);
   }
 
   protected void verifyHeadroom(FSAppAttempt schedulerApp,


[21/50] [abbrv] hadoop git commit: HADOOP-11870. [JDK8] AuthenticationFilter, CertificateUtil, SignerSecretProviders, KeyAuthorizationKeyProvider Javadoc issues (rkanter)

Posted by ji...@apache.org.
HADOOP-11870. [JDK8] AuthenticationFilter, CertificateUtil, SignerSecretProviders, KeyAuthorizationKeyProvider Javadoc issues (rkanter)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/9fec02c0
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9fec02c0
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9fec02c0

Branch: refs/heads/HDFS-7240
Commit: 9fec02c069f9bb24b5ee99031917075b4c7a7682
Parents: 6bae596
Author: Robert Kanter <rk...@apache.org>
Authored: Mon Apr 27 13:25:11 2015 -0700
Committer: Robert Kanter <rk...@apache.org>
Committed: Mon Apr 27 13:25:11 2015 -0700

----------------------------------------------------------------------
 .../authentication/server/AuthenticationFilter.java    | 13 +++++++++++++
 .../security/authentication/util/CertificateUtil.java  |  3 ++-
 .../util/RolloverSignerSecretProvider.java             |  2 +-
 .../authentication/util/SignerSecretProvider.java      |  2 +-
 .../authentication/util/ZKSignerSecretProvider.java    |  2 +-
 hadoop-common-project/hadoop-common/CHANGES.txt        |  3 +++
 .../key/kms/server/KeyAuthorizationKeyProvider.java    |  4 ++--
 7 files changed, 23 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fec02c0/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java
index 9cfa3c2..0f86623 100644
--- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java
@@ -589,6 +589,13 @@ public class AuthenticationFilter implements Filter {
   /**
    * Delegates call to the servlet filter chain. Sub-classes my override this
    * method to perform pre and post tasks.
+   *
+   * @param filterChain the filter chain object.
+   * @param request the request object.
+   * @param response the response object.
+   *
+   * @throws IOException thrown if an IO error occurred.
+   * @throws ServletException thrown if a processing error occurred.
    */
   protected void doFilter(FilterChain filterChain, HttpServletRequest request,
       HttpServletResponse response) throws IOException, ServletException {
@@ -598,9 +605,15 @@ public class AuthenticationFilter implements Filter {
   /**
    * Creates the Hadoop authentication HTTP cookie.
    *
+   * @param resp the response object.
    * @param token authentication token for the cookie.
+   * @param domain the cookie domain.
+   * @param path the cokie path.
    * @param expires UNIX timestamp that indicates the expire date of the
    *                cookie. It has no effect if its value &lt; 0.
+   * @param isSecure is the cookie secure?
+   * @param token the token.
+   * @param expires the cookie expiration time.
    *
    * XXX the following code duplicate some logic in Jetty / Servlet API,
    * because of the fact that Hadoop is stuck at servlet 2.5 and jetty 6

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fec02c0/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/CertificateUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/CertificateUtil.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/CertificateUtil.java
index 77b2530..1ca59ae 100644
--- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/CertificateUtil.java
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/CertificateUtil.java
@@ -36,7 +36,8 @@ public class CertificateUtil {
    *
    * @param pem
    *          - the pem encoding from config without the header and footer
-   * @return RSAPublicKey
+   * @return RSAPublicKey the RSA public key
+   * @throws ServletException thrown if a processing error occurred
    */
   public static RSAPublicKey parseRSAPublicKey(String pem) throws ServletException {
     String fullPem = PEM_HEADER + pem + PEM_FOOTER;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fec02c0/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/RolloverSignerSecretProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/RolloverSignerSecretProvider.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/RolloverSignerSecretProvider.java
index bdca3e4..fda5572 100644
--- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/RolloverSignerSecretProvider.java
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/RolloverSignerSecretProvider.java
@@ -61,7 +61,7 @@ public abstract class RolloverSignerSecretProvider
    * @param config configuration properties
    * @param servletContext servlet context
    * @param tokenValidity The amount of time a token is valid for
-   * @throws Exception
+   * @throws Exception thrown if an error occurred
    */
   @Override
   public void init(Properties config, ServletContext servletContext,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fec02c0/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/SignerSecretProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/SignerSecretProvider.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/SignerSecretProvider.java
index 2e0b985..e937862 100644
--- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/SignerSecretProvider.java
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/SignerSecretProvider.java
@@ -34,7 +34,7 @@ public abstract class SignerSecretProvider {
    * @param config configuration properties
    * @param servletContext servlet context
    * @param tokenValidity The amount of time a token is valid for
-   * @throws Exception
+   * @throws Exception thrown if an error occurred
    */
   public abstract void init(Properties config, ServletContext servletContext,
           long tokenValidity) throws Exception;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fec02c0/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZKSignerSecretProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZKSignerSecretProvider.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZKSignerSecretProvider.java
index 11bfccd..5e5f087 100644
--- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZKSignerSecretProvider.java
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZKSignerSecretProvider.java
@@ -377,7 +377,7 @@ public class ZKSignerSecretProvider extends RolloverSignerSecretProvider {
    * This method creates the Curator client and connects to ZooKeeper.
    * @param config configuration properties
    * @return A Curator client
-   * @throws Exception
+   * @throws Exception thrown if an error occurred
    */
   protected CuratorFramework createCuratorClient(Properties config)
           throws Exception {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fec02c0/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 390dbaf..1783e8d 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -573,6 +573,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11876. Refactor code to make it more readable, minor
     maybePrintStats bug (Zoran Dimitrijevic via raviprak)
 
+    HADOOP-11870. [JDK8] AuthenticationFilter, CertificateUtil,
+    SignerSecretProviders, KeyAuthorizationKeyProvider Javadoc issues (rkanter)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fec02c0/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KeyAuthorizationKeyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KeyAuthorizationKeyProvider.java b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KeyAuthorizationKeyProvider.java
index 074f1fb..e8cabc8 100644
--- a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KeyAuthorizationKeyProvider.java
+++ b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KeyAuthorizationKeyProvider.java
@@ -93,8 +93,8 @@ public class KeyAuthorizationKeyProvider extends KeyProviderCryptoExtension {
    * The constructor takes a {@link KeyProviderCryptoExtension} and an
    * implementation of <code>KeyACLs</code>. All calls are delegated to the
    * provider keyProvider after authorization check (if required)
-   * @param keyProvider 
-   * @param acls
+   * @param keyProvider  the key provider
+   * @param acls the Key ACLs
    */
   public KeyAuthorizationKeyProvider(KeyProviderCryptoExtension keyProvider,
       KeyACLs acls) {


[46/50] [abbrv] hadoop git commit: HADOOP-11891. OsSecureRandom should lazily fill its reservoir (asuresh)

Posted by ji...@apache.org.
HADOOP-11891. OsSecureRandom should lazily fill its reservoir (asuresh)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/f0db797b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f0db797b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f0db797b

Branch: refs/heads/HDFS-7240
Commit: f0db797be28ca221d540c6a3accd6bff9a7996fa
Parents: c55d609
Author: Arun Suresh <as...@apache.org>
Authored: Thu Apr 30 13:59:43 2015 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Thu Apr 30 13:59:43 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt  |  2 ++
 .../hadoop/crypto/random/OsSecureRandom.java     | 19 ++++---------------
 2 files changed, 6 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f0db797b/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 373b33e..d2e1d4a 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -603,6 +603,8 @@ Release 2.7.1 - UNRELEASED
     HADOOP-11802. DomainSocketWatcher thread terminates sometimes after there
     is an I/O error during requestShortCircuitShm (cmccabe)
 
+    HADOOP-11891. OsSecureRandom should lazily fill its reservoir (asuresh)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f0db797b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/random/OsSecureRandom.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/random/OsSecureRandom.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/random/OsSecureRandom.java
index fee4186..2010e6d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/random/OsSecureRandom.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/random/OsSecureRandom.java
@@ -58,6 +58,9 @@ public class OsSecureRandom extends Random implements Closeable, Configurable {
   private void fillReservoir(int min) {
     if (pos >= reservoir.length - min) {
       try {
+        if (stream == null) {
+          stream = new FileInputStream(new File(randomDevPath));
+        }
         IOUtils.readFully(stream, reservoir, 0, reservoir.length);
       } catch (IOException e) {
         throw new RuntimeException("failed to fill reservoir", e);
@@ -75,21 +78,7 @@ public class OsSecureRandom extends Random implements Closeable, Configurable {
     this.randomDevPath = conf.get(
         HADOOP_SECURITY_SECURE_RANDOM_DEVICE_FILE_PATH_KEY,
         HADOOP_SECURITY_SECURE_RANDOM_DEVICE_FILE_PATH_DEFAULT);
-    File randomDevFile = new File(randomDevPath);
-
-    try {
-      close();
-      this.stream = new FileInputStream(randomDevFile);
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-
-    try {
-      fillReservoir(0);
-    } catch (RuntimeException e) {
-      close();
-      throw e;
-    }
+    close();
   }
 
   @Override


[07/50] [abbrv] hadoop git commit: HDFS-7673. synthetic load generator docs give incorrect/incomplete commands (Brahma Reddy Battula via aw)

Posted by ji...@apache.org.
HDFS-7673. synthetic load generator docs give incorrect/incomplete commands (Brahma Reddy Battula via aw)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/f83c55a6
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f83c55a6
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f83c55a6

Branch: refs/heads/HDFS-7240
Commit: f83c55a6be4d6482d05613446be6322a5bce8add
Parents: 78c6b46
Author: Allen Wittenauer <aw...@apache.org>
Authored: Sat Apr 25 15:05:43 2015 +0100
Committer: Allen Wittenauer <aw...@apache.org>
Committed: Sat Apr 25 15:05:43 2015 +0100

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                    | 3 +++
 .../hadoop-hdfs/src/site/markdown/SLGUserGuide.md              | 6 +++---
 2 files changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f83c55a6/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index a7b5ed3..07c5151 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -318,6 +318,9 @@ Trunk (Unreleased)
     HDFS-8110. Remove unsupported 'hdfs namenode -rollingUpgrade downgrade'
     from document. (J.Andreina via aajisaka)
 
+    HDFS-7673. synthetic load generator docs give incorrect/incomplete commands
+    (Brahma Reddy Battula via aw)
+
 Release 2.8.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f83c55a6/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/SLGUserGuide.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/SLGUserGuide.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/SLGUserGuide.md
index 48b92c2..38b293e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/SLGUserGuide.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/SLGUserGuide.md
@@ -32,7 +32,7 @@ Synopsis
 
 The synopsis of the command is:
 
-        java LoadGenerator [options]
+        yarn jar <HADOOP_HOME>/share/hadoop/mapreduce/hadoop-mapreduce-client-jobclient-<hadoop-version>.jar NNloadGenerator [options]
 
 Options include:
 
@@ -102,7 +102,7 @@ The generated namespace structure is described by two files in the output direct
 
 The synopsis of the command is:
 
-        java StructureGenerator [options]
+        yarn jar <HADOOP_HOME>/share/hadoop/mapreduce/hadoop-mapreduce-client-jobclient-<hadoop-version>.jar NNstructureGenerator [options]
 
 Options include:
 
@@ -140,7 +140,7 @@ This tool reads the directory structure and file structure from the input direct
 
 The synopsis of the command is:
 
-        java DataGenerator [options]
+        yarn jar <HADOOP_HOME>/share/hadoop/mapreduce/hadoop-mapreduce-client-jobclient-<hadoop-version>.jar NNdataGenerator [options]
 
 Options include:
 


[35/50] [abbrv] hadoop git commit: HDFS-8269. getBlockLocations() does not resolve the .reserved path and generates incorrect edit logs when updating the atime. Contributed by Haohui Mai.

Posted by ji...@apache.org.
HDFS-8269. getBlockLocations() does not resolve the .reserved path and generates incorrect edit logs when updating the atime. Contributed by Haohui Mai.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/3dd6395b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3dd6395b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3dd6395b

Branch: refs/heads/HDFS-7240
Commit: 3dd6395bb2448e5b178a51c864e3c9a3d12e8bc9
Parents: 7947e5b
Author: Haohui Mai <wh...@apache.org>
Authored: Wed Apr 29 11:12:45 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Wed Apr 29 11:12:45 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../hdfs/server/namenode/FSNamesystem.java      |  63 ++++++---
 .../hdfs/server/namenode/NamenodeFsck.java      |   4 +-
 .../hadoop/hdfs/server/namenode/TestFsck.java   |   8 +-
 .../server/namenode/TestGetBlockLocations.java  | 133 +++++++++++++++++++
 5 files changed, 188 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3dd6395b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 2dde356..a3f219b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -635,6 +635,9 @@ Release 2.7.1 - UNRELEASED
     HDFS-8273. FSNamesystem#Delete() should not call logSync() when holding the
     lock. (wheat9)
 
+    HDFS-8269. getBlockLocations() does not resolve the .reserved path and
+    generates incorrect edit logs when updating the atime. (wheat9)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3dd6395b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 0ec81d8..b938263 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -1697,13 +1697,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   static class GetBlockLocationsResult {
-    final INodesInPath iip;
+    final boolean updateAccessTime;
     final LocatedBlocks blocks;
     boolean updateAccessTime() {
-      return iip != null;
+      return updateAccessTime;
     }
-    private GetBlockLocationsResult(INodesInPath iip, LocatedBlocks blocks) {
-      this.iip = iip;
+    private GetBlockLocationsResult(
+        boolean updateAccessTime, LocatedBlocks blocks) {
+      this.updateAccessTime = updateAccessTime;
       this.blocks = blocks;
     }
   }
@@ -1712,34 +1713,58 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * Get block locations within the specified range.
    * @see ClientProtocol#getBlockLocations(String, long, long)
    */
-  LocatedBlocks getBlockLocations(String clientMachine, String src,
+  LocatedBlocks getBlockLocations(String clientMachine, String srcArg,
       long offset, long length) throws IOException {
     checkOperation(OperationCategory.READ);
     GetBlockLocationsResult res = null;
+    FSPermissionChecker pc = getPermissionChecker();
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      res = getBlockLocations(src, offset, length, true, true);
+      res = getBlockLocations(pc, srcArg, offset, length, true, true);
     } catch (AccessControlException e) {
-      logAuditEvent(false, "open", src);
+      logAuditEvent(false, "open", srcArg);
       throw e;
     } finally {
       readUnlock();
     }
 
-    logAuditEvent(true, "open", src);
+    logAuditEvent(true, "open", srcArg);
 
     if (res.updateAccessTime()) {
+      byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(
+          srcArg);
+      String src = srcArg;
       writeLock();
       final long now = now();
       try {
         checkOperation(OperationCategory.WRITE);
-        INode inode = res.iip.getLastINode();
-        boolean updateAccessTime = now > inode.getAccessTime() +
-            getAccessTimePrecision();
+        /**
+         * Resolve the path again and update the atime only when the file
+         * exists.
+         *
+         * XXX: Races can still occur even after resolving the path again.
+         * For example:
+         *
+         * <ul>
+         *   <li>Get the block location for "/a/b"</li>
+         *   <li>Rename "/a/b" to "/c/b"</li>
+         *   <li>The second resolution still points to "/a/b", which is
+         *   wrong.</li>
+         * </ul>
+         *
+         * The behavior is incorrect but consistent with the one before
+         * HDFS-7463. A better fix is to change the edit log of SetTime to
+         * use inode id instead of a path.
+         */
+        src = dir.resolvePath(pc, srcArg, pathComponents);
+        final INodesInPath iip = dir.getINodesInPath(src, true);
+        INode inode = iip.getLastINode();
+        boolean updateAccessTime = inode != null &&
+            now > inode.getAccessTime() + getAccessTimePrecision();
         if (!isInSafeMode() && updateAccessTime) {
           boolean changed = FSDirAttrOp.setTimes(dir,
-              inode, -1, now, false, res.iip.getLatestSnapshotId());
+              inode, -1, now, false, iip.getLatestSnapshotId());
           if (changed) {
             getEditLog().logTimes(src, -1, now);
           }
@@ -1773,8 +1798,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * @throws IOException
    */
   GetBlockLocationsResult getBlockLocations(
-      String src, long offset, long length, boolean needBlockToken,
-      boolean checkSafeMode) throws IOException {
+      FSPermissionChecker pc, String src, long offset, long length,
+      boolean needBlockToken, boolean checkSafeMode) throws IOException {
     if (offset < 0) {
       throw new HadoopIllegalArgumentException(
           "Negative offset is not supported. File: " + src);
@@ -1784,7 +1809,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           "Negative length is not supported. File: " + src);
     }
     final GetBlockLocationsResult ret = getBlockLocationsInt(
-        src, offset, length, needBlockToken);
+        pc, src, offset, length, needBlockToken);
 
     if (checkSafeMode && isInSafeMode()) {
       for (LocatedBlock b : ret.blocks.getLocatedBlocks()) {
@@ -1805,12 +1830,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   private GetBlockLocationsResult getBlockLocationsInt(
-      final String srcArg, long offset, long length, boolean needBlockToken)
+      FSPermissionChecker pc, final String srcArg, long offset, long length,
+      boolean needBlockToken)
       throws IOException {
     String src = srcArg;
-    FSPermissionChecker pc = getPermissionChecker();
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
-    src = dir.resolvePath(pc, src, pathComponents);
+    src = dir.resolvePath(pc, srcArg, pathComponents);
     final INodesInPath iip = dir.getINodesInPath(src, true);
     final INodeFile inode = INodeFile.valueOf(iip.getLastINode(), src);
     if (isPermissionEnabled) {
@@ -1846,7 +1871,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     boolean updateAccessTime = isAccessTimeSupported() && !isInSafeMode()
         && !iip.isSnapshot()
         && now > inode.getAccessTime() + getAccessTimePrecision();
-    return new GetBlockLocationsResult(updateAccessTime ? iip : null, blocks);
+    return new GetBlockLocationsResult(updateAccessTime, blocks);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3dd6395b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index 23fea12..0cfe31a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
@@ -484,7 +484,9 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
     FSNamesystem fsn = namenode.getNamesystem();
     fsn.readLock();
     try {
-      blocks = fsn.getBlockLocations(path, 0, fileLen, false, false).blocks;
+      blocks = fsn.getBlockLocations(
+          fsn.getPermissionChecker(), path, 0, fileLen, false, false)
+          .blocks;
     } catch (FileNotFoundException fnfe) {
       blocks = null;
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3dd6395b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
index 4c3fa9c..8fe273b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
@@ -24,6 +24,7 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyBoolean;
 import static org.mockito.Matchers.anyLong;
 import static org.mockito.Matchers.anyString;
@@ -1154,10 +1155,11 @@ public class TestFsck {
     FSNamesystem fsName = mock(FSNamesystem.class);
     BlockManager blockManager = mock(BlockManager.class);
     DatanodeManager dnManager = mock(DatanodeManager.class);
-    
+
     when(namenode.getNamesystem()).thenReturn(fsName);
-    when(fsName.getBlockLocations(
-        anyString(), anyLong(), anyLong(), anyBoolean(), anyBoolean()))
+    when(fsName.getBlockLocations(any(FSPermissionChecker.class), anyString(),
+                                  anyLong(), anyLong(),
+                                  anyBoolean(), anyBoolean()))
         .thenThrow(new FileNotFoundException());
     when(fsName.getBlockManager()).thenReturn(blockManager);
     when(blockManager.getDatanodeManager()).thenReturn(dnManager);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3dd6395b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetBlockLocations.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetBlockLocations.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetBlockLocations.java
new file mode 100644
index 0000000..a19eb1d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetBlockLocations.java
@@ -0,0 +1,133 @@
+/**
+ * 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.hdfs.server.namenode;
+
+import org.apache.commons.io.Charsets;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.io.IOException;
+import java.util.ArrayList;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY;
+import static org.apache.hadoop.util.Time.now;
+import static org.mockito.Mockito.anyLong;
+import static org.mockito.Mockito.anyString;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class TestGetBlockLocations {
+  private static final String FILE_NAME = "foo";
+  private static final String FILE_PATH = "/" + FILE_NAME;
+  private static final long MOCK_INODE_ID = 16386;
+  private static final String RESERVED_PATH =
+      "/.reserved/.inodes/" + MOCK_INODE_ID;
+
+  @Test(timeout = 30000)
+  public void testResolveReservedPath() throws IOException {
+    FSNamesystem fsn = setupFileSystem();
+    FSEditLog editlog = fsn.getEditLog();
+    fsn.getBlockLocations("dummy", RESERVED_PATH, 0, 1024);
+    verify(editlog).logTimes(eq(FILE_PATH), anyLong(), anyLong());
+    fsn.close();
+  }
+
+  @Test(timeout = 30000)
+  public void testGetBlockLocationsRacingWithDelete() throws IOException {
+    FSNamesystem fsn = spy(setupFileSystem());
+    final FSDirectory fsd = fsn.getFSDirectory();
+    FSEditLog editlog = fsn.getEditLog();
+
+    doAnswer(new Answer<Void>() {
+
+      @Override
+      public Void answer(InvocationOnMock invocation) throws Throwable {
+        INodesInPath iip = fsd.getINodesInPath(FILE_PATH, true);
+        FSDirDeleteOp.delete(fsd, iip, new INode.BlocksMapUpdateInfo(),
+                             new ArrayList<INode>(), now());
+        invocation.callRealMethod();
+        return null;
+      }
+    }).when(fsn).writeLock();
+    fsn.getBlockLocations("dummy", RESERVED_PATH, 0, 1024);
+
+    verify(editlog, never()).logTimes(anyString(), anyLong(), anyLong());
+    fsn.close();
+  }
+
+  @Test(timeout = 30000)
+  public void testGetBlockLocationsRacingWithRename() throws IOException {
+    FSNamesystem fsn = spy(setupFileSystem());
+    final FSDirectory fsd = fsn.getFSDirectory();
+    FSEditLog editlog = fsn.getEditLog();
+    final String DST_PATH = "/bar";
+    final boolean[] renamed = new boolean[1];
+
+    doAnswer(new Answer<Void>() {
+
+      @Override
+      public Void answer(InvocationOnMock invocation) throws Throwable {
+        invocation.callRealMethod();
+        if (!renamed[0]) {
+          FSDirRenameOp.renameTo(fsd, fsd.getPermissionChecker(), FILE_PATH,
+                                 DST_PATH, new INode.BlocksMapUpdateInfo(),
+                                 false);
+          renamed[0] = true;
+        }
+        return null;
+      }
+    }).when(fsn).writeLock();
+    fsn.getBlockLocations("dummy", RESERVED_PATH, 0, 1024);
+
+    verify(editlog).logTimes(eq(DST_PATH), anyLong(), anyLong());
+    fsn.close();
+  }
+
+  private static FSNamesystem setupFileSystem() throws IOException {
+    Configuration conf = new Configuration();
+    conf.setLong(DFS_NAMENODE_ACCESSTIME_PRECISION_KEY, 1L);
+    FSEditLog editlog = mock(FSEditLog.class);
+    FSImage image = mock(FSImage.class);
+    when(image.getEditLog()).thenReturn(editlog);
+    final FSNamesystem fsn = new FSNamesystem(conf, image, true);
+
+    final FSDirectory fsd = fsn.getFSDirectory();
+    INodesInPath iip = fsd.getINodesInPath("/", true);
+    PermissionStatus perm = new PermissionStatus(
+        "hdfs", "supergroup",
+        FsPermission.createImmutable((short) 0x1ff));
+    final INodeFile file = new INodeFile(
+        MOCK_INODE_ID, FILE_NAME.getBytes(Charsets.UTF_8),
+        perm, 1, 1, new BlockInfoContiguous[] {}, (short) 1,
+        DFS_BLOCK_SIZE_DEFAULT);
+    fsn.getFSDirectory().addINode(iip, file);
+    return fsn;
+  }
+
+}


[02/50] [abbrv] hadoop git commit: HDFS-8211. DataNode UUID is always null in the JMX counter. (Contributed by Anu Engineer)

Posted by ji...@apache.org.
HDFS-8211. DataNode UUID is always null in the JMX counter. (Contributed by Anu Engineer)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/dcc5455e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/dcc5455e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/dcc5455e

Branch: refs/heads/HDFS-7240
Commit: dcc5455e07be75ca44eb6a33d4e706eec11b9905
Parents: 4a3dabd
Author: Arpit Agarwal <ar...@apache.org>
Authored: Fri Apr 24 16:47:48 2015 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Fri Apr 24 16:47:48 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../hadoop/hdfs/server/datanode/DataNode.java   |  4 +-
 .../hdfs/server/datanode/TestDataNodeUUID.java  | 65 ++++++++++++++++++++
 3 files changed, 70 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/dcc5455e/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 317211e..a7b5ed3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -560,6 +560,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8191. Fix byte to integer casting in SimulatedFSDataset#simulatedByte.
     (Zhe Zhang via wang)
 
+    HDFS-8211. DataNode UUID is always null in the JMX counter. (Anu Engineer
+    via Arpit Agarwal)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dcc5455e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 23ab43a..2401d9c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -1226,7 +1226,7 @@ public class DataNode extends ReconfigurableBase
    *
    * @throws IOException
    */
-  private synchronized void checkDatanodeUuid() throws IOException {
+  synchronized void checkDatanodeUuid() throws IOException {
     if (storage.getDatanodeUuid() == null) {
       storage.setDatanodeUuid(generateUuid());
       storage.writeAll();
@@ -3159,7 +3159,7 @@ public class DataNode extends ReconfigurableBase
   }
 
   public String getDatanodeUuid() {
-    return id == null ? null : id.getDatanodeUuid();
+    return storage == null ? null : storage.getDatanodeUuid();
   }
 
   boolean shouldRun() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dcc5455e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeUUID.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeUUID.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeUUID.java
new file mode 100644
index 0000000..34e53a3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeUUID.java
@@ -0,0 +1,65 @@
+/**
+ * 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.hdfs.server.datanode;
+
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.junit.Test;
+
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+
+public class TestDataNodeUUID {
+
+  /**
+   * This test makes sure that we have a valid
+   * Node ID after the checkNodeUUID is done.
+   */
+  @Test
+  public void testDatanodeUuid() throws Exception {
+
+    final InetSocketAddress NN_ADDR = new InetSocketAddress(
+      "localhost", 5020);
+    Configuration conf = new HdfsConfiguration();
+    conf.set(DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY, "0.0.0.0:0");
+    conf.set(DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
+    conf.set(DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_KEY, "0.0.0.0:0");
+    FileSystem.setDefaultUri(conf,
+      "hdfs://" + NN_ADDR.getHostName() + ":" + NN_ADDR.getPort());
+    ArrayList<StorageLocation> locations = new ArrayList<>();
+
+    DataNode dn = new DataNode(conf, locations, null);
+
+    //Assert that Node iD is null
+    String nullString = null;
+    assertEquals(dn.getDatanodeUuid(), nullString);
+
+    // CheckDataNodeUUID will create an UUID if UUID is null
+    dn.checkDatanodeUuid();
+
+    // Make sure that we have a valid DataNodeUUID at that point of time.
+    assertNotEquals(dn.getDatanodeUuid(), nullString);
+  }
+}


[32/50] [abbrv] hadoop git commit: HDFS-8280. Code Cleanup in DFSInputStream. Contributed by Jing Zhao.

Posted by ji...@apache.org.
HDFS-8280. Code Cleanup in DFSInputStream. Contributed by Jing Zhao.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/439614b0
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/439614b0
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/439614b0

Branch: refs/heads/HDFS-7240
Commit: 439614b0c8a3df3d8b7967451c5331a0e034e13a
Parents: c79e7f7
Author: Haohui Mai <wh...@apache.org>
Authored: Tue Apr 28 18:11:59 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Tue Apr 28 18:11:59 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   2 +
 .../org/apache/hadoop/hdfs/DFSInputStream.java  | 141 ++++++++-----------
 2 files changed, 61 insertions(+), 82 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/439614b0/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 1c4cfb4..e7fa8fd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -478,6 +478,8 @@ Release 2.8.0 - UNRELEASED
     HDFS-8176. Record from/to snapshots in audit log for snapshot diff report.
     (J. Andreina via jing9)
 
+    HDFS-8280. Code Cleanup in DFSInputStream. (Jing Zhao via wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/439614b0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
index 3f90397..3290223 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
@@ -601,7 +601,6 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
             targetBlock.getBlockSize() - 1;
       this.currentLocatedBlock = targetBlock;
 
-      assert (target==pos) : "Wrong postion " + pos + " expect " + target;
       long offsetIntoBlock = target - targetBlock.getStartOffset();
 
       DNAddrPair retval = chooseDataNode(targetBlock, null);
@@ -610,35 +609,12 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       StorageType storageType = retval.storageType;
 
       try {
-        ExtendedBlock blk = targetBlock.getBlock();
-        Token<BlockTokenIdentifier> accessToken = targetBlock.getBlockToken();
-        CachingStrategy curCachingStrategy;
-        boolean shortCircuitForbidden;
-        synchronized(infoLock) {
-          curCachingStrategy = cachingStrategy;
-          shortCircuitForbidden = shortCircuitForbidden();
-        }
-        blockReader = new BlockReaderFactory(dfsClient.getConf()).
-            setInetSocketAddress(targetAddr).
-            setRemotePeerFactory(dfsClient).
-            setDatanodeInfo(chosenNode).
-            setStorageType(storageType).
-            setFileName(src).
-            setBlock(blk).
-            setBlockToken(accessToken).
-            setStartOffset(offsetIntoBlock).
-            setVerifyChecksum(verifyChecksum).
-            setClientName(dfsClient.clientName).
-            setLength(blk.getNumBytes() - offsetIntoBlock).
-            setCachingStrategy(curCachingStrategy).
-            setAllowShortCircuitLocalReads(!shortCircuitForbidden).
-            setClientCacheContext(dfsClient.getClientContext()).
-            setUserGroupInformation(dfsClient.ugi).
-            setConfiguration(dfsClient.getConfiguration()).
-            build();
+        blockReader = getBlockReader(targetBlock, offsetIntoBlock,
+            targetBlock.getBlockSize() - offsetIntoBlock, targetAddr,
+            storageType, chosenNode);
         if(connectFailedOnce) {
           DFSClient.LOG.info("Successfully connected to " + targetAddr +
-                             " for " + blk);
+                             " for " + targetBlock.getBlock());
         }
         return chosenNode;
       } catch (IOException ex) {
@@ -663,6 +639,37 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     }
   }
 
+  protected BlockReader getBlockReader(LocatedBlock targetBlock,
+      long offsetInBlock, long length, InetSocketAddress targetAddr,
+      StorageType storageType, DatanodeInfo datanode) throws IOException {
+    ExtendedBlock blk = targetBlock.getBlock();
+    Token<BlockTokenIdentifier> accessToken = targetBlock.getBlockToken();
+    CachingStrategy curCachingStrategy;
+    boolean shortCircuitForbidden;
+    synchronized (infoLock) {
+      curCachingStrategy = cachingStrategy;
+      shortCircuitForbidden = shortCircuitForbidden();
+    }
+    return new BlockReaderFactory(dfsClient.getConf()).
+        setInetSocketAddress(targetAddr).
+        setRemotePeerFactory(dfsClient).
+        setDatanodeInfo(datanode).
+        setStorageType(storageType).
+        setFileName(src).
+        setBlock(blk).
+        setBlockToken(accessToken).
+        setStartOffset(offsetInBlock).
+        setVerifyChecksum(verifyChecksum).
+        setClientName(dfsClient.clientName).
+        setLength(length).
+        setCachingStrategy(curCachingStrategy).
+        setAllowShortCircuitLocalReads(!shortCircuitForbidden).
+        setClientCacheContext(dfsClient.getClientContext()).
+        setUserGroupInformation(dfsClient.ugi).
+        setConfiguration(dfsClient.getConfiguration()).
+        build();
+  }
+
   /**
    * Close it down!
    */
@@ -935,9 +942,10 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
   private DNAddrPair chooseDataNode(LocatedBlock block,
       Collection<DatanodeInfo> ignoredNodes) throws IOException {
     while (true) {
-      try {
-        return getBestNodeDNAddrPair(block, ignoredNodes);
-      } catch (IOException ie) {
+      DNAddrPair result = getBestNodeDNAddrPair(block, ignoredNodes);
+      if (result != null) {
+        return result;
+      } else {
         String errMsg = getBestNodeDNAddrPairErrorString(block.getLocations(),
           deadNodes, ignoredNodes);
         String blockInfo = block.getBlock() + " file=" + src;
@@ -954,7 +962,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
           DFSClient.LOG.info("No node available for " + blockInfo);
         }
         DFSClient.LOG.info("Could not obtain " + block.getBlock()
-            + " from any node: " + ie + errMsg
+            + " from any node: " + errMsg
             + ". Will get new block locations from namenode and retry...");
         try {
           // Introducing a random factor to the wait time before another retry.
@@ -977,7 +985,6 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         openInfo();
         block = getBlockAt(block.getStartOffset());
         failures++;
-        continue;
       }
     }
   }
@@ -986,11 +993,10 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    * Get the best node from which to stream the data.
    * @param block LocatedBlock, containing nodes in priority order.
    * @param ignoredNodes Do not choose nodes in this array (may be null)
-   * @return The DNAddrPair of the best node.
-   * @throws IOException
+   * @return The DNAddrPair of the best node. Null if no node can be chosen.
    */
   private DNAddrPair getBestNodeDNAddrPair(LocatedBlock block,
-      Collection<DatanodeInfo> ignoredNodes) throws IOException {
+      Collection<DatanodeInfo> ignoredNodes) {
     DatanodeInfo[] nodes = block.getLocations();
     StorageType[] storageTypes = block.getStorageTypes();
     DatanodeInfo chosenNode = null;
@@ -1010,9 +1016,10 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       }
     }
     if (chosenNode == null) {
-      throw new IOException("No live nodes contain block " + block.getBlock() +
+      DFSClient.LOG.warn("No live nodes contain block " + block.getBlock() +
           " after checking nodes = " + Arrays.toString(nodes) +
           ", ignoredNodes = " + ignoredNodes);
+      return null;
     }
     final String dnAddr =
         chosenNode.getXferAddr(dfsClient.getConf().isConnectToDnViaHostname());
@@ -1102,40 +1109,13 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       // cached block locations may have been updated by chooseDataNode()
       // or fetchBlockAt(). Always get the latest list of locations at the
       // start of the loop.
-      CachingStrategy curCachingStrategy;
-      boolean allowShortCircuitLocalReads;
       LocatedBlock block = getBlockAt(blockStartOffset);
-      synchronized(infoLock) {
-        curCachingStrategy = cachingStrategy;
-        allowShortCircuitLocalReads = !shortCircuitForbidden();
-      }
-      DatanodeInfo chosenNode = datanode.info;
-      InetSocketAddress targetAddr = datanode.addr;
-      StorageType storageType = datanode.storageType;
       BlockReader reader = null;
-
       try {
         DFSClientFaultInjector.get().fetchFromDatanodeException();
-        Token<BlockTokenIdentifier> blockToken = block.getBlockToken();
         int len = (int) (end - start + 1);
-        reader = new BlockReaderFactory(dfsClient.getConf()).
-            setInetSocketAddress(targetAddr).
-            setRemotePeerFactory(dfsClient).
-            setDatanodeInfo(chosenNode).
-            setStorageType(storageType).
-            setFileName(src).
-            setBlock(block.getBlock()).
-            setBlockToken(blockToken).
-            setStartOffset(start).
-            setVerifyChecksum(verifyChecksum).
-            setClientName(dfsClient.clientName).
-            setLength(len).
-            setCachingStrategy(curCachingStrategy).
-            setAllowShortCircuitLocalReads(allowShortCircuitLocalReads).
-            setClientCacheContext(dfsClient.getClientContext()).
-            setUserGroupInformation(dfsClient.ugi).
-            setConfiguration(dfsClient.getConfiguration()).
-            build();
+        reader = getBlockReader(block, start, len, datanode.addr,
+            datanode.storageType, datanode.info);
         int nread = reader.readAll(buf, offset, len);
         updateReadStatistics(readStatistics, nread, reader);
 
@@ -1148,34 +1128,33 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       } catch (ChecksumException e) {
         String msg = "fetchBlockByteRange(). Got a checksum exception for "
             + src + " at " + block.getBlock() + ":" + e.getPos() + " from "
-            + chosenNode;
+            + datanode.info;
         DFSClient.LOG.warn(msg);
         // we want to remember what we have tried
-        addIntoCorruptedBlockMap(block.getBlock(), chosenNode, corruptedBlockMap);
-        addToDeadNodes(chosenNode);
+        addIntoCorruptedBlockMap(block.getBlock(), datanode.info,
+            corruptedBlockMap);
+        addToDeadNodes(datanode.info);
         throw new IOException(msg);
       } catch (IOException e) {
         if (e instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
           DFSClient.LOG.info("Will fetch a new encryption key and retry, " 
-              + "encryption key was invalid when connecting to " + targetAddr
+              + "encryption key was invalid when connecting to " + datanode.addr
               + " : " + e);
           // The encryption key used is invalid.
           refetchEncryptionKey--;
           dfsClient.clearDataEncryptionKey();
-          continue;
-        } else if (refetchToken > 0 && tokenRefetchNeeded(e, targetAddr)) {
+        } else if (refetchToken > 0 && tokenRefetchNeeded(e, datanode.addr)) {
           refetchToken--;
           try {
             fetchBlockAt(block.getStartOffset());
           } catch (IOException fbae) {
             // ignore IOE, since we can retry it later in a loop
           }
-          continue;
         } else {
-          String msg = "Failed to connect to " + targetAddr + " for file "
+          String msg = "Failed to connect to " + datanode.addr + " for file "
               + src + " for block " + block.getBlock() + ":" + e;
           DFSClient.LOG.warn("Connection failure: " + msg, e);
-          addToDeadNodes(chosenNode);
+          addToDeadNodes(datanode.info);
           throw new IOException(msg);
         }
       } finally {
@@ -1187,10 +1166,9 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
   }
 
   /**
-   * Like {@link #fetchBlockByteRange(LocatedBlock, long, long, byte[],
-   * int, Map)} except we start up a second, parallel, 'hedged' read
-   * if the first read is taking longer than configured amount of
-   * time.  We then wait on which ever read returns first.
+   * Like {@link #fetchBlockByteRange} except we start up a second, parallel,
+   * 'hedged' read if the first read is taking longer than configured amount of
+   * time. We then wait on which ever read returns first.
    */
   private void hedgedFetchBlockByteRange(long blockStartOffset, long start,
       long end, byte[] buf, int offset,
@@ -1248,9 +1226,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         // ongoing. Call getBestNodeDNAddrPair instead of chooseDataNode.
         // If no nodes to do hedged reads against, pass.
         try {
-          try {
-            chosenNode = getBestNodeDNAddrPair(block, ignored);
-          } catch (IOException ioe) {
+          chosenNode = getBestNodeDNAddrPair(block, ignored);
+          if (chosenNode == null) {
             chosenNode = chooseDataNode(block, ignored);
           }
           bb = ByteBuffer.allocate(len);


[31/50] [abbrv] hadoop git commit: HDFS-8273. FSNamesystem#Delete() should not call logSync() when holding the lock. Contributed by Haohui Mai.

Posted by ji...@apache.org.
HDFS-8273. FSNamesystem#Delete() should not call logSync() when holding the lock. Contributed by Haohui Mai.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/c79e7f7d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c79e7f7d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c79e7f7d

Branch: refs/heads/HDFS-7240
Commit: c79e7f7d997596e0c38ae4cddff2bd0910581c16
Parents: 5190923
Author: Haohui Mai <wh...@apache.org>
Authored: Tue Apr 28 18:05:46 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Tue Apr 28 18:05:46 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +++
 .../hdfs/server/namenode/FSDirDeleteOp.java     |  1 -
 .../hdfs/server/namenode/FSNamesystem.java      | 22 ++++++++++++--------
 3 files changed, 16 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c79e7f7d/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 5d9e983..1c4cfb4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -627,6 +627,9 @@ Release 2.7.1 - UNRELEASED
     HDFS-8070. Pre-HDFS-7915 DFSClient cannot use short circuit on
     post-HDFS-7915 DataNode (cmccabe)
 
+    HDFS-8273. FSNamesystem#Delete() should not call logSync() when holding the
+    lock. (wheat9)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c79e7f7d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirDeleteOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirDeleteOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirDeleteOp.java
index 02eb1de..2192c24 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirDeleteOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirDeleteOp.java
@@ -175,7 +175,6 @@ class FSDirDeleteOp {
     incrDeletedFileCount(filesRemoved);
 
     fsn.removeLeasesAndINodes(src, removedINodes, true);
-    fsd.getEditLog().logSync();
 
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* Namesystem.delete: "

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c79e7f7d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 229c4d1..0ec81d8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -3690,6 +3690,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     } finally {
       writeUnlock();
     }
+    getEditLog().logSync();
     if (toRemovedBlocks != null) {
       removeBlocks(toRemovedBlocks); // Incremental deletion of blocks
     }
@@ -4695,22 +4696,21 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
      * blocks and unlink them from the namespace.
      */
     private void clearCorruptLazyPersistFiles()
-        throws SafeModeException, AccessControlException,
-        UnresolvedLinkException, IOException {
+        throws IOException {
 
       BlockStoragePolicy lpPolicy = blockManager.getStoragePolicy("LAZY_PERSIST");
 
-      List<BlockCollection> filesToDelete = new ArrayList<BlockCollection>();
-
+      List<BlockCollection> filesToDelete = new ArrayList<>();
+      boolean changed = false;
       writeLock();
-
       try {
         final Iterator<Block> it = blockManager.getCorruptReplicaBlockIterator();
 
         while (it.hasNext()) {
           Block b = it.next();
           BlockInfoContiguous blockInfo = blockManager.getStoredBlock(b);
-          if (blockInfo.getBlockCollection().getStoragePolicyID() == lpPolicy.getId()) {
+          if (blockInfo.getBlockCollection().getStoragePolicyID()
+              == lpPolicy.getId()) {
             filesToDelete.add(blockInfo.getBlockCollection());
           }
         }
@@ -4718,9 +4718,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         for (BlockCollection bc : filesToDelete) {
           LOG.warn("Removing lazyPersist file " + bc.getName() + " with no replicas.");
           BlocksMapUpdateInfo toRemoveBlocks =
-          FSDirDeleteOp.deleteInternal(
-              FSNamesystem.this, bc.getName(),
-              INodesInPath.fromINode((INodeFile) bc), false);
+              FSDirDeleteOp.deleteInternal(
+                  FSNamesystem.this, bc.getName(),
+                  INodesInPath.fromINode((INodeFile) bc), false);
+          changed |= toRemoveBlocks != null;
           if (toRemoveBlocks != null) {
             removeBlocks(toRemoveBlocks); // Incremental deletion of blocks
           }
@@ -4728,6 +4729,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       } finally {
         writeUnlock();
       }
+      if (changed) {
+        getEditLog().logSync();
+      }
     }
 
     @Override


[25/50] [abbrv] hadoop git commit: HDFS-8232. Missing datanode counters when using Metrics2 sink interface. Contributed by Anu Engineer.

Posted by ji...@apache.org.
HDFS-8232. Missing datanode counters when using Metrics2 sink interface. Contributed by Anu Engineer.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/feb68cb5
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/feb68cb5
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/feb68cb5

Branch: refs/heads/HDFS-7240
Commit: feb68cb5470dc3e6c16b6bc1549141613e360601
Parents: db1b674
Author: cnauroth <cn...@apache.org>
Authored: Mon Apr 27 16:48:13 2015 -0700
Committer: cnauroth <cn...@apache.org>
Committed: Mon Apr 27 16:48:13 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |  27 ++++
 .../datanode/metrics/DataNodeMetricHelper.java  |  79 +++++++++++
 .../server/datanode/metrics/FSDatasetMBean.java |   3 +-
 .../server/datanode/SimulatedFSDataset.java     |  20 ++-
 .../datanode/TestDataNodeFSDataSetSink.java     | 136 +++++++++++++++++++
 .../extdataset/ExternalDatasetImpl.java         |  19 ++-
 7 files changed, 281 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/feb68cb5/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index d56ea0c..326de0b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -574,6 +574,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8205. CommandFormat#parse() should not parse option as
     value of option. (Peter Shi and Xiaoyu Yao via Arpit Agarwal)
 
+    HDFS-8232. Missing datanode counters when using Metrics2 sink interface.
+    (Anu Engineer via cnauroth)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/feb68cb5/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index 8869f5a..b87daec 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -71,6 +71,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeMetricHelper;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
 import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
 import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
@@ -104,6 +105,9 @@ import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.MultipleIOException;
 import org.apache.hadoop.io.nativeio.NativeIO;
+import org.apache.hadoop.metrics2.MetricsCollector;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.util.MBeans;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.DataChecksum;
@@ -316,6 +320,13 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     lazyWriter = new Daemon(new LazyWriter(conf));
     lazyWriter.start();
     registerMBean(datanode.getDatanodeUuid());
+
+    // Add a Metrics2 Source Interface. This is same
+    // data as MXBean. We can remove the registerMbean call
+    // in a release where we can break backward compatibility
+    MetricsSystem ms = DefaultMetricsSystem.instance();
+    ms.register("FSDatasetState", "FSDatasetState", this);
+
     localFS = FileSystem.getLocal(conf);
     blockPinningEnabled = conf.getBoolean(
       DFSConfigKeys.DFS_DATANODE_BLOCK_PINNING_ENABLED,
@@ -636,6 +647,22 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     return cacheManager.getNumBlocksFailedToUncache();
   }
 
+  /**
+   * Get metrics from the metrics source
+   *
+   * @param collector to contain the resulting metrics snapshot
+   * @param all if true, return all metrics even if unchanged.
+   */
+  @Override
+  public void getMetrics(MetricsCollector collector, boolean all) {
+    try {
+      DataNodeMetricHelper.getMetrics(collector, this, "FSDatasetState");
+    } catch (Exception e) {
+        LOG.warn("Exception thrown while metric collection. Exception : "
+          + e.getMessage());
+    }
+  }
+
   @Override // FSDatasetMBean
   public long getNumBlocksCached() {
     return cacheManager.getNumBlocksCached();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/feb68cb5/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetricHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetricHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetricHelper.java
new file mode 100644
index 0000000..8bbe08b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetricHelper.java
@@ -0,0 +1,79 @@
+/**
+ * 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.hdfs.server.datanode.metrics;
+
+import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
+import org.apache.hadoop.metrics2.MetricsCollector;
+import org.apache.hadoop.metrics2.MetricsTag;
+import org.apache.hadoop.metrics2.lib.Interns;
+
+import java.io.IOException;
+
+public class DataNodeMetricHelper {
+
+  /**
+   * Get metrics helper provides Helper function for
+   * metrics2 interface to act as a Metric source
+   *
+   * @param collector Metrics Collector that is passed in
+   * @param beanClass The Class that currently impliments the metric functions
+   * @param context A string that idenitifies the context
+   *
+   * @throws IOException
+   */
+  public static void getMetrics(MetricsCollector collector,
+                                FSDatasetMBean beanClass, String context)
+    throws IOException {
+
+    if (beanClass == null) {
+      throw new IOException("beanClass cannot be null");
+    }
+
+    String className = beanClass.getClass().getName();
+
+    collector.addRecord(className)
+      .setContext(context)
+      .addGauge(Interns.info("Capacity", "Total storage capacity"),
+        beanClass.getCapacity())
+      .addGauge(Interns.info("DfsUsed", "Total bytes used by dfs datanode"),
+        beanClass.getDfsUsed())
+      .addGauge(Interns.info("Remaining", "Total bytes of free storage"),
+        beanClass.getRemaining())
+      .add(new MetricsTag(Interns.info("StorageInfo", "Storage ID"),
+        beanClass.getStorageInfo()))
+      .addGauge(Interns.info("NumFailedVolumes", "Number of failed Volumes" +
+        " in the data Node"), beanClass.getNumFailedVolumes())
+      .addGauge(Interns.info("LastVolumeFailureDate", "Last Volume failure in" +
+        " milliseconds from epoch"), beanClass.getLastVolumeFailureDate())
+      .addGauge(Interns.info("EstimatedCapacityLostTotal", "Total capacity lost"
+        + " due to volume failure"), beanClass.getEstimatedCapacityLostTotal())
+      .addGauge(Interns.info("CacheUsed", "Datanode cache used in bytes"),
+        beanClass.getCacheUsed())
+      .addGauge(Interns.info("CacheCapacity", "Datanode cache capacity"),
+        beanClass.getCacheCapacity())
+      .addGauge(Interns.info("NumBlocksCached", "Datanode number" +
+        " of blocks cached"), beanClass.getNumBlocksCached())
+      .addGauge(Interns.info("NumBlocksFailedToCache", "Datanode number of " +
+        "blocks failed to cache"), beanClass.getNumBlocksFailedToCache())
+      .addGauge(Interns.info("NumBlocksFailedToUnCache", "Datanode number of" +
+          " blocks failed in cache eviction"),
+        beanClass.getNumBlocksFailedToUncache());
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/feb68cb5/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/FSDatasetMBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/FSDatasetMBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/FSDatasetMBean.java
index 5f22540..c2f175b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/FSDatasetMBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/FSDatasetMBean.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.datanode.metrics;
 import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.metrics2.MetricsSource;
 
 /**
  * 
@@ -37,7 +38,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
  *
  */
 @InterfaceAudience.Private
-public interface FSDatasetMBean {
+public interface FSDatasetMBean extends MetricsSource {
   
   /**
    * Returns the total space (in bytes) used by a block pool

http://git-wip-us.apache.org/repos/asf/hadoop/blob/feb68cb5/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
index 060e055..bc24237 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
@@ -23,8 +23,6 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.nio.channels.ClosedChannelException;
-import java.util.ArrayList;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.LinkedList;
@@ -52,6 +50,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaInputStreams;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
+import org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeMetricHelper;
 import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
@@ -60,9 +59,9 @@ import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.metrics2.MetricsCollector;
 import org.apache.hadoop.metrics2.util.MBeans;
 import org.apache.hadoop.util.DataChecksum;
-import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 
 /**
  * This class implements a simulated FSDataset.
@@ -690,6 +689,21 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
     return 0l;
   }
 
+  /**
+   * Get metrics from the metrics source
+   *
+   * @param collector to contain the resulting metrics snapshot
+   * @param all if true, return all metrics even if unchanged.
+   */
+  @Override
+  public void getMetrics(MetricsCollector collector, boolean all) {
+    try {
+      DataNodeMetricHelper.getMetrics(collector, this, "SimulatedFSDataset");
+    } catch (Exception e){
+        //ignore Exceptions
+    }
+  }
+
   @Override // FsDatasetSpi
   public synchronized long getLength(ExtendedBlock b) throws IOException {
     final Map<Block, BInfo> map = getMap(b.getBlockPoolId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/feb68cb5/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeFSDataSetSink.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeFSDataSetSink.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeFSDataSetSink.java
new file mode 100644
index 0000000..dbd6bb0
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeFSDataSetSink.java
@@ -0,0 +1,136 @@
+/**
+ * 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.hdfs.server.datanode;
+
+import org.apache.commons.configuration.SubsetConfiguration;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.metrics2.AbstractMetric;
+import org.apache.hadoop.metrics2.MetricsRecord;
+import org.apache.hadoop.metrics2.MetricsSink;
+import org.apache.hadoop.metrics2.MetricsTag;
+import org.apache.hadoop.metrics2.impl.MetricsSystemImpl;
+import org.junit.Test;
+
+import java.util.Set;
+import java.util.TreeSet;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestDataNodeFSDataSetSink {
+  private static final MetricsSystemImpl ms = new
+    MetricsSystemImpl("TestFSDataSet");
+
+  class FSDataSetSinkTest implements MetricsSink {
+    private Set<String> nameMap;
+    private int count;
+
+    /**
+     * add a metrics record in the sink
+     *
+     * @param record the record to add
+     */
+    @Override
+    public void putMetrics(MetricsRecord record) {
+      // let us do this only once, otherwise
+      // our count could go out of sync.
+      if (count == 0) {
+        for (AbstractMetric m : record.metrics()) {
+          if (nameMap.contains(m.name())) {
+            count++;
+          }
+        }
+
+        for (MetricsTag t : record.tags()) {
+          if (nameMap.contains(t.name())) {
+            count++;
+          }
+        }
+      }
+    }
+
+    /**
+     * Flush any buffered metrics
+     */
+    @Override
+    public void flush() {
+
+    }
+
+    /**
+     * Initialize the plugin
+     *
+     * @param conf the configuration object for the plugin
+     */
+    @Override
+    public void init(SubsetConfiguration conf) {
+      nameMap = new TreeSet<>();
+      nameMap.add("DfsUsed");
+      nameMap.add("Capacity");
+      nameMap.add("Remaining");
+      nameMap.add("StorageInfo");
+      nameMap.add("NumFailedVolumes");
+      nameMap.add("LastVolumeFailureDate");
+      nameMap.add("EstimatedCapacityLostTotal");
+      nameMap.add("CacheUsed");
+      nameMap.add("CacheCapacity");
+      nameMap.add("NumBlocksCached");
+      nameMap.add("NumBlocksFailedToCache");
+      nameMap.add("NumBlocksFailedToUnCache");
+      nameMap.add("Context");
+      nameMap.add("Hostname");
+    }
+
+    public int getMapCount() {
+      return nameMap.size();
+    }
+
+    public int getFoundKeyCount() {
+      return count;
+    }
+  }
+
+  @Test
+  /**
+   * This test creates a Source and then calls into the Sink that we
+   * have registered. That is calls into FSDataSetSinkTest
+   */
+  public void testFSDataSetMetrics() throws InterruptedException {
+    Configuration conf = new HdfsConfiguration();
+    String bpid = "FSDatSetSink-Test";
+    SimulatedFSDataset fsdataset = new SimulatedFSDataset(null, conf);
+    fsdataset.addBlockPool(bpid, conf);
+    FSDataSetSinkTest sink = new FSDataSetSinkTest();
+    sink.init(null);
+    ms.init("Test");
+    ms.start();
+    ms.register("FSDataSetSource", "FSDataSetSource", fsdataset);
+    ms.register("FSDataSetSink", "FSDataSetSink", sink);
+    ms.startMetricsMBeans();
+    ms.publishMetricsNow();
+
+    Thread.sleep(4000);
+
+    ms.stopMetricsMBeans();
+    ms.shutdown();
+
+    // make sure we got all expected metric in the call back
+    assertEquals(sink.getMapCount(), sink.getFoundKeyCount());
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/feb68cb5/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
index 2c6d868..b7c2028 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
@@ -35,14 +35,14 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaInputStreams;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
+import org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeMetricHelper;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
-import org.apache.hadoop.util.DiskChecker;
-import org.apache.hadoop.util.DiskChecker.DiskErrorException;
+import org.apache.hadoop.metrics2.MetricsCollector;
 
 public class ExternalDatasetImpl implements FsDatasetSpi<ExternalVolumeImpl> {
 
@@ -420,6 +420,21 @@ public class ExternalDatasetImpl implements FsDatasetSpi<ExternalVolumeImpl> {
     return 0;
   }
 
+  /**
+   * Get metrics from the metrics source
+   *
+   * @param collector to contain the resulting metrics snapshot
+   * @param all if true, return all metrics even if unchanged.
+   */
+  @Override
+  public void getMetrics(MetricsCollector collector, boolean all) {
+    try {
+      DataNodeMetricHelper.getMetrics(collector, this, "ExternalDataset");
+    } catch (Exception e){
+        //ignore exceptions
+    }
+  }
+
   @Override
   public void setPinning(ExtendedBlock block) throws IOException {    
   }


[12/50] [abbrv] hadoop git commit: HADOOP-11865. Incorrect path mentioned in document for accessing script files (J.Andreina via aw)

Posted by ji...@apache.org.
HADOOP-11865. Incorrect path mentioned in document for accessing script files (J.Andreina via aw)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/8b69c825
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8b69c825
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8b69c825

Branch: refs/heads/HDFS-7240
Commit: 8b69c825e571967597134518568fe5cac8cb3f46
Parents: 47279c3
Author: Allen Wittenauer <aw...@apache.org>
Authored: Sun Apr 26 09:55:46 2015 -0700
Committer: Allen Wittenauer <aw...@apache.org>
Committed: Sun Apr 26 09:55:46 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt                  | 3 +++
 hadoop-common-project/hadoop-kms/src/site/markdown/index.md.vm   | 2 +-
 .../hadoop-hdfs/src/site/markdown/HdfsUserGuide.md               | 4 ++--
 3 files changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b69c825/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 826c77e..5ba71a4 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -445,6 +445,9 @@ Trunk (Unreleased)
 
     HADOOP-11797. releasedocmaker.py needs to put ASF headers on output (aw)
 
+    HADOOP-11865. Incorrect path mentioned in document for accessing script
+    files (J.Andreina via aw)
+
   OPTIMIZATIONS
 
     HADOOP-7761. Improve the performance of raw comparisons. (todd)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b69c825/hadoop-common-project/hadoop-kms/src/site/markdown/index.md.vm
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/site/markdown/index.md.vm b/hadoop-common-project/hadoop-kms/src/site/markdown/index.md.vm
index 44b5bfb..ad4bfca 100644
--- a/hadoop-common-project/hadoop-kms/src/site/markdown/index.md.vm
+++ b/hadoop-common-project/hadoop-kms/src/site/markdown/index.md.vm
@@ -101,7 +101,7 @@ The Aggregation interval is configured via the property :
 
 $H3 Start/Stop the KMS
 
-To start/stop KMS use KMS's bin/kms.sh script. For example:
+To start/stop KMS use KMS's sbin/kms.sh script. For example:
 
     hadoop-${project.version} $ sbin/kms.sh start
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b69c825/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsUserGuide.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsUserGuide.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsUserGuide.md
index ffd8532..54197a5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsUserGuide.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsUserGuide.md
@@ -307,7 +307,7 @@ When Hadoop is upgraded on an existing cluster, as with any software upgrade, it
 
 *   Stop the cluster and distribute new version of Hadoop.
 
-*   Run the new version with `-upgrade` option (`bin/start-dfs.sh -upgrade`).
+*   Run the new version with `-upgrade` option (`sbin/start-dfs.sh -upgrade`).
 
 *   Most of the time, cluster works just fine. Once the new HDFS is
     considered working well (may be after a few days of operation),
@@ -319,7 +319,7 @@ When Hadoop is upgraded on an existing cluster, as with any software upgrade, it
 
     * stop the cluster and distribute earlier version of Hadoop.
 
-    * start the cluster with rollback option. (`bin/start-dfs.sh -rollback`).
+    * start the cluster with rollback option. (`sbin/start-dfs.sh -rollback`).
 
 When upgrading to a new version of HDFS, it is necessary to rename or delete any paths that are reserved in the new version of HDFS. If the NameNode encounters a reserved path during upgrade, it will print an error like the following:
 


[16/50] [abbrv] hadoop git commit: MAPREDUCE-6252. JobHistoryServer should not fail when encountering a missing directory. Contributed by Craig Welch.

Posted by ji...@apache.org.
MAPREDUCE-6252. JobHistoryServer should not fail when encountering a
missing directory. Contributed by Craig Welch.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/5e67c4d3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5e67c4d3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5e67c4d3

Branch: refs/heads/HDFS-7240
Commit: 5e67c4d384193b38a85655c8f93193596821faa5
Parents: 618ba70
Author: Devaraj K <de...@apache.org>
Authored: Mon Apr 27 15:01:42 2015 +0530
Committer: Devaraj K <de...@apache.org>
Committed: Mon Apr 27 15:01:42 2015 +0530

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |  3 +++
 .../mapreduce/v2/hs/HistoryFileManager.java     | 19 ++++++++++-------
 .../mapreduce/v2/hs/TestHistoryFileManager.java | 22 ++++++++++++++++++++
 3 files changed, 37 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e67c4d3/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 4166e6a..f9488fb 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -349,6 +349,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6333. TestEvents,TestAMWebServicesTasks,TestAppController are
     broken due to MAPREDUCE-6297. (Siqi Li via gera)
 
+    MAPREDUCE-6252. JobHistoryServer should not fail when encountering a 
+    missing directory. (Craig Welch via devaraj)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e67c4d3/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryFileManager.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryFileManager.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryFileManager.java
index 65f8a4f..69f814d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryFileManager.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryFileManager.java
@@ -740,17 +740,22 @@ public class HistoryFileManager extends AbstractService {
     }
   }
 
-  private static List<FileStatus> scanDirectory(Path path, FileContext fc,
+  @VisibleForTesting
+  protected static List<FileStatus> scanDirectory(Path path, FileContext fc,
       PathFilter pathFilter) throws IOException {
     path = fc.makeQualified(path);
     List<FileStatus> jhStatusList = new ArrayList<FileStatus>();
-    RemoteIterator<FileStatus> fileStatusIter = fc.listStatus(path);
-    while (fileStatusIter.hasNext()) {
-      FileStatus fileStatus = fileStatusIter.next();
-      Path filePath = fileStatus.getPath();
-      if (fileStatus.isFile() && pathFilter.accept(filePath)) {
-        jhStatusList.add(fileStatus);
+    try {
+      RemoteIterator<FileStatus> fileStatusIter = fc.listStatus(path);
+      while (fileStatusIter.hasNext()) {
+        FileStatus fileStatus = fileStatusIter.next();
+        Path filePath = fileStatus.getPath();
+        if (fileStatus.isFile() && pathFilter.accept(filePath)) {
+          jhStatusList.add(fileStatus);
+        }
       }
+    } catch (FileNotFoundException fe) {
+      LOG.error("Error while scanning directory " + path, fe);
     }
     return jhStatusList;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e67c4d3/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestHistoryFileManager.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestHistoryFileManager.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestHistoryFileManager.java
index e2e943a..1c5cc5c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestHistoryFileManager.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestHistoryFileManager.java
@@ -21,13 +21,17 @@ package org.apache.hadoop.mapreduce.v2.hs;
 
 import java.io.File;
 import java.io.FileOutputStream;
+import java.io.FileNotFoundException;
 import java.util.UUID;
+import java.util.List;
 
 import org.junit.Assert;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -45,6 +49,8 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TestName;
 
+import static org.mockito.Mockito.*;
+
 public class TestHistoryFileManager {
   private static MiniDFSCluster dfsCluster = null;
   private static MiniDFSCluster dfsCluster2 = null;
@@ -199,4 +205,20 @@ public class TestHistoryFileManager {
     testCreateHistoryDirs(dfsCluster.getConfiguration(0), clock);
   }
 
+  @Test
+  public void testScanDirectory() throws Exception {
+
+    Path p = new Path("any");
+    FileContext fc = mock(FileContext.class);
+    when(fc.makeQualified(p)).thenReturn(p);
+    when(fc.listStatus(p)).thenThrow(new FileNotFoundException());
+
+    List<FileStatus> lfs = HistoryFileManager.scanDirectory(p, fc, null);
+
+    //primarily, succcess is that an exception was not thrown.  Also nice to
+    //check this
+    Assert.assertNotNull(lfs);
+
+  }
+
 }


[22/50] [abbrv] hadoop git commit: Update CHANGES.txt - Pulled in YARN-3465, YARN-3516, and YARN-3464 to branch-2.7 (for 2.7.1)

Posted by ji...@apache.org.
Update CHANGES.txt - Pulled in YARN-3465, YARN-3516, and YARN-3464 to branch-2.7 (for 2.7.1)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/32cd2c8d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/32cd2c8d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/32cd2c8d

Branch: refs/heads/HDFS-7240
Commit: 32cd2c8d429ddb87348299c00b7d851246a25b4e
Parents: 9fec02c
Author: Karthik Kambatla <ka...@apache.org>
Authored: Mon Apr 27 13:42:45 2015 -0700
Committer: Karthik Kambatla <ka...@apache.org>
Committed: Mon Apr 27 13:43:43 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt | 18 +++++++++---------
 1 file changed, 9 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/32cd2c8d/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index fdc3f4a..1ac7a13 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -233,9 +233,6 @@ Release 2.8.0 - UNRELEASED
     YARN-2890. MiniYarnCluster should turn on timeline service if
     configured to do so. (Mit Desai via hitesh)
 
-    YARN-3465. Use LinkedHashMap to preserve order of resource requests. 
-    (Zhihai Xu via kasha)
-
     YARN-3266. RMContext#inactiveNodes should have NodeId as map key.
     (Chengbing Liu via jianhe)
 
@@ -257,9 +254,6 @@ Release 2.8.0 - UNRELEASED
     YARN-3434. Interaction between reservations and userlimit can result in 
     significant ULF violation (tgraves)
 
-    YARN-3516. killing ContainerLocalizer action doesn't take effect when
-    private localizer receives FETCH_FAILURE status.(zhihai xu via xgong)
-
     YARN-3387. Previous AM's container completed status couldn't pass to current
     AM if AM and RM restarted during the same time. (sandflee via jianhe)
 
@@ -268,9 +262,6 @@ Release 2.8.0 - UNRELEASED
     YARN-3537. NPE when NodeManager.serviceInit fails and stopRecoveryStore
     invoked (Brahma Reddy Battula via jlowe)
 
-    YARN-3464. Race condition in LocalizerRunner kills localizer before 
-    localizing all resources. (Zhihai Xu via kasha)
-
     YARN-3530. ATS throws exception on trying to filter results without otherinfo.
     (zhijie shen via xgong)
 
@@ -306,6 +297,15 @@ Release 2.7.1 - UNRELEASED
     YARN-3472. Fixed possible leak in DelegationTokenRenewer#allTokens.
     (Rohith Sharmaks via jianhe)
 
+    YARN-3465. Use LinkedHashMap to preserve order of resource requests. 
+    (Zhihai Xu via kasha)
+
+    YARN-3516. killing ContainerLocalizer action doesn't take effect when
+    private localizer receives FETCH_FAILURE status.(zhihai xu via xgong)
+
+    YARN-3464. Race condition in LocalizerRunner kills localizer before 
+    localizing all resources. (Zhihai Xu via kasha)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES


[43/50] [abbrv] hadoop git commit: YARN-3564. Fix TestContainerAllocation.testAMContainerAllocationWhenDNSUnavailable fails randomly. (Jian He via wangda)

Posted by ji...@apache.org.
YARN-3564. Fix TestContainerAllocation.testAMContainerAllocationWhenDNSUnavailable fails randomly. (Jian He via wangda)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e2e8f771
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e2e8f771
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e2e8f771

Branch: refs/heads/HDFS-7240
Commit: e2e8f771183df798e926abc97116316a05b19c9a
Parents: de9404f
Author: Wangda Tan <wa...@apache.org>
Authored: Thu Apr 30 11:03:13 2015 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Thu Apr 30 11:03:19 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                   | 3 +++
 .../scheduler/capacity/TestContainerAllocation.java               | 1 -
 2 files changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2e8f771/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index f583e6a..cf3d910 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -274,6 +274,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3533. Test: Fix launchAM in MockRM to wait for attempt to be scheduled.
     (Anubhav Dhoot via jianhe)
 
+    YARN-3564. Fix TestContainerAllocation.testAMContainerAllocationWhenDNSUnavailable 
+    fails randomly. (Jian He via wangda)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2e8f771/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
index 6aceda5..0ea993f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
@@ -325,7 +325,6 @@ public class TestContainerAllocation {
     }
 
     SecurityUtilTestHelper.setTokenServiceUseIp(false);
-    rm1.waitForState(attempt.getAppAttemptId(), RMAppAttemptState.ALLOCATED);
     MockRM.launchAndRegisterAM(app1, rm1, nm1);
   }
 }


[26/50] [abbrv] hadoop git commit: Moving MAPREDUCE-6252 to the 2.7.1 CHANGES.txt

Posted by ji...@apache.org.
Moving MAPREDUCE-6252 to the 2.7.1 CHANGES.txt


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/99fe03e4
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/99fe03e4
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/99fe03e4

Branch: refs/heads/HDFS-7240
Commit: 99fe03e439b0f9afd01754d998c6eb64f0f70300
Parents: feb68cb
Author: Devaraj K <de...@apache.org>
Authored: Tue Apr 28 12:42:16 2015 +0530
Committer: Devaraj K <de...@apache.org>
Committed: Tue Apr 28 12:42:16 2015 +0530

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/99fe03e4/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index eb6feb9..d27a022 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -350,9 +350,6 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6333. TestEvents,TestAMWebServicesTasks,TestAppController are
     broken due to MAPREDUCE-6297. (Siqi Li via gera)
 
-    MAPREDUCE-6252. JobHistoryServer should not fail when encountering a 
-    missing directory. (Craig Welch via devaraj)
-
     MAPREDUCE-6341. Fix typo in mapreduce tutorial. (John Michael Luy
     via ozawa)
 
@@ -376,6 +373,9 @@ Release 2.7.1 - UNRELEASED
     MAPREDUCE-6324. Fixed MapReduce uber jobs to not fail the udpate of AM-RM
     tokens when they roll-over. (Jason Lowe via vinodkv)
 
+    MAPREDUCE-6252. JobHistoryServer should not fail when encountering a 
+    missing directory. (Craig Welch via devaraj)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES


[49/50] [abbrv] hadoop git commit: HDFS-8300. Fix unit test failures and findbugs warning caused by HDFS-8283. Contributed by Jing Zhao.

Posted by ji...@apache.org.
HDFS-8300. Fix unit test failures and findbugs warning caused by HDFS-8283. Contributed by Jing Zhao.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/98a61766
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/98a61766
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/98a61766

Branch: refs/heads/HDFS-7240
Commit: 98a61766286321468bf801a9f17a843d7eae8d9e
Parents: 87e9978
Author: Jing Zhao <ji...@apache.org>
Authored: Thu Apr 30 19:27:07 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Thu Apr 30 19:27:07 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../org/apache/hadoop/hdfs/DFSOutputStream.java |  2 +-
 .../org/apache/hadoop/hdfs/DataStreamer.java    | 32 ++++++++++++--------
 .../apache/hadoop/hdfs/TestDFSOutputStream.java |  9 ++++--
 4 files changed, 29 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/98a61766/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 7e4a6f2..3bee852 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -592,6 +592,9 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-8214. Secondary NN Web UI shows wrong date for Last Checkpoint. (clamb via wang)
 
+    HDFS-8300. Fix unit test failures and findbugs warning caused by HDFS-8283.
+    (jing9)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98a61766/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
index 4646b60..ae5d3eb 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
@@ -762,7 +762,7 @@ public class DFSOutputStream extends FSOutputSummer
 
   protected synchronized void closeImpl() throws IOException {
     if (isClosed()) {
-      streamer.getLastException().check();
+      streamer.getLastException().check(true);
       return;
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98a61766/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
index 3727d20..8e874eb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
@@ -174,13 +174,13 @@ class DataStreamer extends Daemon {
     packets.clear();
   }
   
-  static class LastException {
-    private Throwable thrown;
+  static class LastExceptionInStreamer {
+    private IOException thrown;
 
     synchronized void set(Throwable t) {
-      Preconditions.checkNotNull(t);
-      Preconditions.checkState(thrown == null);
-      this.thrown = t;
+      assert t != null;
+      this.thrown = t instanceof IOException ?
+          (IOException) t : new IOException(t);
     }
 
     synchronized void clear() {
@@ -188,17 +188,23 @@ class DataStreamer extends Daemon {
     }
 
     /** Check if there already is an exception. */
-    synchronized void check() throws IOException {
+    synchronized void check(boolean resetToNull) throws IOException {
       if (thrown != null) {
-        throw new IOException(thrown);
+        if (LOG.isTraceEnabled()) {
+          // wrap and print the exception to know when the check is called
+          LOG.trace("Got Exception while checking", new Throwable(thrown));
+        }
+        final IOException e = thrown;
+        if (resetToNull) {
+          thrown = null;
+        }
+        throw e;
       }
     }
 
     synchronized void throwException4Close() throws IOException {
-      check();
-      final IOException ioe = new ClosedChannelException();
-      thrown = ioe;
-      throw ioe;
+      check(false);
+      throw new ClosedChannelException();
     }
   }
 
@@ -234,7 +240,7 @@ class DataStreamer extends Daemon {
   private long lastQueuedSeqno = -1;
   private long lastAckedSeqno = -1;
   private long bytesCurBlock = 0; // bytes written in current block
-  private final LastException lastException = new LastException();
+  private final LastExceptionInStreamer lastException = new LastExceptionInStreamer();
   private Socket s;
 
   private final DFSClient dfsClient;
@@ -1741,7 +1747,7 @@ class DataStreamer extends Daemon {
   /**
    * @return the last exception
    */
-  LastException getLastException(){
+  LastExceptionInStreamer getLastException(){
     return lastException;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98a61766/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
index eac1fcd..e627455 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
@@ -28,6 +28,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DataStreamer.LastExceptionInStreamer;
 import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -65,9 +66,10 @@ public class TestDFSOutputStream {
     DataStreamer streamer = (DataStreamer) Whitebox
         .getInternalState(dos, "streamer");
     @SuppressWarnings("unchecked")
-    AtomicReference<IOException> ex = (AtomicReference<IOException>) Whitebox
+    LastExceptionInStreamer ex = (LastExceptionInStreamer) Whitebox
         .getInternalState(streamer, "lastException");
-    Assert.assertEquals(null, ex.get());
+    Throwable thrown = (Throwable) Whitebox.getInternalState(ex, "thrown");
+    Assert.assertNull(thrown);
 
     dos.close();
 
@@ -78,7 +80,8 @@ public class TestDFSOutputStream {
     } catch (IOException e) {
       Assert.assertEquals(e, dummy);
     }
-    Assert.assertEquals(null, ex.get());
+    thrown = (Throwable) Whitebox.getInternalState(ex, "thrown");
+    Assert.assertNull(thrown);
     dos.close();
   }
 


[47/50] [abbrv] hadoop git commit: HADOOP-11866. increase readability and reliability of checkstyle, shellcheck, and whitespace reports (aw)

Posted by ji...@apache.org.
HADOOP-11866. increase readability and reliability of checkstyle, shellcheck, and whitespace reports (aw)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/5f8112ff
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5f8112ff
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5f8112ff

Branch: refs/heads/HDFS-7240
Commit: 5f8112ffd220598d997c92f681d3f69022898110
Parents: f0db797
Author: Allen Wittenauer <aw...@apache.org>
Authored: Thu Apr 30 15:15:32 2015 -0700
Committer: Allen Wittenauer <aw...@apache.org>
Committed: Thu Apr 30 15:15:32 2015 -0700

----------------------------------------------------------------------
 dev-support/test-patch.d/checkstyle.sh          | 216 ++++++++++++-------
 dev-support/test-patch.d/shellcheck.sh          |  52 ++++-
 dev-support/test-patch.d/whitespace.sh          |  12 +-
 dev-support/test-patch.sh                       |  73 ++++++-
 hadoop-common-project/hadoop-common/CHANGES.txt |   3 +
 5 files changed, 255 insertions(+), 101 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f8112ff/dev-support/test-patch.d/checkstyle.sh
----------------------------------------------------------------------
diff --git a/dev-support/test-patch.d/checkstyle.sh b/dev-support/test-patch.d/checkstyle.sh
index 460709e..6311584 100755
--- a/dev-support/test-patch.d/checkstyle.sh
+++ b/dev-support/test-patch.d/checkstyle.sh
@@ -29,8 +29,39 @@ function checkstyle_filefilter
   fi
 }
 
+function checkstyle_mvnrunner
+{
+  local logfile=$1
+  local output=$2
+  local tmp=${PATCH_DIR}/$$.${RANDOM}
+  local j
+
+  "${MVN}" clean test checkstyle:checkstyle -DskipTests \
+    -Dcheckstyle.consoleOutput=true \
+    "-D${PROJECT_NAME}PatchProcess" 2>&1 \
+      | tee "${logfile}" \
+      | ${GREP} ^/ \
+      | ${SED} -e "s,${BASEDIR},.,g" \
+          > "${tmp}"
+
+  # the checkstyle output files are massive, so
+  # let's reduce the work by filtering out files
+  # that weren't changed.  Some modules are
+  # MASSIVE and this can cut the output down to
+  # by orders of magnitude!!
+  for j in ${CHANGED_FILES}; do
+    ${GREP} "${j}" "${tmp}" >> "${output}"
+  done
+
+  rm "${tmp}" 2>/dev/null
+}
+
 function checkstyle_preapply
 {
+  local module_suffix
+  local modules=${CHANGED_MODULES}
+  local module
+
   verify_needed_test checkstyle
 
   if [[ $? == 0 ]]; then
@@ -40,23 +71,78 @@ function checkstyle_preapply
   big_console_header "checkstyle plugin: prepatch"
 
   start_clock
-  echo_and_redirect "${PATCH_DIR}/${PATCH_BRANCH}checkstyle.txt" "${MVN}" test checkstyle:checkstyle-aggregate -DskipTests "-D${PROJECT_NAME}PatchProcess"
-  if [[ $? != 0 ]] ; then
-    echo "Pre-patch ${PATCH_BRANCH} checkstyle compilation is broken?"
-    add_jira_table -1 checkstyle "Pre-patch ${PATCH_BRANCH} checkstyle compilation may be broken."
-    return 1
-  fi
 
-  cp -p "${BASEDIR}/target/checkstyle-result.xml" \
-    "${PATCH_DIR}/checkstyle-result-${PATCH_BRANCH}.xml"
+  for module in ${modules}
+  do
+    pushd "${module}" >/dev/null
+    echo "  Running checkstyle in ${module}"
+    module_suffix=$(basename "${module}")
+
+    checkstyle_mvnrunner \
+      "${PATCH_DIR}/maven-${PATCH_BRANCH}checkstyle-${module_suffix}.txt" \
+      "${PATCH_DIR}/${PATCH_BRANCH}checkstyle${module_suffix}.txt"
+
+    if [[ $? != 0 ]] ; then
+      echo "Pre-patch ${PATCH_BRANCH} checkstyle compilation is broken?"
+      add_jira_table -1 checkstyle "Pre-patch ${PATCH_BRANCH} ${module} checkstyle compilation may be broken."
+    fi
+    popd >/dev/null
+  done
 
   # keep track of how much as elapsed for us already
   CHECKSTYLE_TIMER=$(stop_clock)
   return 0
 }
 
+function checkstyle_calcdiffs
+{
+  local orig=$1
+  local new=$2
+  local diffout=$3
+  local tmp=${PATCH_DIR}/cs.$$.${RANDOM}
+  local count=0
+  local j
+
+  # first, pull out just the errors
+  # shellcheck disable=SC2016
+  ${AWK} -F: '{print $NF}' "${orig}" >> "${tmp}.branch"
+
+  # shellcheck disable=SC2016
+  ${AWK} -F: '{print $NF}' "${new}" >> "${tmp}.patch"
+
+  # compare the errors, generating a string of line
+  # numbers.  Sorry portability: GNU diff makes this too easy
+  ${DIFF} --unchanged-line-format="" \
+     --old-line-format="" \
+     --new-line-format="%dn " \
+     "${tmp}.branch" \
+     "${tmp}.patch" > "${tmp}.lined"
+
+  # now, pull out those lines of the raw output
+  # shellcheck disable=SC2013
+  for j in $(cat "${tmp}.lined"); do
+    # shellcheck disable=SC2086
+    head -${j} "${new}" | tail -1 >> "${diffout}"
+  done
+
+  if [[ -f "${diffout}" ]]; then
+    # shellcheck disable=SC2016
+    count=$(wc -l "${diffout}" | ${AWK} '{print $1}' )
+  fi
+  rm "${tmp}.branch" "${tmp}.patch" "${tmp}.lined" 2>/dev/null
+  echo "${count}"
+}
+
 function checkstyle_postapply
 {
+  local rc=0
+  local module
+  local modules=${CHANGED_MODULES}
+  local module_suffix
+  local numprepatch=0
+  local numpostpatch=0
+  local diffpostpatch=0
+
   verify_needed_test checkstyle
 
   if [[ $? == 0 ]]; then
@@ -71,79 +157,49 @@ function checkstyle_postapply
   # by setting the clock back
   offset_clock "${CHECKSTYLE_TIMER}"
 
-  echo_and_redirect "${PATCH_DIR}/patchcheckstyle.txt" "${MVN}" test checkstyle:checkstyle-aggregate -DskipTests "-D${PROJECT_NAME}PatchProcess"
-  if [[ $? != 0 ]] ; then
-    echo "Post-patch checkstyle compilation is broken."
-    add_jira_table -1 checkstyle "Post-patch checkstyle compilation is broken."
-    return 1
-  fi
-
-  cp -p "${BASEDIR}/target/checkstyle-result.xml" \
-    "${PATCH_DIR}/checkstyle-result-patch.xml"
-
-  checkstyle_runcomparison
-
-  # shellcheck disable=SC2016
-  CHECKSTYLE_POSTPATCH=$(wc -l "${PATCH_DIR}/checkstyle-result-diff.txt" | ${AWK} '{print $1}')
-
-  if [[ ${CHECKSTYLE_POSTPATCH} -gt 0 ]] ; then
-
-    add_jira_table -1 checkstyle "The applied patch generated "\
-      "${CHECKSTYLE_POSTPATCH}" \
-      " additional checkstyle issues."
-    add_jira_footer checkstyle "@@BASE@@/checkstyle-result-diff.txt"
-
+  for module in ${modules}
+  do
+    pushd "${module}" >/dev/null
+    echo "  Running checkstyle in ${module}"
+    module_suffix=$(basename "${module}")
+
+    checkstyle_mvnrunner \
+      "${PATCH_DIR}/maven-patchcheckstyle-${module_suffix}.txt" \
+      "${PATCH_DIR}/patchcheckstyle${module_suffix}.txt"
+
+    if [[ $? != 0 ]] ; then
+      ((rc = rc +1))
+      echo "Post-patch checkstyle compilation is broken."
+      add_jira_table -1 checkstyle "Post-patch checkstyle ${module} compilation is broken."
+      continue
+    fi
+
+    #shellcheck disable=SC2016
+    diffpostpatch=$(checkstyle_calcdiffs \
+      "${PATCH_DIR}/${PATCH_BRANCH}checkstyle${module_suffix}.txt" \
+      "${PATCH_DIR}/patchcheckstyle${module_suffix}.txt" \
+      "${PATCH_DIR}/diffcheckstyle${module_suffix}.txt" )
+
+    if [[ ${diffpostpatch} -gt 0 ]] ; then
+      ((rc = rc + 1))
+
+      # shellcheck disable=SC2016
+      numprepatch=$(wc -l "${PATCH_DIR}/${PATCH_BRANCH}checkstyle${module_suffix}.txt" | ${AWK} '{print $1}')
+      # shellcheck disable=SC2016
+      numpostpatch=$(wc -l "${PATCH_DIR}/patchcheckstyle${module_suffix}.txt" | ${AWK} '{print $1}')
+
+      add_jira_table -1 checkstyle "The applied patch generated "\
+        "${diffpostpatch} new checkstyle issues (total was ${numprepatch}, now ${numpostpatch})."
+      footer="${footer} @@BASE@@/diffcheckstyle${module_suffix}.txt"
+    fi
+
+    popd >/dev/null
+  done
+
+  if [[ ${rc} -gt 0 ]] ; then
+    add_jira_footer checkstyle "${footer}"
     return 1
   fi
   add_jira_table +1 checkstyle "There were no new checkstyle issues."
   return 0
-}
-
-
-function checkstyle_runcomparison
-{
-
-  python <(cat <<EOF
-import os
-import sys
-import xml.etree.ElementTree as etree
-from collections import defaultdict
-
-if len(sys.argv) != 3 :
-  print "usage: %s checkstyle-result-master.xml checkstyle-result-patch.xml" % sys.argv[0]
-  exit(1)
-
-def path_key(x):
-  path = x.attrib['name']
-  return path[path.find('${PROJECT_NAME}-'):]
-
-def print_row(path, master_errors, patch_errors):
-    print '%s\t%s\t%s' % (k,master_dict[k],child_errors)
-
-master = etree.parse(sys.argv[1])
-patch = etree.parse(sys.argv[2])
-
-master_dict = defaultdict(int)
-
-for child in master.getroot().getchildren():
-    if child.tag != 'file':
-        continue
-    child_errors = len(child.getchildren())
-    if child_errors == 0:
-        continue
-    master_dict[path_key(child)] = child_errors
-
-for child in patch.getroot().getchildren():
-    if child.tag != 'file':
-        continue
-    child_errors = len(child.getchildren())
-    if child_errors == 0:
-        continue
-    k = path_key(child)
-    if child_errors > master_dict[k]:
-        print_row(k, master_dict[k], child_errors)
-
-EOF
-) "${PATCH_DIR}/checkstyle-result-${PATCH_BRANCH}.xml" "${PATCH_DIR}/checkstyle-result-patch.xml" > "${PATCH_DIR}/checkstyle-result-diff.txt"
-
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f8112ff/dev-support/test-patch.d/shellcheck.sh
----------------------------------------------------------------------
diff --git a/dev-support/test-patch.d/shellcheck.sh b/dev-support/test-patch.d/shellcheck.sh
index c1084a2..5f38b6a 100755
--- a/dev-support/test-patch.d/shellcheck.sh
+++ b/dev-support/test-patch.d/shellcheck.sh
@@ -87,6 +87,45 @@ function shellcheck_preapply
   return 0
 }
 
+function shellcheck_calcdiffs
+{
+  local orig=$1
+  local new=$2
+  local diffout=$3
+  local tmp=${PATCH_DIR}/sc.$$.${RANDOM}
+  local count=0
+  local j
+
+  # first, pull out just the errors
+  # shellcheck disable=SC2016
+  ${AWK} -F: '{print $NF}' "${orig}" >> "${tmp}.branch"
+
+  # shellcheck disable=SC2016
+  ${AWK} -F: '{print $NF}' "${new}" >> "${tmp}.patch"
+
+  # compare the errors, generating a string of line
+  # numbers.  Sorry portability: GNU diff makes this too easy
+  ${DIFF} --unchanged-line-format="" \
+     --old-line-format="" \
+     --new-line-format="%dn " \
+     "${tmp}.branch" \
+     "${tmp}.patch" > "${tmp}.lined"
+
+  # now, pull out those lines of the raw output
+  # shellcheck disable=SC2013
+  for j in $(cat "${tmp}.lined"); do
+    # shellcheck disable=SC2086
+    head -${j} "${new}" | tail -1 >> "${diffout}"
+  done
+
+  if [[ -f "${diffout}" ]]; then
+    # shellcheck disable=SC2016
+    count=$(wc -l "${diffout}" | ${AWK} '{print $1}' )
+  fi
+  rm "${tmp}.branch" "${tmp}.patch" "${tmp}.lined" 2>/dev/null
+  echo "${count}"
+}
+
 function shellcheck_postapply
 {
   local i
@@ -121,16 +160,13 @@ function shellcheck_postapply
   # shellcheck disable=SC2016
   numPostpatch=$(wc -l "${PATCH_DIR}/patchshellcheck-result.txt" | ${AWK} '{print $1}')
 
-  ${DIFF} -u "${PATCH_DIR}/${PATCH_BRANCH}shellcheck-result.txt" \
+  diffPostpatch=$(shellcheck_calcdiffs \
+    "${PATCH_DIR}/${PATCH_BRANCH}shellcheck-result.txt" \
     "${PATCH_DIR}/patchshellcheck-result.txt" \
-      | ${GREP} '^+\.' \
-      > "${PATCH_DIR}/diffpatchshellcheck.txt"
-
-  # shellcheck disable=SC2016
-  diffPostpatch=$(wc -l "${PATCH_DIR}/diffpatchshellcheck.txt" | ${AWK} '{print $1}')
+      "${PATCH_DIR}/diffpatchshellcheck.txt"
+    )
 
-  if [[ ${diffPostpatch} -gt 0
-    && ${numPostpatch} -gt ${numPrepatch} ]] ; then
+  if [[ ${diffPostpatch} -gt 0 ]] ; then
     add_jira_table -1 shellcheck "The applied patch generated "\
       "${diffPostpatch} new shellcheck (v${SHELLCHECK_VERSION}) issues (total was ${numPrepatch}, now ${numPostpatch})."
     add_jira_footer shellcheck "@@BASE@@/diffpatchshellcheck.txt"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f8112ff/dev-support/test-patch.d/whitespace.sh
----------------------------------------------------------------------
diff --git a/dev-support/test-patch.d/whitespace.sh b/dev-support/test-patch.d/whitespace.sh
index deac654..324481c 100755
--- a/dev-support/test-patch.d/whitespace.sh
+++ b/dev-support/test-patch.d/whitespace.sh
@@ -16,25 +16,31 @@
 
 add_plugin whitespace
 
-function whitespace_preapply
+function whitespace_postapply
 {
   local count
+  local j
 
   big_console_header "Checking for whitespace at the end of lines"
   start_clock
 
-  ${GREP} '^+' "${PATCH_DIR}/patch" | ${GREP} '[[:blank:]]$' > "${PATCH_DIR}/whitespace.txt"
+  pushd "${BASEDIR}" >/dev/null
+  for j in ${CHANGED_FILES}; do
+    ${GREP} -nHE '[[:blank:]]$' "./${j}" | ${GREP} -f "${GITDIFFLINES}" >> "${PATCH_DIR}/whitespace.txt"
+  done
 
   # shellcheck disable=SC2016
   count=$(wc -l "${PATCH_DIR}/whitespace.txt" | ${AWK} '{print $1}')
 
   if [[ ${count} -gt 0 ]]; then
     add_jira_table -1 whitespace "The patch has ${count}"\
-      " line(s) that end in whitespace."
+      " line(s) that end in whitespace. Use git apply --whitespace=fix."
     add_jira_footer whitespace "@@BASE@@/whitespace.txt"
+    popd >/dev/null
     return 1
   fi
 
+  popd >/dev/null
   add_jira_table +1 whitespace "The patch has no lines that end in whitespace."
   return 0
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f8112ff/dev-support/test-patch.sh
----------------------------------------------------------------------
diff --git a/dev-support/test-patch.sh b/dev-support/test-patch.sh
index ae21837..b6e1b03 100755
--- a/dev-support/test-patch.sh
+++ b/dev-support/test-patch.sh
@@ -67,7 +67,7 @@ function setup_defaults
       EGREP=${EGREP:-/usr/xpg4/bin/egrep}
       GREP=${GREP:-/usr/xpg4/bin/grep}
       PATCH=${PATCH:-patch}
-      DIFF=${DIFF:-diff}
+      DIFF=${DIFF:-/usr/gnu/bin/diff}
       JIRACLI=${JIRA:-jira}
     ;;
     *)
@@ -449,7 +449,7 @@ function verify_patchdir_still_exists
   if [[ ! -d ${PATCH_DIR} ]]; then
       rm "${commentfile}" 2>/dev/null
 
-      echo "(!) The patch artifact directory on has been removed! " > "${commentfile}"
+      echo "(!) The patch artifact directory has been removed! " > "${commentfile}"
       echo "This is a fatal error for test-patch.sh.  Aborting. " >> "${commentfile}"
       echo
       cat ${commentfile}
@@ -468,6 +468,49 @@ function verify_patchdir_still_exists
     fi
 }
 
+## @description generate a list of all files and line numbers that
+## @description that were added/changed in the source repo
+## @audience    private
+## @stability   stable
+## @params      filename
+## @replaceable no
+function compute_gitdiff
+{
+  local outfile=$1
+  local file
+  local line
+  local startline
+  local counter
+  local numlines
+  local actual
+
+  pushd "${BASEDIR}" >/dev/null
+  while read line; do
+    if [[ ${line} =~ ^\+\+\+ ]]; then
+      file="./"$(echo "${line}" | cut -f2- -d/)
+      continue
+    elif [[ ${line} =~ ^@@ ]]; then
+      startline=$(echo "${line}" | cut -f3 -d' ' | cut -f1 -d, | tr -d + )
+      numlines=$(echo "${line}" | cut -f3 -d' ' | cut -s -f2 -d, )
+      # if this is empty, then just this line
+      # if it is 0, then no lines were added and this part of the patch
+      # is strictly a delete
+      if [[ ${numlines} == 0 ]]; then
+        continue
+      elif [[ -z ${numlines} ]]; then
+        numlines=1
+      fi
+      counter=0
+      until [[ ${counter} -gt ${numlines} ]]; do
+          ((actual=counter+startline))
+          echo "${file}:${actual}:" >> "${outfile}"
+          ((counter=counter+1))
+      done
+    fi
+  done < <("${GIT}" diff --unified=0 --no-color)
+  popd >/dev/null
+}
+
 ## @description  Print the command to be executing to the screen. Then
 ## @description  run the command, sending stdout and stderr to the given filename
 ## @description  This will also ensure that any directories in ${BASEDIR} have
@@ -481,7 +524,7 @@ function verify_patchdir_still_exists
 ## @returns      $?
 function echo_and_redirect
 {
-  logfile=$1
+  local logfile=$1
   shift
 
   verify_patchdir_still_exists
@@ -522,7 +565,7 @@ function hadoop_usage
 
   echo "Shell binary overrides:"
   echo "--awk-cmd=<cmd>        The 'awk' command to use (default 'awk')"
-  echo "--diff-cmd=<cmd>       The 'diff' command to use (default 'diff')"
+  echo "--diff-cmd=<cmd>       The GNU-compatible 'diff' command to use (default 'diff')"
   echo "--git-cmd=<cmd>        The 'git' command to use (default 'git')"
   echo "--grep-cmd=<cmd>       The 'grep' command to use (default 'grep')"
   echo "--mvn-cmd=<cmd>        The 'mvn' command to use (default \${MAVEN_HOME}/bin/mvn, or 'mvn')"
@@ -585,6 +628,10 @@ function parse_args
       --grep-cmd=*)
         GREP=${i#*=}
       ;;
+      --help|-help|-h|help|--h|--\?|-\?|\?)
+        hadoop_usage
+        exit 0
+      ;;
       --java-home)
         JAVA_HOME=${i#*=}
       ;;
@@ -680,6 +727,8 @@ function parse_args
       cleanup_and_exit 1
     fi
   fi
+
+  GITDIFFLINES=${PATCH_DIR}/gitdifflines.txt
 }
 
 ## @description  Locate the pom.xml file for a given directory
@@ -716,12 +765,14 @@ function find_changed_files
   # get a list of all of the files that have been changed,
   # except for /dev/null (which would be present for new files).
   # Additionally, remove any a/ b/ patterns at the front
-  # of the patch filenames
+  # of the patch filenames and any revision info at the end
+  # shellcheck disable=SC2016
   CHANGED_FILES=$(${GREP} -E '^(\+\+\+|---) ' "${PATCH_DIR}/patch" \
     | ${SED} \
       -e 's,^....,,' \
       -e 's,^[ab]/,,' \
     | ${GREP} -v /dev/null \
+    | ${AWK} '{print $1}' \
     | sort -u)
 }
 
@@ -1552,7 +1603,7 @@ function check_javac
         > "${PATCH_DIR}/diffJavacWarnings.txt"
 
         add_jira_table -1 javac "The applied patch generated "\
-        "$((patchJavacWarnings-branchJavacWarnings))" \
+        "$((patchJavacWarnings-${PATCH_BRANCH}JavacWarnings))" \
         " additional warning messages."
 
         add_jira_footer javac "@@BASE@@/diffJavacWarnings.txt"
@@ -1712,6 +1763,7 @@ function check_findbugs
       "${PATCH_DIR}/patchFindbugsWarnings${module_suffix}.xml" \
       "${PATCH_DIR}/patchFindbugsWarnings${module_suffix}.xml"
 
+    #shellcheck disable=SC2016
     newFindbugsWarnings=$("${FINDBUGS_HOME}/bin/filterBugs" \
       -first "01/01/2000" "${PATCH_DIR}/patchFindbugsWarnings${module_suffix}.xml" \
       "${PATCH_DIR}/newPatchFindbugsWarnings${module_suffix}.xml" \
@@ -1887,10 +1939,12 @@ function check_unittests
       test_timeouts="${test_timeouts} ${module_test_timeouts}"
       result=1
     fi
-    #shellcheck disable=SC2026,SC2038
+
+    #shellcheck disable=SC2026,SC2038,SC2016
     module_failed_tests=$(find . -name 'TEST*.xml'\
       | xargs "${GREP}" -l -E "<failure|<error"\
       | ${AWK} -F/ '{sub("TEST-org.apache.",""); sub(".xml",""); print $NF}')
+
     if [[ -n "${module_failed_tests}" ]] ; then
       failed_tests="${failed_tests} ${module_failed_tests}"
       result=1
@@ -2054,8 +2108,6 @@ function output_to_console
     printf "%s\n" "${comment}"
     ((i=i+1))
   done
-
-
 }
 
 ## @description  Print out the finished details to the JIRA issue
@@ -2189,7 +2241,6 @@ function postcheckout
       #shellcheck disable=SC2086
       ${plugin}_postcheckout
 
-
       (( RESULT = RESULT + $? ))
       if [[ ${RESULT} != 0 ]] ; then
         output_to_console 1
@@ -2244,6 +2295,8 @@ function postapply
   local plugin
   local retval
 
+  compute_gitdiff "${GITDIFFLINES}"
+
   check_javac
   retval=$?
   if [[ ${retval} -gt 1 ]] ; then

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f8112ff/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index d2e1d4a..0a53396 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -582,6 +582,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11821. Fix findbugs warnings in hadoop-sls.
     (Brahma Reddy Battula via aajisaka)
 
+    HADOOP-11866. increase readability and reliability of checkstyle,
+    shellcheck, and whitespace reports (aw)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES


[08/50] [abbrv] hadoop git commit: HDFS-8247. TestDiskspaceQuotaUpdate#testAppendOverTypeQuota is failing. Contributed by Xiaoyu Yao.

Posted by ji...@apache.org.
HDFS-8247. TestDiskspaceQuotaUpdate#testAppendOverTypeQuota is failing. Contributed by Xiaoyu Yao.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/a00e001a
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a00e001a
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a00e001a

Branch: refs/heads/HDFS-7240
Commit: a00e001a1a9fa2c6287b2f078e425e9bb157e5ca
Parents: f83c55a
Author: cnauroth <cn...@apache.org>
Authored: Sat Apr 25 11:37:27 2015 -0700
Committer: cnauroth <cn...@apache.org>
Committed: Sat Apr 25 11:37:27 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                     | 3 +++
 .../hadoop/hdfs/server/namenode/TestDiskspaceQuotaUpdate.java   | 5 +++--
 2 files changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a00e001a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 07c5151..c1aac54 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -566,6 +566,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8211. DataNode UUID is always null in the JMX counter. (Anu Engineer
     via Arpit Agarwal)
 
+    HDFS-8247. TestDiskspaceQuotaUpdate#testAppendOverTypeQuota is failing.
+    (Xiaoyu Yao via cnauroth)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a00e001a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDiskspaceQuotaUpdate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDiskspaceQuotaUpdate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDiskspaceQuotaUpdate.java
index 15ba15e..184cd87 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDiskspaceQuotaUpdate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDiskspaceQuotaUpdate.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.QuotaByStorageTypeExceededException;
 import org.apache.hadoop.ipc.RemoteException;
 import org.junit.After;
 import org.junit.Assert;
@@ -250,8 +251,8 @@ public class TestDiskspaceQuotaUpdate {
     try {
       DFSTestUtil.appendFile(dfs, file, BLOCKSIZE);
       Assert.fail("append didn't fail");
-    } catch (RemoteException e) {
-      assertTrue(e.getClassName().contains("QuotaByStorageTypeExceededException"));
+    } catch (QuotaByStorageTypeExceededException e) {
+      //ignore
     }
 
     // check that the file exists, isn't UC, and has no dangling lease


[17/50] [abbrv] hadoop git commit: MAPREDUCE-6341. Fix typo in mapreduce tutorial. Contributed by John Michael Luy.

Posted by ji...@apache.org.
MAPREDUCE-6341. Fix typo in mapreduce tutorial. Contributed by John Michael Luy.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/9a3dda3d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9a3dda3d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9a3dda3d

Branch: refs/heads/HDFS-7240
Commit: 9a3dda3d345b94cf6f9062c00395500596829cf1
Parents: 5e67c4d
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Tue Apr 28 01:47:37 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Tue Apr 28 01:47:37 2015 +0900

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                    |  3 +++
 .../src/site/markdown/MapReduceTutorial.md              | 12 +++++++-----
 2 files changed, 10 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a3dda3d/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index f9488fb..f1d0493 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -352,6 +352,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6252. JobHistoryServer should not fail when encountering a 
     missing directory. (Craig Welch via devaraj)
 
+    MAPREDUCE-6341. Fix typo in mapreduce tutorial. (John Michael Luy
+    via ozawa)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a3dda3d/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/MapReduceTutorial.md
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/MapReduceTutorial.md b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/MapReduceTutorial.md
index ccc9590..cd087d5 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/MapReduceTutorial.md
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/MapReduceTutorial.md
@@ -210,7 +210,9 @@ Assuming that:
 
 Sample text-files as input:
 
-    $ bin/hadoop fs -ls /user/joe/wordcount/input/ /user/joe/wordcount/input/file01 /user/joe/wordcount/input/file02
+    $ bin/hadoop fs -ls /user/joe/wordcount/input/
+    /user/joe/wordcount/input/file01
+    /user/joe/wordcount/input/file02
     
     $ bin/hadoop fs -cat /user/joe/wordcount/input/file01
     Hello World Bye World
@@ -224,12 +226,12 @@ Run the application:
 
 Output:
 
-    $ bin/hadoop fs -cat /user/joe/wordcount/output/part-r-00000`
+    $ bin/hadoop fs -cat /user/joe/wordcount/output/part-r-00000
     Bye 1
     Goodbye 1
     Hadoop 2
     Hello 2
-    World 2`
+    World 2
 
 Applications can specify a comma separated list of paths which would be present in the current working directory of the task using the option `-files`. The `-libjars` option allows applications to add jars to the classpaths of the maps and reduces. The option `-archives` allows them to pass comma separated list of archives as arguments. These archives are unarchived and a link with name of the archive is created in the current working directory of tasks. More details about the command line options are available at [Commands Guide](../../hadoop-project-dist/hadoop-common/CommandsManual.html).
 
@@ -288,13 +290,13 @@ The output of the first map:
 
     < Bye, 1>
     < Hello, 1>
-    < World, 2>`
+    < World, 2>
 
 The output of the second map:
 
     < Goodbye, 1>
     < Hadoop, 2>
-    < Hello, 1>`
+    < Hello, 1>
 
 ```java
 public void reduce(Text key, Iterable<IntWritable> values,


[19/50] [abbrv] hadoop git commit: HDFS-8205. CommandFormat#parse() should not parse option as value of option. (Contributed by Peter Shi and Xiaoyu Yao)

Posted by ji...@apache.org.
HDFS-8205. CommandFormat#parse() should not parse option as value of option. (Contributed by Peter Shi and Xiaoyu Yao)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/0d5b0143
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/0d5b0143
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/0d5b0143

Branch: refs/heads/HDFS-7240
Commit: 0d5b0143cc003e132ce454415e35d55d46311416
Parents: 7f07c4d
Author: Arpit Agarwal <ar...@apache.org>
Authored: Mon Apr 27 12:23:34 2015 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Mon Apr 27 12:23:34 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/fs/shell/CommandFormat.java   |  3 +-
 .../org/apache/hadoop/fs/shell/TestCount.java   | 28 +++++++++++++++
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../org/apache/hadoop/hdfs/tools/DFSAdmin.java  |  4 +--
 .../src/test/resources/testHDFSConf.xml         | 38 ++++++++++++++++++++
 5 files changed, 73 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d5b0143/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandFormat.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandFormat.java
index 371168d..0aa3d65 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandFormat.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandFormat.java
@@ -114,7 +114,8 @@ public class CommandFormat {
         options.put(opt, Boolean.TRUE);
       } else if (optionsWithValue.containsKey(opt)) {
         args.remove(pos);
-        if (pos < args.size() && (args.size() > minPar)) {
+        if (pos < args.size() && (args.size() > minPar)
+                && !args.get(pos).startsWith("-")) {
           arg = args.get(pos);
           args.remove(pos);
         } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d5b0143/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java
index 22d9a21..44fc1e6 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java
@@ -316,6 +316,34 @@ public class TestCount {
   }
 
   @Test
+  public void processPathWithQuotasByQTVH() throws Exception {
+    Path path = new Path("mockfs:/test");
+
+    when(mockFs.getFileStatus(eq(path))).thenReturn(fileStat);
+
+    PrintStream out = mock(PrintStream.class);
+
+    Count count = new Count();
+    count.out = out;
+
+    LinkedList<String> options = new LinkedList<String>();
+    options.add("-q");
+    options.add("-t");
+    options.add("-v");
+    options.add("-h");
+    options.add("dummy");
+    count.processOptions(options);
+    String withStorageTypeHeader =
+        // <----13---> <-------17------>
+        "   DISK_QUOTA    REM_DISK_QUOTA " +
+        "    SSD_QUOTA     REM_SSD_QUOTA " +
+        "ARCHIVE_QUOTA REM_ARCHIVE_QUOTA " +
+        "PATHNAME";
+    verify(out).println(withStorageTypeHeader);
+    verifyNoMoreInteractions(out);
+  }
+
+  @Test
   public void processPathWithQuotasByMultipleStorageTypesContent() throws Exception {
     Path path = new Path("mockfs:/test");
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d5b0143/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index b7199c7..00b5db5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -618,6 +618,9 @@ Release 2.7.1 - UNRELEASED
     HDFS-8070. Pre-HDFS-7915 DFSClient cannot use short circuit on
     post-HDFS-7915 DataNode (cmccabe)
 
+    HDFS-8205. CommandFormat#parse() should not parse option as
+    value of option. (Peter Shi and Xiaoyu Yao via Arpit Agarwal)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d5b0143/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
index d608281..11f2c32 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
@@ -221,9 +221,9 @@ public class DFSAdmin extends FsShell {
     ClearSpaceQuotaCommand(String[] args, int pos, FileSystem fs) {
       super(fs);
       CommandFormat c = new CommandFormat(1, Integer.MAX_VALUE);
+      c.addOptionWithValue("storageType");
       List<String> parameters = c.parse(args, pos);
-      String storageTypeString =
-          StringUtils.popOptionWithArgument("-storageType", parameters);
+      String storageTypeString = c.getOptValue("storageType");
       if (storageTypeString != null) {
         this.type = StorageType.parseStorageType(storageTypeString);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d5b0143/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
index 2d3de1f..56713f5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
@@ -7722,6 +7722,44 @@
     </test>
 
     <test> <!-- TESTED -->
+      <description>setSpaceQuota -storageType: directory with quota by storage type</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /ttt</command>
+        <dfs-admin-command>-fs NAMENODE -setSpaceQuota 1m -storageType DISK /ttt </dfs-admin-command>
+        <command>-fs NAMENODE -count -q -t DISK /ttt</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -r /ttt</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>( |\t)*1048576( |\t)*1048576 /ttt</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test> <!-- TESTED -->
+      <description>clrSpaceQuota -storageType: directory quota by storage type</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /ttt</command>
+        <dfs-admin-command>-fs NAMENODE -setSpaceQuota 1m -storageType DISK /ttt </dfs-admin-command>
+        <command>-fs NAMENODE -count -q -t DISK /ttt</command>
+        <dfs-admin-command>-fs NAMENODE -clrSpaceQuota -storageType DISK /ttt </dfs-admin-command>
+        <command>-fs NAMENODE -count -q -t DISK /ttt</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -r /ttt</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>( |\t)*none( |\t)*inf /ttt</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test> <!-- TESTED -->
       <description>count: directory using relative path with -q option</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir -p dir1</command>


[39/50] [abbrv] hadoop git commit: MAPREDUCE-6339. Job history file is not flushed correctly because isTimerActive flag is not set true when flushTimerTask is scheduled. Contributed by zhihai xu.

Posted by ji...@apache.org.
MAPREDUCE-6339. Job history file is not flushed correctly because
isTimerActive flag is not set true when flushTimerTask is scheduled.
Contributed by zhihai xu.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/f5b38477
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f5b38477
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f5b38477

Branch: refs/heads/HDFS-7240
Commit: f5b38477f9d0827b238fadd260c1dd2889531fd4
Parents: aa22450
Author: Devaraj K <de...@apache.org>
Authored: Thu Apr 30 12:33:22 2015 +0530
Committer: Devaraj K <de...@apache.org>
Committed: Thu Apr 30 12:33:22 2015 +0530

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                          | 3 +++
 .../hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java   | 7 +++++++
 .../mapreduce/jobhistory/TestJobHistoryEventHandler.java      | 2 ++
 3 files changed, 12 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f5b38477/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index e9e1a4b..c073470 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -382,6 +382,9 @@ Release 2.7.1 - UNRELEASED
     MAPREDUCE-6334. Fetcher#copyMapOutput is leaking usedMemory upon
     IOException during InMemoryMapOutput shuffle handler (Eric Payne via jlowe)
 
+    MAPREDUCE-6339. Job history file is not flushed correctly because isTimerActive 
+    flag is not set true when flushTimerTask is scheduled. (zhihai xu via devaraj)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f5b38477/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
index a0e7041..6b0ea79 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
@@ -72,6 +72,7 @@ import org.codehaus.jackson.map.ObjectMapper;
 import org.codehaus.jackson.node.ArrayNode;
 import org.codehaus.jackson.node.ObjectNode;
 
+import com.google.common.annotations.VisibleForTesting;
 /**
  * The job history events get routed to this class. This class writes the Job
  * history events to the DFS directly into a staging dir and then moved to a
@@ -1259,6 +1260,7 @@ public class JobHistoryEventHandler extends AbstractService
           if (!isTimerShutDown) {
             flushTimerTask = new FlushTimerTask(this);
             flushTimer.schedule(flushTimerTask, flushTimeout);
+            isTimerActive = true;
           }
         }
       }
@@ -1378,4 +1380,9 @@ public class JobHistoryEventHandler extends AbstractService
     }
     return JobState.KILLED.toString();
   }
+
+  @VisibleForTesting
+  boolean getFlushTimerStatus() {
+    return isTimerActive;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f5b38477/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
index de260c9..49be35b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
@@ -223,10 +223,12 @@ public class TestJobHistoryEventHandler {
       }
 
       handleNextNEvents(jheh, 9);
+      Assert.assertTrue(jheh.getFlushTimerStatus());
       verify(mockWriter, times(0)).flush();
 
       Thread.sleep(2 * 4 * 1000l); // 4 seconds should be enough. Just be safe.
       verify(mockWriter).flush();
+      Assert.assertFalse(jheh.getFlushTimerStatus());
     } finally {
       jheh.stop();
       verify(mockWriter).close();


[38/50] [abbrv] hadoop git commit: HDFS-8214. Secondary NN Web UI shows wrong date for Last Checkpoint. Contributed by Charles Lamb.

Posted by ji...@apache.org.
HDFS-8214. Secondary NN Web UI shows wrong date for Last Checkpoint. Contributed by Charles Lamb.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/aa224504
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/aa224504
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/aa224504

Branch: refs/heads/HDFS-7240
Commit: aa22450442ebe39916a6fd460fe97e347945526d
Parents: 4c1af15
Author: Andrew Wang <wa...@apache.org>
Authored: Wed Apr 29 17:37:56 2015 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Wed Apr 29 17:37:56 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  2 ++
 .../hdfs/server/namenode/SecondaryNameNode.java | 35 ++++++++++++++------
 .../namenode/SecondaryNameNodeInfoMXBean.java   |  6 ++++
 .../src/main/webapps/secondary/status.html      |  2 +-
 .../src/main/webapps/static/dfs-dust.js         |  3 ++
 5 files changed, 36 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa224504/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index a3f219b..fbeb45d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -582,6 +582,8 @@ Release 2.8.0 - UNRELEASED
     HDFS-8232. Missing datanode counters when using Metrics2 sink interface.
     (Anu Engineer via cnauroth)
 
+    HDFS-8214. Secondary NN Web UI shows wrong date for Last Checkpoint. (clamb via wang)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa224504/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
index ec7e0c9..b499e74 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
@@ -108,6 +108,7 @@ public class SecondaryNameNode implements Runnable,
 
   private final long starttime = Time.now();
   private volatile long lastCheckpointTime = 0;
+  private volatile long lastCheckpointWallclockTime = 0;
 
   private URL fsName;
   private CheckpointStorage checkpointImage;
@@ -134,8 +135,9 @@ public class SecondaryNameNode implements Runnable,
       + "\nName Node Address      : " + nameNodeAddr
       + "\nStart Time             : " + new Date(starttime)
       + "\nLast Checkpoint        : " + (lastCheckpointTime == 0? "--":
-				       ((Time.monotonicNow() - lastCheckpointTime) / 1000))
-	                            + " seconds ago"
+        new Date(lastCheckpointWallclockTime))
+      + " (" + ((Time.monotonicNow() - lastCheckpointTime) / 1000)
+      + " seconds ago)"
       + "\nCheckpoint Period      : " + checkpointConf.getPeriod() + " seconds"
       + "\nCheckpoint Transactions: " + checkpointConf.getTxnCount()
       + "\nCheckpoint Dirs        : " + checkpointDirs
@@ -388,12 +390,14 @@ public class SecondaryNameNode implements Runnable,
         if(UserGroupInformation.isSecurityEnabled())
           UserGroupInformation.getCurrentUser().checkTGTAndReloginFromKeytab();
         
-        final long now = Time.monotonicNow();
+        final long monotonicNow = Time.monotonicNow();
+        final long now = Time.now();
 
         if (shouldCheckpointBasedOnCount() ||
-            now >= lastCheckpointTime + 1000 * checkpointConf.getPeriod()) {
+            monotonicNow >= lastCheckpointTime + 1000 * checkpointConf.getPeriod()) {
           doCheckpoint();
-          lastCheckpointTime = now;
+          lastCheckpointTime = monotonicNow;
+          lastCheckpointWallclockTime = now;
         }
       } catch (IOException e) {
         LOG.error("Exception in doCheckpoint", e);
@@ -695,22 +699,31 @@ public class SecondaryNameNode implements Runnable,
     checkpointThread.start();
   }
 
-  @Override // SecondaryNameNodeInfoMXXBean
+  @Override // SecondaryNameNodeInfoMXBean
   public String getHostAndPort() {
     return NetUtils.getHostPortString(nameNodeAddr);
   }
 
-  @Override // SecondaryNameNodeInfoMXXBean
+  @Override // SecondaryNameNodeInfoMXBean
   public long getStartTime() {
     return starttime;
   }
 
-  @Override // SecondaryNameNodeInfoMXXBean
+  @Override // SecondaryNameNodeInfoMXBean
   public long getLastCheckpointTime() {
-    return lastCheckpointTime;
+    return lastCheckpointWallclockTime;
   }
 
-  @Override // SecondaryNameNodeInfoMXXBean
+  @Override // SecondaryNameNodeInfoMXBean
+  public long getLastCheckpointDeltaMs() {
+    if (lastCheckpointTime == 0) {
+      return -1;
+    } else {
+      return (Time.monotonicNow() - lastCheckpointTime);
+    }
+  }
+
+  @Override // SecondaryNameNodeInfoMXBean
   public String[] getCheckpointDirectories() {
     ArrayList<String> r = Lists.newArrayListWithCapacity(checkpointDirs.size());
     for (URI d : checkpointDirs) {
@@ -719,7 +732,7 @@ public class SecondaryNameNode implements Runnable,
     return r.toArray(new String[r.size()]);
   }
 
-  @Override // SecondaryNameNodeInfoMXXBean
+  @Override // SecondaryNameNodeInfoMXBean
   public String[] getCheckpointEditlogDirectories() {
     ArrayList<String> r = Lists.newArrayListWithCapacity(checkpointEditsDirs.size());
     for (URI d : checkpointEditsDirs) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa224504/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNodeInfoMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNodeInfoMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNodeInfoMXBean.java
index 01f6fac..785c5ee 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNodeInfoMXBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNodeInfoMXBean.java
@@ -42,6 +42,12 @@ public interface SecondaryNameNodeInfoMXBean extends VersionInfoMXBean {
   public long getLastCheckpointTime();
 
   /**
+   * @return the number of msec since the last checkpoint, or -1 if no
+   * checkpoint has been done yet.
+   */
+  public long getLastCheckpointDeltaMs();
+
+  /**
    * @return the directories that store the checkpoint images
    */
   public String[] getCheckpointDirectories();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa224504/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/status.html
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/status.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/status.html
index 7893a17..2ef58ad 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/status.html
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/status.html
@@ -65,7 +65,7 @@
   <tr><th>Compiled</th><td>{CompileInfo}</td></tr>
   <tr><th>NameNode Address</th><td>{HostAndPort}</td></tr>
   <tr><th>Started</th><td>{StartTime|date_tostring}</td></tr>
-  <tr><th>Last Checkpoint</th><td>{@if cond="{LastCheckpointTime} === 0"}Never{:else}{LastCheckpointTime|date_tostring}{/if}</td></tr>
+  <tr><th>Last Checkpoint</th><td>{@if cond="{LastCheckpointTime} === 0"}Never{:else}{LastCheckpointTime|date_tostring} ({LastCheckpointDeltaMs|fmt_time} ago){/if}</td></tr>
   <tr><th>Checkpoint Period</th><td>{CheckpointPeriod} seconds</td></tr>
   <tr><th>Checkpoint Transactions</th><td>{TxnCount}</td></tr>
 </table>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa224504/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dfs-dust.js
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dfs-dust.js b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dfs-dust.js
index 3c8efd9..f483d23 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dfs-dust.js
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dfs-dust.js
@@ -40,6 +40,9 @@
     },
 
     'fmt_time': function (v) {
+      if (v < 0) {
+        return "unknown";
+      }
       var s = Math.floor(v / 1000), h = Math.floor(s / 3600);
       s -= h * 3600;
       var m = Math.floor(s / 60);


[23/50] [abbrv] hadoop git commit: MAPREDUCE-6324. Fixed MapReduce uber jobs to not fail the udpate of AM-RM tokens when they roll-over. Contributed by Jason Lowe.

Posted by ji...@apache.org.
MAPREDUCE-6324. Fixed MapReduce uber jobs to not fail the udpate of AM-RM tokens when they roll-over. Contributed by Jason Lowe.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/9fc32c5c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9fc32c5c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9fc32c5c

Branch: refs/heads/HDFS-7240
Commit: 9fc32c5c4d1d5f50c605bdb0e3b13f44c86660c8
Parents: 32cd2c8
Author: Vinod Kumar Vavilapalli <vi...@apache.org>
Authored: Mon Apr 27 14:58:16 2015 -0700
Committer: Vinod Kumar Vavilapalli <vi...@apache.org>
Committed: Mon Apr 27 14:58:16 2015 -0700

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |   4 +
 .../v2/app/local/LocalContainerAllocator.java   |  28 +++-
 .../app/local/TestLocalContainerAllocator.java  | 152 +++++++++++++++++--
 3 files changed, 173 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fc32c5c/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index f1d0493..eb6feb9 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -3,6 +3,7 @@ Hadoop MapReduce Change Log
 Trunk (Unreleased)
 
   INCOMPATIBLE CHANGES
+
     MAPREDUCE-5785. Derive heap size or mapreduce.*.memory.mb automatically.
     (Gera Shegalov and Karthik Kambatla via gera)
 
@@ -372,6 +373,9 @@ Release 2.7.1 - UNRELEASED
     MAPREDUCE-6238. MR2 can't run local jobs with -libjars command options
     which is a regression from MR1 (zxu via rkanter)
 
+    MAPREDUCE-6324. Fixed MapReduce uber jobs to not fail the udpate of AM-RM
+    tokens when they roll-over. (Jason Lowe via vinodkv)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fc32c5c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/local/LocalContainerAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/local/LocalContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/local/LocalContainerAllocator.java
index 74dfb39..aed1023 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/local/LocalContainerAllocator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/local/LocalContainerAllocator.java
@@ -18,11 +18,13 @@
 
 package org.apache.hadoop.mapreduce.v2.app.local;
 
+import java.io.IOException;
 import java.util.ArrayList;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.JobCounter;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
@@ -35,17 +37,22 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerAssigned
 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.RMCommunicator;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 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.api.records.Token;
+import org.apache.hadoop.yarn.client.ClientRMProxy;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.ApplicationAttemptNotFoundException;
 import org.apache.hadoop.yarn.exceptions.ApplicationMasterNotRegisteredException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 
 /**
  * Allocates containers locally. Doesn't allocate a real container;
@@ -99,8 +106,9 @@ public class LocalContainerAllocator extends RMCommunicator
         AllocateRequest.newInstance(this.lastResponseID,
           super.getApplicationProgress(), new ArrayList<ResourceRequest>(),
         new ArrayList<ContainerId>(), null);
+    AllocateResponse allocateResponse = null;
     try {
-      scheduler.allocate(allocateRequest);
+      allocateResponse = scheduler.allocate(allocateRequest);
       // Reset retry count if no exception occurred.
       retrystartTime = System.currentTimeMillis();
     } catch (ApplicationAttemptNotFoundException e) {
@@ -131,6 +139,24 @@ public class LocalContainerAllocator extends RMCommunicator
       // continue to attempt to contact the RM.
       throw e;
     }
+
+    if (allocateResponse != null) {
+      this.lastResponseID = allocateResponse.getResponseId();
+      Token token = allocateResponse.getAMRMToken();
+      if (token != null) {
+        updateAMRMToken(token);
+      }
+    }
+  }
+
+  private void updateAMRMToken(Token token) throws IOException {
+    org.apache.hadoop.security.token.Token<AMRMTokenIdentifier> amrmToken =
+        new org.apache.hadoop.security.token.Token<AMRMTokenIdentifier>(token
+          .getIdentifier().array(), token.getPassword().array(), new Text(
+          token.getKind()), new Text(token.getService()));
+    UserGroupInformation currentUGI = UserGroupInformation.getCurrentUser();
+    currentUGI.addToken(amrmToken);
+    amrmToken.setService(ClientRMProxy.getAMRMTokenService(getConfig()));
   }
 
   @SuppressWarnings("unchecked")

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fc32c5c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java
index 90dbe48..f901ed8 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java
@@ -22,23 +22,43 @@ import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.Collections;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.ClusterInfo;
 import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
 import org.apache.hadoop.mapreduce.v2.app.job.Job;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 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.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerResourceDecrease;
+import org.apache.hadoop.yarn.api.records.ContainerResourceIncrease;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.NMToken;
+import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.client.ClientRMProxy;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.ipc.RPCUtil;
+import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
+import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -48,8 +68,13 @@ public class TestLocalContainerAllocator {
   public void testRMConnectionRetry() throws Exception {
     // verify the connection exception is thrown
     // if we haven't exhausted the retry interval
+    ApplicationMasterProtocol mockScheduler =
+        mock(ApplicationMasterProtocol.class);
+    when(mockScheduler.allocate(isA(AllocateRequest.class)))
+      .thenThrow(RPCUtil.getRemoteException(new IOException("forcefail")));
     Configuration conf = new Configuration();
-    LocalContainerAllocator lca = new StubbedLocalContainerAllocator();
+    LocalContainerAllocator lca =
+        new StubbedLocalContainerAllocator(mockScheduler);
     lca.init(conf);
     lca.start();
     try {
@@ -63,7 +88,7 @@ public class TestLocalContainerAllocator {
 
     // verify YarnRuntimeException is thrown when the retry interval has expired
     conf.setLong(MRJobConfig.MR_AM_TO_RM_WAIT_INTERVAL_MS, 0);
-    lca = new StubbedLocalContainerAllocator();
+    lca = new StubbedLocalContainerAllocator(mockScheduler);
     lca.init(conf);
     lca.start();
     try {
@@ -76,12 +101,84 @@ public class TestLocalContainerAllocator {
     }
   }
 
+  @Test
+  public void testAllocResponseId() throws Exception {
+    ApplicationMasterProtocol scheduler = new MockScheduler();
+    Configuration conf = new Configuration();
+    LocalContainerAllocator lca =
+        new StubbedLocalContainerAllocator(scheduler);
+    lca.init(conf);
+    lca.start();
+
+    // do two heartbeats to verify the response ID is being tracked
+    lca.heartbeat();
+    lca.heartbeat();
+    lca.close();
+  }
+
+  @Test
+  public void testAMRMTokenUpdate() throws Exception {
+    Configuration conf = new Configuration();
+    ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(
+        ApplicationId.newInstance(1, 1), 1);
+    AMRMTokenIdentifier oldTokenId = new AMRMTokenIdentifier(attemptId, 1);
+    AMRMTokenIdentifier newTokenId = new AMRMTokenIdentifier(attemptId, 2);
+    Token<AMRMTokenIdentifier> oldToken = new Token<AMRMTokenIdentifier>(
+        oldTokenId.getBytes(), "oldpassword".getBytes(), oldTokenId.getKind(),
+        new Text());
+    Token<AMRMTokenIdentifier> newToken = new Token<AMRMTokenIdentifier>(
+        newTokenId.getBytes(), "newpassword".getBytes(), newTokenId.getKind(),
+        new Text());
+
+    MockScheduler scheduler = new MockScheduler();
+    scheduler.amToken = newToken;
+
+    final LocalContainerAllocator lca =
+        new StubbedLocalContainerAllocator(scheduler);
+    lca.init(conf);
+    lca.start();
+
+    UserGroupInformation testUgi = UserGroupInformation.createUserForTesting(
+        "someuser", new String[0]);
+    testUgi.addToken(oldToken);
+    testUgi.doAs(new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws Exception {
+            lca.heartbeat();
+            return null;
+          }
+    });
+    lca.close();
+
+    // verify there is only one AMRM token in the UGI and it matches the
+    // updated token from the RM
+    int tokenCount = 0;
+    Token<? extends TokenIdentifier> ugiToken = null;
+    for (Token<? extends TokenIdentifier> token : testUgi.getTokens()) {
+      if (AMRMTokenIdentifier.KIND_NAME.equals(token.getKind())) {
+        ugiToken = token;
+        ++tokenCount;
+      }
+    }
+
+    Assert.assertEquals("too many AMRM tokens", 1, tokenCount);
+    Assert.assertArrayEquals("token identifier not updated",
+        newToken.getIdentifier(), ugiToken.getIdentifier());
+    Assert.assertArrayEquals("token password not updated",
+        newToken.getPassword(), ugiToken.getPassword());
+    Assert.assertEquals("AMRM token service not updated",
+        new Text(ClientRMProxy.getAMRMTokenService(conf)),
+        ugiToken.getService());
+  }
+
   private static class StubbedLocalContainerAllocator
     extends LocalContainerAllocator {
+    private ApplicationMasterProtocol scheduler;
 
-    public StubbedLocalContainerAllocator() {
+    public StubbedLocalContainerAllocator(ApplicationMasterProtocol scheduler) {
       super(mock(ClientService.class), createAppContext(),
           "nmhost", 1, 2, null);
+      this.scheduler = scheduler;
     }
 
     @Override
@@ -99,13 +196,6 @@ public class TestLocalContainerAllocator {
 
     @Override
     protected ApplicationMasterProtocol createSchedulerProxy() {
-      ApplicationMasterProtocol scheduler = mock(ApplicationMasterProtocol.class);
-      try {
-        when(scheduler.allocate(isA(AllocateRequest.class)))
-          .thenThrow(RPCUtil.getRemoteException(new IOException("forcefail")));
-      } catch (YarnException e) {
-      } catch (IOException e) {
-      }
       return scheduler;
     }
 
@@ -126,4 +216,46 @@ public class TestLocalContainerAllocator {
       return ctx;
     }
   }
+
+  private static class MockScheduler implements ApplicationMasterProtocol {
+    int responseId = 0;
+    Token<AMRMTokenIdentifier> amToken = null;
+
+    @Override
+    public RegisterApplicationMasterResponse registerApplicationMaster(
+        RegisterApplicationMasterRequest request) throws YarnException,
+        IOException {
+      return null;
+    }
+
+    @Override
+    public FinishApplicationMasterResponse finishApplicationMaster(
+        FinishApplicationMasterRequest request) throws YarnException,
+        IOException {
+      return null;
+    }
+
+    @Override
+    public AllocateResponse allocate(AllocateRequest request)
+        throws YarnException, IOException {
+      Assert.assertEquals("response ID mismatch",
+          responseId, request.getResponseId());
+      ++responseId;
+      org.apache.hadoop.yarn.api.records.Token yarnToken = null;
+      if (amToken != null) {
+        yarnToken = org.apache.hadoop.yarn.api.records.Token.newInstance(
+            amToken.getIdentifier(), amToken.getKind().toString(),
+            amToken.getPassword(), amToken.getService().toString());
+      }
+      return AllocateResponse.newInstance(responseId,
+          Collections.<ContainerStatus>emptyList(),
+          Collections.<Container>emptyList(),
+          Collections.<NodeReport>emptyList(),
+          Resources.none(), null, 1, null,
+          Collections.<NMToken>emptyList(),
+          yarnToken,
+          Collections.<ContainerResourceIncrease>emptyList(),
+          Collections.<ContainerResourceDecrease>emptyList());
+    }
+  }
 }


[45/50] [abbrv] hadoop git commit: HDFS-8200. Refactor FSDirStatAndListingOp. Contributed by Haohui Mai.

Posted by ji...@apache.org.
HDFS-8200. Refactor FSDirStatAndListingOp. Contributed by Haohui Mai.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/c55d6090
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c55d6090
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c55d6090

Branch: refs/heads/HDFS-7240
Commit: c55d609053fe24b3a50fbe17dc1b47717b453ed6
Parents: 7e8639f
Author: Haohui Mai <wh...@apache.org>
Authored: Thu Apr 30 13:41:46 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Thu Apr 30 13:41:46 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   2 +
 .../server/namenode/FSDirStatAndListingOp.java  | 155 ++++++++++---------
 .../hdfs/server/namenode/FSEditLogLoader.java   |   6 +-
 3 files changed, 91 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c55d6090/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index e092dc8..29e3f85 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -486,6 +486,8 @@ Release 2.8.0 - UNRELEASED
     HDFS-5574. Remove buffer copy in BlockReader.skip.
     (Binglin Chang via aajisaka)
 
+    HDFS-8200. Refactor FSDirStatAndListingOp. (wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c55d6090/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
index cea2b82..e77ddae 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
@@ -24,7 +24,6 @@ import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.DirectoryListingStartAfterNotFoundException;
 import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.fs.InvalidPathException;
-import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtil;
@@ -180,10 +179,14 @@ class FSDirStatAndListingOp {
           .BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
 
       if (!targetNode.isDirectory()) {
+        INodeAttributes nodeAttrs = getINodeAttributes(
+            fsd, src, HdfsFileStatus.EMPTY_NAME, targetNode,
+            snapshot);
         return new DirectoryListing(
-            new HdfsFileStatus[]{createFileStatus(fsd, src,
-                HdfsFileStatus.EMPTY_NAME, targetNode, needLocation,
-                parentStoragePolicy, snapshot, isRawPath, iip)}, 0);
+            new HdfsFileStatus[]{ createFileStatus(
+                fsd, HdfsFileStatus.EMPTY_NAME, targetNode, nodeAttrs,
+                needLocation, parentStoragePolicy, snapshot, isRawPath, iip)
+            }, 0);
       }
 
       final INodeDirectory dirInode = targetNode.asDirectory();
@@ -200,8 +203,11 @@ class FSDirStatAndListingOp {
         byte curPolicy = isSuperUser && !cur.isSymlink()?
             cur.getLocalStoragePolicyID():
             HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
-        listing[i] = createFileStatus(fsd, src, cur.getLocalNameBytes(), cur,
-            needLocation, getStoragePolicyID(curPolicy,
+        INodeAttributes nodeAttrs = getINodeAttributes(
+            fsd, src, cur.getLocalNameBytes(), cur,
+            snapshot);
+        listing[i] = createFileStatus(fsd, cur.getLocalNameBytes(),
+            cur, nodeAttrs, needLocation, getStoragePolicyID(curPolicy,
                 parentStoragePolicy), snapshot, isRawPath, iip);
         listingCnt++;
         if (needLocation) {
@@ -253,9 +259,15 @@ class FSDirStatAndListingOp {
     final HdfsFileStatus listing[] = new HdfsFileStatus[numOfListing];
     for (int i = 0; i < numOfListing; i++) {
       Snapshot.Root sRoot = snapshots.get(i + skipSize).getRoot();
-      listing[i] = createFileStatus(fsd, src, sRoot.getLocalNameBytes(), sRoot,
-          HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, Snapshot.CURRENT_STATE_ID,
-          false, INodesInPath.fromINode(sRoot));
+      INodeAttributes nodeAttrs = getINodeAttributes(
+          fsd, src, sRoot.getLocalNameBytes(),
+          node, Snapshot.CURRENT_STATE_ID);
+      listing[i] = createFileStatus(
+          fsd, sRoot.getLocalNameBytes(),
+          sRoot, nodeAttrs,
+          HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
+          Snapshot.CURRENT_STATE_ID, false,
+          INodesInPath.fromINode(sRoot));
     }
     return new DirectoryListing(
         listing, snapshots.size() - skipSize - numOfListing);
@@ -276,11 +288,20 @@ class FSDirStatAndListingOp {
     fsd.readLock();
     try {
       final INode i = src.getLastINode();
-      byte policyId = includeStoragePolicy && i != null && !i.isSymlink() ?
-          i.getStoragePolicyID() : HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
-      return i == null ? null : createFileStatus(
-          fsd, path, HdfsFileStatus.EMPTY_NAME, i, policyId,
-          src.getPathSnapshotId(), isRawPath, src);
+      if (i == null) {
+        return null;
+      }
+
+      byte policyId = includeStoragePolicy && !i.isSymlink() ?
+          i.getStoragePolicyID() : HdfsConstantsClient
+          .BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
+      INodeAttributes nodeAttrs = getINodeAttributes(
+          fsd, path, HdfsFileStatus.EMPTY_NAME, i, src.getPathSnapshotId());
+      return createFileStatus(
+          fsd, HdfsFileStatus.EMPTY_NAME,
+          i, nodeAttrs, policyId,
+          src.getPathSnapshotId(),
+          isRawPath, src);
     } finally {
       fsd.readUnlock();
     }
@@ -309,23 +330,6 @@ class FSDirStatAndListingOp {
     }
   }
 
-  /**
-   * Currently we only support "ls /xxx/.snapshot" which will return all the
-   * snapshots of a directory. The FSCommand Ls will first call getFileInfo to
-   * make sure the file/directory exists (before the real getListing call).
-   * Since we do not have a real INode for ".snapshot", we return an empty
-   * non-null HdfsFileStatus here.
-   */
-  private static HdfsFileStatus getFileInfo4DotSnapshot(
-      FSDirectory fsd, String src)
-      throws UnresolvedLinkException {
-    if (fsd.getINode4DotSnapshot(src) != null) {
-      return new HdfsFileStatus(0, true, 0, 0, 0, 0, null, null, null, null,
-          HdfsFileStatus.EMPTY_NAME, -1L, 0, null,
-          HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED);
-    }
-    return null;
-  }
 
   /**
    * create an hdfs file status from an inode
@@ -339,52 +343,63 @@ class FSDirStatAndListingOp {
    * @return a file status
    * @throws java.io.IOException if any error occurs
    */
-  static HdfsFileStatus createFileStatus(
-      FSDirectory fsd, String fullPath, byte[] path, INode node,
+  private static HdfsFileStatus createFileStatus(
+      FSDirectory fsd, byte[] path, INode node, INodeAttributes nodeAttrs,
       boolean needLocation, byte storagePolicy, int snapshot, boolean isRawPath,
       INodesInPath iip)
       throws IOException {
     if (needLocation) {
-      return createLocatedFileStatus(fsd, fullPath, path, node, storagePolicy,
-          snapshot, isRawPath, iip);
+      return createLocatedFileStatus(fsd, path, node, nodeAttrs, storagePolicy,
+                                     snapshot, isRawPath, iip);
     } else {
-      return createFileStatus(fsd, fullPath, path, node, storagePolicy, snapshot,
-          isRawPath, iip);
+      return createFileStatus(fsd, path, node, nodeAttrs, storagePolicy,
+                              snapshot, isRawPath, iip);
     }
   }
 
   /**
    * Create FileStatus by file INode
    */
-  static HdfsFileStatus createFileStatus(
+  static HdfsFileStatus createFileStatusForEditLog(
       FSDirectory fsd, String fullPath, byte[] path, INode node,
       byte storagePolicy, int snapshot, boolean isRawPath,
       INodesInPath iip) throws IOException {
-     long size = 0;     // length is zero for directories
-     short replication = 0;
-     long blocksize = 0;
-     final boolean isEncrypted;
-
-     final FileEncryptionInfo feInfo = isRawPath ? null :
-         fsd.getFileEncryptionInfo(node, snapshot, iip);
-
-     if (node.isFile()) {
-       final INodeFile fileNode = node.asFile();
-       size = fileNode.computeFileSize(snapshot);
-       replication = fileNode.getFileReplication(snapshot);
-       blocksize = fileNode.getPreferredBlockSize();
-       isEncrypted = (feInfo != null) ||
-           (isRawPath && fsd.isInAnEZ(INodesInPath.fromINode(node)));
-     } else {
-       isEncrypted = fsd.isInAnEZ(INodesInPath.fromINode(node));
-     }
-
-     int childrenNum = node.isDirectory() ?
-         node.asDirectory().getChildrenNum(snapshot) : 0;
-
-     INodeAttributes nodeAttrs =
-         fsd.getAttributes(fullPath, path, node, snapshot);
-     return new HdfsFileStatus(
+    INodeAttributes nodeAttrs = getINodeAttributes(
+        fsd, fullPath, path, node, snapshot);
+    return createFileStatus(fsd, path, node, nodeAttrs,
+                            storagePolicy, snapshot, isRawPath, iip);
+  }
+
+  /**
+   * Create FileStatus by file INode
+   */
+  static HdfsFileStatus createFileStatus(
+      FSDirectory fsd, byte[] path, INode node,
+      INodeAttributes nodeAttrs, byte storagePolicy, int snapshot,
+      boolean isRawPath, INodesInPath iip) throws IOException {
+    long size = 0;     // length is zero for directories
+    short replication = 0;
+    long blocksize = 0;
+    final boolean isEncrypted;
+
+    final FileEncryptionInfo feInfo = isRawPath ? null :
+        fsd.getFileEncryptionInfo(node, snapshot, iip);
+
+    if (node.isFile()) {
+      final INodeFile fileNode = node.asFile();
+      size = fileNode.computeFileSize(snapshot);
+      replication = fileNode.getFileReplication(snapshot);
+      blocksize = fileNode.getPreferredBlockSize();
+      isEncrypted = (feInfo != null) ||
+          (isRawPath && fsd.isInAnEZ(INodesInPath.fromINode(node)));
+    } else {
+      isEncrypted = fsd.isInAnEZ(INodesInPath.fromINode(node));
+    }
+
+    int childrenNum = node.isDirectory() ?
+        node.asDirectory().getChildrenNum(snapshot) : 0;
+
+    return new HdfsFileStatus(
         size,
         node.isDirectory(),
         replication,
@@ -402,13 +417,18 @@ class FSDirStatAndListingOp {
         storagePolicy);
   }
 
+  private static INodeAttributes getINodeAttributes(
+      FSDirectory fsd, String fullPath, byte[] path, INode node, int snapshot) {
+    return fsd.getAttributes(fullPath, path, node, snapshot);
+  }
+
   /**
    * Create FileStatus with location info by file INode
    */
   private static HdfsLocatedFileStatus createLocatedFileStatus(
-      FSDirectory fsd, String fullPath, byte[] path, INode node,
-      byte storagePolicy, int snapshot, boolean isRawPath,
-      INodesInPath iip) throws IOException {
+      FSDirectory fsd, byte[] path, INode node, INodeAttributes nodeAttrs,
+      byte storagePolicy, int snapshot,
+      boolean isRawPath, INodesInPath iip) throws IOException {
     assert fsd.hasReadLock();
     long size = 0; // length is zero for directories
     short replication = 0;
@@ -442,8 +462,6 @@ class FSDirStatAndListingOp {
     int childrenNum = node.isDirectory() ?
         node.asDirectory().getChildrenNum(snapshot) : 0;
 
-    INodeAttributes nodeAttrs =
-        fsd.getAttributes(fullPath, path, node, snapshot);
     HdfsLocatedFileStatus status =
         new HdfsLocatedFileStatus(size, node.isDirectory(), replication,
           blocksize, node.getModificationTime(snapshot),
@@ -468,7 +486,6 @@ class FSDirStatAndListingOp {
    * return an FsPermissionExtension.
    *
    * @param node INode to check
-   * @param snapshot int snapshot ID
    * @param isEncrypted boolean true if the file/dir is encrypted
    * @return FsPermission from inode, with ACL bit on if the inode has an ACL
    * and encrypted bit on if it represents an encrypted file/dir.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c55d6090/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index 43171de..d234cf2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@ -378,7 +378,7 @@ public class FSEditLogLoader {
 
         // add the op into retry cache if necessary
         if (toAddRetryCache) {
-          HdfsFileStatus stat = FSDirStatAndListingOp.createFileStatus(
+          HdfsFileStatus stat = FSDirStatAndListingOp.createFileStatusForEditLog(
               fsNamesys.dir, path, HdfsFileStatus.EMPTY_NAME, newFile,
               HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, Snapshot.CURRENT_STATE_ID,
               false, iip);
@@ -397,7 +397,7 @@ public class FSEditLogLoader {
               false);
           // add the op into retry cache if necessary
           if (toAddRetryCache) {
-            HdfsFileStatus stat = FSDirStatAndListingOp.createFileStatus(
+            HdfsFileStatus stat = FSDirStatAndListingOp.createFileStatusForEditLog(
                 fsNamesys.dir, path,
                 HdfsFileStatus.EMPTY_NAME, newFile,
                 HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
@@ -471,7 +471,7 @@ public class FSEditLogLoader {
             false, false);
         // add the op into retry cache if necessary
         if (toAddRetryCache) {
-          HdfsFileStatus stat = FSDirStatAndListingOp.createFileStatus(
+          HdfsFileStatus stat = FSDirStatAndListingOp.createFileStatusForEditLog(
               fsNamesys.dir, path, HdfsFileStatus.EMPTY_NAME, file,
               HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
               Snapshot.CURRENT_STATE_ID, false, iip);


[41/50] [abbrv] hadoop git commit: HADOOP-11821. Fix findbugs warnings in hadoop-sls. Contributed by Brahma Reddy Battula.

Posted by ji...@apache.org.
HADOOP-11821. Fix findbugs warnings in hadoop-sls. Contributed by Brahma Reddy Battula.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/f384a063
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f384a063
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f384a063

Branch: refs/heads/HDFS-7240
Commit: f384a063a653b33d69f7d2c7d4fd45c24b5aa46a
Parents: e89fc53
Author: Akira Ajisaka <aa...@apache.org>
Authored: Thu Apr 30 19:34:41 2015 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Thu Apr 30 19:34:41 2015 +0900

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |   3 +
 .../hadoop/yarn/sls/RumenToSLSConverter.java    |  47 ++++----
 .../org/apache/hadoop/yarn/sls/SLSRunner.java   |  41 +++----
 .../sls/scheduler/ResourceSchedulerWrapper.java |  13 ++-
 .../sls/scheduler/SLSCapacityScheduler.java     | 112 ++++++++-----------
 .../apache/hadoop/yarn/sls/utils/SLSUtils.java  |  27 +++--
 .../apache/hadoop/yarn/sls/web/SLSWebApp.java   |  31 +++--
 7 files changed, 132 insertions(+), 142 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f384a063/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 39d14c9..373b33e 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -579,6 +579,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11881. test-patch.sh javac result is wildly wrong (Kengo Seki via
     aw)
 
+    HADOOP-11821. Fix findbugs warnings in hadoop-sls.
+    (Brahma Reddy Battula via aajisaka)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f384a063/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/RumenToSLSConverter.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/RumenToSLSConverter.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/RumenToSLSConverter.java
index 2d4b4ae..63d022b 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/RumenToSLSConverter.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/RumenToSLSConverter.java
@@ -17,22 +17,12 @@
  */
 package org.apache.hadoop.yarn.sls;
 
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.CommandLineParser;
-import org.apache.commons.cli.GnuParser;
-import org.apache.commons.cli.Options;
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.codehaus.jackson.JsonFactory;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.codehaus.jackson.map.ObjectWriter;
-
-import org.apache.hadoop.yarn.sls.utils.SLSUtils;
-
 import java.io.File;
-import java.io.FileReader;
-import java.io.FileWriter;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
 import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
 import java.io.Reader;
 import java.io.Writer;
 import java.util.ArrayList;
@@ -44,6 +34,17 @@ import java.util.Set;
 import java.util.TreeMap;
 import java.util.TreeSet;
 
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.sls.utils.SLSUtils;
+import org.codehaus.jackson.JsonFactory;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.map.ObjectWriter;
+
 @Private
 @Unstable
 public class RumenToSLSConverter {
@@ -119,10 +120,10 @@ public class RumenToSLSConverter {
 
   private static void generateSLSLoadFile(String inputFile, String outputFile)
           throws IOException {
-    Reader input = new FileReader(inputFile);
-    try {
-      Writer output = new FileWriter(outputFile);
-      try {
+    try (Reader input =
+        new InputStreamReader(new FileInputStream(inputFile), "UTF-8")) {
+      try (Writer output =
+          new OutputStreamWriter(new FileOutputStream(outputFile), "UTF-8")) {
         ObjectMapper mapper = new ObjectMapper();
         ObjectWriter writer = mapper.writerWithDefaultPrettyPrinter();
         Iterator<Map> i = mapper.readValues(
@@ -131,19 +132,15 @@ public class RumenToSLSConverter {
           Map m = i.next();
           output.write(writer.writeValueAsString(createSLSJob(m)) + EOL);
         }
-      } finally {
-        output.close();
       }
-    } finally {
-      input.close();
     }
   }
 
   @SuppressWarnings("unchecked")
   private static void generateSLSNodeFile(String outputFile)
           throws IOException {
-    Writer output = new FileWriter(outputFile);
-    try {
+    try (Writer output =
+        new OutputStreamWriter(new FileOutputStream(outputFile), "UTF-8")) {
       ObjectMapper mapper = new ObjectMapper();
       ObjectWriter writer = mapper.writerWithDefaultPrettyPrinter();
       for (Map.Entry<String, Set<String>> entry : rackNodeMap.entrySet()) {
@@ -158,8 +155,6 @@ public class RumenToSLSConverter {
         rack.put("nodes", nodes);
         output.write(writer.writeValueAsString(rack) + EOL);
       }
-    } finally {
-      output.close();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f384a063/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java
index 9baa736..b36edc9 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java
@@ -18,51 +18,51 @@
 package org.apache.hadoop.yarn.sls;
 
 import java.io.File;
-import java.io.FileReader;
+import java.io.FileInputStream;
 import java.io.IOException;
+import java.io.InputStreamReader;
 import java.io.Reader;
 import java.text.MessageFormat;
-import java.util.Map;
+import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashMap;
-import java.util.Set;
 import java.util.HashSet;
-import java.util.List;
-import java.util.ArrayList;
 import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
 import java.util.Random;
-import java.util.Arrays;
+import java.util.Set;
 
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.Options;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.tools.rumen.JobTraceReader;
 import org.apache.hadoop.tools.rumen.LoggedJob;
 import org.apache.hadoop.tools.rumen.LoggedTask;
 import org.apache.hadoop.tools.rumen.LoggedTaskAttempt;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.sls.appmaster.AMSimulator;
 import org.apache.hadoop.yarn.sls.conf.SLSConfiguration;
 import org.apache.hadoop.yarn.sls.nodemanager.NMSimulator;
 import org.apache.hadoop.yarn.sls.scheduler.ContainerSimulator;
 import org.apache.hadoop.yarn.sls.scheduler.ResourceSchedulerWrapper;
 import org.apache.hadoop.yarn.sls.scheduler.SLSCapacityScheduler;
+import org.apache.hadoop.yarn.sls.scheduler.SchedulerWrapper;
 import org.apache.hadoop.yarn.sls.scheduler.TaskRunner;
-import  org.apache.hadoop.yarn.sls.scheduler.SchedulerWrapper;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.CommandLineParser;
-import org.apache.commons.cli.GnuParser;
-import org.apache.commons.cli.Options;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
-import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.sls.utils.SLSUtils;
 import org.apache.log4j.Logger;
 import org.codehaus.jackson.JsonFactory;
@@ -277,7 +277,8 @@ public class SLSRunner {
     JsonFactory jsonF = new JsonFactory();
     ObjectMapper mapper = new ObjectMapper();
     for (String inputTrace : inputTraces) {
-      Reader input = new FileReader(inputTrace);
+      Reader input =
+          new InputStreamReader(new FileInputStream(inputTrace), "UTF-8");
       try {
         Iterator<Map> i = mapper.readValues(jsonF.createJsonParser(input),
                 Map.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f384a063/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java
index 0a80291..08cb1e6 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java
@@ -19,8 +19,9 @@ package org.apache.hadoop.yarn.sls.scheduler;
 
 import java.io.BufferedWriter;
 import java.io.File;
-import java.io.FileWriter;
+import java.io.FileOutputStream;
 import java.io.IOException;
+import java.io.OutputStreamWriter;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -487,8 +488,9 @@ final public class ResourceSchedulerWrapper
             TimeUnit.MILLISECONDS);
 
     // application running information
-    jobRuntimeLogBW = new BufferedWriter(
-            new FileWriter(metricsOutputDir + "/jobruntime.csv"));
+    jobRuntimeLogBW =
+        new BufferedWriter(new OutputStreamWriter(new FileOutputStream(
+            metricsOutputDir + "/jobruntime.csv"), "UTF-8"));
     jobRuntimeLogBW.write("JobID,real_start_time,real_end_time," +
             "simulate_start_time,simulate_end_time" + EOL);
     jobRuntimeLogBW.flush();
@@ -692,8 +694,9 @@ final public class ResourceSchedulerWrapper
     private boolean firstLine = true;
     public MetricsLogRunnable() {
       try {
-        metricsLogBW = new BufferedWriter(
-                new FileWriter(metricsOutputDir + "/realtimetrack.json"));
+        metricsLogBW =
+            new BufferedWriter(new OutputStreamWriter(new FileOutputStream(
+                metricsOutputDir + "/realtimetrack.json"), "UTF-8"));
         metricsLogBW.write("[");
       } catch (IOException e) {
         e.printStackTrace();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f384a063/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java
index 06addfb..a4416db 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java
@@ -17,90 +17,66 @@
  */
 package org.apache.hadoop.yarn.sls.scheduler;
 
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.util.ShutdownHookManager;
-import org.apache.hadoop.yarn.sls.SLSRunner;
-import org.apache.hadoop.yarn.sls.conf.SLSConfiguration;
-import org.apache.hadoop.yarn.sls.web.SLSWebApp;
-import com.codahale.metrics.Counter;
-import com.codahale.metrics.CsvReporter;
-import com.codahale.metrics.Gauge;
-import com.codahale.metrics.Histogram;
-import com.codahale.metrics.MetricRegistry;
-import com.codahale.metrics.SlidingWindowReservoir;
-import com.codahale.metrics.Timer;
-
-import org.apache.hadoop.security.AccessControlException;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.ShutdownHookManager;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.api.records.QueueInfo;
-import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
-import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
-import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
-import org.apache.hadoop.yarn.server.resourcemanager.rmnode
-        .UpdatedContainerInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler
-        .ResourceScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler
-        .SchedulerAppReport;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler
-        .SchedulerNodeReport;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
-        .CapacityScheduler;
-
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event
-        .NodeUpdateSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event
-        .SchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event
-        .SchedulerEventType;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair
-        .FairScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo
-        .FifoScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
+import org.apache.hadoop.yarn.sls.SLSRunner;
+import org.apache.hadoop.yarn.sls.conf.SLSConfiguration;
+import org.apache.hadoop.yarn.sls.web.SLSWebApp;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.log4j.Logger;
 
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Set;
-import java.util.SortedMap;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.CsvReporter;
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.SlidingWindowReservoir;
+import com.codahale.metrics.Timer;
 
 @Private
 @Unstable
@@ -490,8 +466,9 @@ public class SLSCapacityScheduler extends CapacityScheduler implements
             TimeUnit.MILLISECONDS);
 
     // application running information
-    jobRuntimeLogBW = new BufferedWriter(
-            new FileWriter(metricsOutputDir + "/jobruntime.csv"));
+    jobRuntimeLogBW =
+        new BufferedWriter(new OutputStreamWriter(new FileOutputStream(
+            metricsOutputDir + "/jobruntime.csv"), "UTF-8"));
     jobRuntimeLogBW.write("JobID,real_start_time,real_end_time," +
             "simulate_start_time,simulate_end_time" + EOL);
     jobRuntimeLogBW.flush();
@@ -695,8 +672,9 @@ public class SLSCapacityScheduler extends CapacityScheduler implements
     private boolean firstLine = true;
     public MetricsLogRunnable() {
       try {
-        metricsLogBW = new BufferedWriter(
-                new FileWriter(metricsOutputDir + "/realtimetrack.json"));
+        metricsLogBW =
+            new BufferedWriter(new OutputStreamWriter(new FileOutputStream(
+                metricsOutputDir + "/realtimetrack.json"), "UTF-8"));
         metricsLogBW.write("[");
       } catch (IOException e) {
         e.printStackTrace();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f384a063/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/utils/SLSUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/utils/SLSUtils.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/utils/SLSUtils.java
index f62f024..922370c 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/utils/SLSUtils.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/utils/SLSUtils.java
@@ -17,6 +17,17 @@
  */
 package org.apache.hadoop.yarn.sls.utils;
 
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -28,16 +39,6 @@ import org.apache.hadoop.tools.rumen.LoggedTaskAttempt;
 import org.codehaus.jackson.JsonFactory;
 import org.codehaus.jackson.map.ObjectMapper;
 
-import java.io.File;
-import java.io.FileReader;
-import java.io.IOException;
-import java.io.Reader;
-import java.util.Set;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.List;
-import java.util.Iterator;
-
 @Private
 @Unstable
 public class SLSUtils {
@@ -89,7 +90,8 @@ public class SLSUtils {
     Set<String> nodeSet = new HashSet<String>();
     JsonFactory jsonF = new JsonFactory();
     ObjectMapper mapper = new ObjectMapper();
-    Reader input = new FileReader(jobTrace);
+    Reader input =
+        new InputStreamReader(new FileInputStream(jobTrace), "UTF-8");
     try {
       Iterator<Map> i = mapper.readValues(
               jsonF.createJsonParser(input), Map.class);
@@ -116,7 +118,8 @@ public class SLSUtils {
     Set<String> nodeSet = new HashSet<String>();
     JsonFactory jsonF = new JsonFactory();
     ObjectMapper mapper = new ObjectMapper();
-    Reader input = new FileReader(nodeFile);
+    Reader input =
+        new InputStreamReader(new FileInputStream(nodeFile), "UTF-8");
     try {
       Iterator<Map> i = mapper.readValues(
               jsonF.createJsonParser(input), Map.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f384a063/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/web/SLSWebApp.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/web/SLSWebApp.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/web/SLSWebApp.java
index 45301a1..e152696 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/web/SLSWebApp.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/web/SLSWebApp.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.sls.web;
 
 import java.io.File;
 import java.io.IOException;
+import java.io.ObjectInputStream;
 import java.text.MessageFormat;
 import java.util.HashMap;
 import java.util.Map;
@@ -32,24 +33,21 @@ import javax.servlet.http.HttpServletResponse;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event
-        .SchedulerEventType;
-import org.mortbay.jetty.Handler;
-import org.mortbay.jetty.Server;
-import org.mortbay.jetty.handler.AbstractHandler;
-import org.mortbay.jetty.Request;
-
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
 import org.apache.hadoop.yarn.sls.SLSRunner;
 import org.apache.hadoop.yarn.sls.scheduler.FairSchedulerMetrics;
-import org.apache.hadoop.yarn.sls.scheduler.ResourceSchedulerWrapper;
 import org.apache.hadoop.yarn.sls.scheduler.SchedulerMetrics;
 import org.apache.hadoop.yarn.sls.scheduler.SchedulerWrapper;
+import org.mortbay.jetty.Handler;
+import org.mortbay.jetty.Request;
+import org.mortbay.jetty.Server;
+import org.mortbay.jetty.handler.AbstractHandler;
+import org.mortbay.jetty.handler.ResourceHandler;
 
 import com.codahale.metrics.Counter;
 import com.codahale.metrics.Gauge;
 import com.codahale.metrics.Histogram;
 import com.codahale.metrics.MetricRegistry;
-import org.mortbay.jetty.handler.ResourceHandler;
 
 @Private
 @Unstable
@@ -71,9 +69,10 @@ public class SLSWebApp extends HttpServlet {
   private transient Gauge availableVCoresGauge;
   private transient Histogram allocateTimecostHistogram;
   private transient Histogram handleTimecostHistogram;
-  private Map<SchedulerEventType, Histogram> handleOperTimecostHistogramMap;
-  private Map<String, Counter> queueAllocatedMemoryCounterMap;
-  private Map<String, Counter> queueAllocatedVCoresCounterMap;
+  private transient Map<SchedulerEventType, Histogram>
+     handleOperTimecostHistogramMap;
+  private transient Map<String, Counter> queueAllocatedMemoryCounterMap;
+  private transient Map<String, Counter> queueAllocatedVCoresCounterMap;
   private int port;
   private int ajaxUpdateTimeMS = 1000;
   // html page templates
@@ -96,6 +95,14 @@ public class SLSWebApp extends HttpServlet {
     }
   }
 
+  private void readObject(ObjectInputStream in) throws IOException,
+      ClassNotFoundException {
+    in.defaultReadObject();
+    handleOperTimecostHistogramMap = new HashMap<>();
+    queueAllocatedMemoryCounterMap = new HashMap<>();
+    queueAllocatedVCoresCounterMap = new HashMap<>();
+  }
+
   public SLSWebApp(SchedulerWrapper wrapper, int metricsAddressPort) {
     this.wrapper = wrapper;
     metrics = wrapper.getMetrics();


[13/50] [abbrv] hadoop git commit: HADOOP-11357. Print information of the build enviornment in test-patch.sh (aw)

Posted by ji...@apache.org.
HADOOP-11357. Print information of the build enviornment in test-patch.sh (aw)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/8847777a
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8847777a
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8847777a

Branch: refs/heads/HDFS-7240
Commit: 8847777ae428dac52cf65cedd360e3702dd75e2f
Parents: 8b69c82
Author: Allen Wittenauer <aw...@apache.org>
Authored: Sun Apr 26 15:51:08 2015 -0700
Committer: Allen Wittenauer <aw...@apache.org>
Committed: Sun Apr 26 15:51:08 2015 -0700

----------------------------------------------------------------------
 dev-support/test-patch.sh                       | 17 +++++++++++++++++
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 +++
 2 files changed, 20 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8847777a/dev-support/test-patch.sh
----------------------------------------------------------------------
diff --git a/dev-support/test-patch.sh b/dev-support/test-patch.sh
index 2537e85..e331deb 100755
--- a/dev-support/test-patch.sh
+++ b/dev-support/test-patch.sh
@@ -234,6 +234,21 @@ function add_jira_table
   fi
 }
 
+## @description  Put the final environment information at the bottom
+## @description  of the footer table
+## @stability     stable
+## @audience     private
+## @replaceable  yes
+function close_jira_footer
+{
+  # shellcheck disable=SC2016
+  local -r javaversion=$("${JAVA_HOME}/bin/java" -version 2>&1 | head -1 | ${AWK} '{print $NF}' | tr -d \")
+  local -r unamea=$(uname -a)
+
+  add_jira_footer "Java" "${javaversion}"
+  add_jira_footer "uname" "${unamea}"
+}
+
 ## @description  Put the final elapsed time at the bottom of the table.
 ## @audience     private
 ## @stability    stable
@@ -2389,6 +2404,8 @@ postinstall
 
 runtests
 
+close_jira_footer
+
 close_jira_table
 
 output_to_console ${RESULT}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8847777a/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 5ba71a4..597496a 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -519,6 +519,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-10597. RPC Server signals backoff to clients when all request
     queues are full. (Ming Ma via Arpit Agarwal)
 
+    HADOOP-11357. Print information of the build enviornment in test-patch.sh
+    (aw)
+
   OPTIMIZATIONS
 
     HADOOP-11785. Reduce the number of listStatus operation in distcp


[20/50] [abbrv] hadoop git commit: HDFS-8205. Fix CHANGES.txt

Posted by ji...@apache.org.
HDFS-8205. Fix CHANGES.txt


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/6bae5962
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6bae5962
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6bae5962

Branch: refs/heads/HDFS-7240
Commit: 6bae5962cd70ac33fe599c50fb2a906830e5d4b2
Parents: 0d5b014
Author: Arpit Agarwal <ar...@apache.org>
Authored: Mon Apr 27 12:30:50 2015 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Mon Apr 27 12:30:50 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6bae5962/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 00b5db5..d56ea0c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -571,6 +571,9 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-8206. Fix the typos in hadoop-hdfs-httpfs. (Brahma Reddy Battula via xyao)
 
+    HDFS-8205. CommandFormat#parse() should not parse option as
+    value of option. (Peter Shi and Xiaoyu Yao via Arpit Agarwal)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -618,9 +621,6 @@ Release 2.7.1 - UNRELEASED
     HDFS-8070. Pre-HDFS-7915 DFSClient cannot use short circuit on
     post-HDFS-7915 DataNode (cmccabe)
 
-    HDFS-8205. CommandFormat#parse() should not parse option as
-    value of option. (Peter Shi and Xiaoyu Yao via Arpit Agarwal)
-
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES


[30/50] [abbrv] hadoop git commit: MAPREDUCE-6343. JobConf.parseMaximumHeapSizeMB() fails to parse value greater than 2GB expressed in bytes. (Hao Xia via kasha)

Posted by ji...@apache.org.
MAPREDUCE-6343. JobConf.parseMaximumHeapSizeMB() fails to parse value greater than 2GB expressed in bytes. (Hao Xia via kasha)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/51909232
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/51909232
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/51909232

Branch: refs/heads/HDFS-7240
Commit: 519092322dd1bf71984bef1393d8e082643408cd
Parents: bc1bd7e
Author: Karthik Kambatla <ka...@apache.org>
Authored: Tue Apr 28 14:05:26 2015 -0700
Committer: Karthik Kambatla <ka...@apache.org>
Committed: Tue Apr 28 14:05:26 2015 -0700

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                |  3 +++
 .../main/java/org/apache/hadoop/mapred/JobConf.java | 10 +++++-----
 .../java/org/apache/hadoop/mapred/TestJobConf.java  | 16 ++++++++++++++++
 3 files changed, 24 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/51909232/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 2090007..e9e1a4b 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -188,6 +188,9 @@ Trunk (Unreleased)
     MAPREDUCE-6234. TestHighRamJob fails due to the change in MAPREDUCE-5785. 
     (Masatake Iwasaki via kasha)
 
+    MAPREDUCE-6343. JobConf.parseMaximumHeapSizeMB() fails to parse value 
+    greater than 2GB expressed in bytes. (Hao Xia via kasha)
+
   BREAKDOWN OF MAPREDUCE-2841 (NATIVE TASK) SUBTASKS
 
     MAPREDUCE-5985. native-task: Fix build on macosx. Contributed by

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51909232/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java
index 9cac685..059593a 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java
@@ -2080,28 +2080,28 @@ public class JobConf extends Configuration {
     // Find the last matching -Xmx following word boundaries
     Matcher m = JAVA_OPTS_XMX_PATTERN.matcher(javaOpts);
     if (m.matches()) {
-      int size = Integer.parseInt(m.group(1));
+      long size = Long.parseLong(m.group(1));
       if (size <= 0) {
         return -1;
       }
       if (m.group(2).isEmpty()) {
         // -Xmx specified in bytes
-        return size / (1024 * 1024);
+        return (int) (size / (1024 * 1024));
       }
       char unit = m.group(2).charAt(0);
       switch (unit) {
         case 'g':
         case 'G':
           // -Xmx specified in GB
-          return size * 1024;
+          return (int) (size * 1024);
         case 'm':
         case 'M':
           // -Xmx specified in MB
-          return size;
+          return (int) size;
         case 'k':
         case 'K':
           // -Xmx specified in KB
-          return size / 1024;
+          return (int) (size / 1024);
       }
     }
     // -Xmx not specified

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51909232/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestJobConf.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestJobConf.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestJobConf.java
index 0612ade..57fa4e6 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestJobConf.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestJobConf.java
@@ -361,4 +361,20 @@ public class TestJobConf {
       jobConf.getMaxTaskFailuresPerTracker() < jobConf.getMaxReduceAttempts()
       );
   }
+
+  /**
+   * Test parsing various types of Java heap options.
+   */
+  @Test
+  public void testParseMaximumHeapSizeMB() {
+    // happy cases
+    Assert.assertEquals(4096, JobConf.parseMaximumHeapSizeMB("-Xmx4294967296"));
+    Assert.assertEquals(4096, JobConf.parseMaximumHeapSizeMB("-Xmx4194304k"));
+    Assert.assertEquals(4096, JobConf.parseMaximumHeapSizeMB("-Xmx4096m"));
+    Assert.assertEquals(4096, JobConf.parseMaximumHeapSizeMB("-Xmx4g"));
+
+    // sad cases
+    Assert.assertEquals(-1, JobConf.parseMaximumHeapSizeMB("-Xmx4?"));
+    Assert.assertEquals(-1, JobConf.parseMaximumHeapSizeMB(""));
+  }
 }


[28/50] [abbrv] hadoop git commit: HDFS-8204. Mover/Balancer should not schedule two replicas to the same datanode. Contributed by Walter Su

Posted by ji...@apache.org.
HDFS-8204. Mover/Balancer should not schedule two replicas to the same datanode.  Contributed by Walter Su


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/5639bf02
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5639bf02
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5639bf02

Branch: refs/heads/HDFS-7240
Commit: 5639bf02da716b3ecda785979b3d08cdca15972d
Parents: eccf709
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Tue Apr 28 13:05:33 2015 -0700
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Tue Apr 28 13:05:33 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../hadoop/hdfs/server/balancer/Dispatcher.java | 17 +++++-
 .../hdfs/server/balancer/TestBalancer.java      | 63 ++++++++++++++++++++
 3 files changed, 81 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5639bf02/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 326de0b..5d9e983 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -590,6 +590,9 @@ Release 2.7.1 - UNRELEASED
     HDFS-7931. DistributedFileSystem should not look for keyProvider in
     cache if Encryption is disabled (asuresh)
 
+    HDFS-8204. Mover/Balancer should not schedule two replicas to the same
+    datanode.  (Walter Su via szetszwo)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5639bf02/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
index a7a6c4a..5d02bf9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
@@ -971,6 +971,9 @@ public class Dispatcher {
    */
   private boolean isGoodBlockCandidate(StorageGroup source, StorageGroup target,
       StorageType targetStorageType, DBlock block) {
+    if (source.equals(target)) {
+      return false;
+    }
     if (target.storageType != targetStorageType) {
       return false;
     }
@@ -978,9 +981,19 @@ public class Dispatcher {
     if (movedBlocks.contains(block.getBlock())) {
       return false;
     }
-    if (block.isLocatedOn(target)) {
-      return false;
+    final DatanodeInfo targetDatanode = target.getDatanodeInfo();
+    if (source.getDatanodeInfo().equals(targetDatanode)) {
+      // the block is moved inside same DN
+      return true;
     }
+
+    // check if block has replica in target node
+    for (StorageGroup blockLocation : block.getLocations()) {
+      if (blockLocation.getDatanodeInfo().equals(targetDatanode)) {
+        return false;
+      }
+    }
+
     if (cluster.isNodeGroupAware()
         && isOnSameNodeGroupWithReplicas(source, target, block)) {
       return false;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5639bf02/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
index 9aea283..edffb82 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
@@ -1308,6 +1309,68 @@ public class TestBalancer {
   }
 
   /**
+   * Test special case. Two replicas belong to same block should not in same node.
+   * We have 2 nodes.
+   * We have a block in (DN0,SSD) and (DN1,DISK).
+   * Replica in (DN0,SSD) should not be moved to (DN1,SSD).
+   * Otherwise DN1 has 2 replicas.
+   */
+  @Test(timeout=100000)
+  public void testTwoReplicaShouldNotInSameDN() throws Exception {
+    final Configuration conf = new HdfsConfiguration();
+
+    int blockSize = 5 * 1024 * 1024 ;
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
+    conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
+    conf.setLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1L);
+
+    int numOfDatanodes =2;
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(2)
+        .racks(new String[]{"/default/rack0", "/default/rack0"})
+        .storagesPerDatanode(2)
+        .storageTypes(new StorageType[][]{
+            {StorageType.SSD, StorageType.DISK},
+            {StorageType.SSD, StorageType.DISK}})
+        .storageCapacities(new long[][]{
+            {100 * blockSize, 20 * blockSize},
+            {20 * blockSize, 100 * blockSize}})
+        .build();
+
+    try {
+      cluster.waitActive();
+
+      //set "/bar" directory with ONE_SSD storage policy.
+      DistributedFileSystem fs = cluster.getFileSystem();
+      Path barDir = new Path("/bar");
+      fs.mkdir(barDir,new FsPermission((short)777));
+      fs.setStoragePolicy(barDir, HdfsConstants.ONESSD_STORAGE_POLICY_NAME);
+
+      // Insert 30 blocks. So (DN0,SSD) and (DN1,DISK) are about half full,
+      // and (DN0,SSD) and (DN1,DISK) are about 15% full.
+      long fileLen  = 30 * blockSize;
+      // fooFile has ONE_SSD policy. So
+      // (DN0,SSD) and (DN1,DISK) have 2 replicas belong to same block.
+      // (DN0,DISK) and (DN1,SSD) have 2 replicas belong to same block.
+      Path fooFile = new Path(barDir, "foo");
+      createFile(cluster, fooFile, fileLen, (short) numOfDatanodes, 0);
+      // update space info
+      cluster.triggerHeartbeats();
+
+      Balancer.Parameters p = Balancer.Parameters.DEFAULT;
+      Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
+      final int r = Balancer.run(namenodes, p, conf);
+
+      // Replica in (DN0,SSD) was not moved to (DN1,SSD), because (DN1,DISK)
+      // already has one. Otherwise DN1 will have 2 replicas.
+      // For same reason, no replicas were moved.
+      assertEquals(ExitStatus.NO_MOVE_PROGRESS.getExitCode(), r);
+
+    } finally {
+      cluster.shutdown();
+    }
+  }
+  /**
    * @param args
    */
   public static void main(String[] args) throws Exception {


[50/50] [abbrv] hadoop git commit: YARN-2619. Added NodeManager support for disk io isolation through cgroups. Contributed by Varun Vasudev and Wei Yan.

Posted by ji...@apache.org.
YARN-2619. Added NodeManager support for disk io isolation through cgroups. Contributed by Varun Vasudev and Wei Yan.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/1b3b9e5c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1b3b9e5c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1b3b9e5c

Branch: refs/heads/HDFS-7240
Commit: 1b3b9e5c31c38388c1ce4208c65e8dd5f956da82
Parents: 98a6176
Author: Vinod Kumar Vavilapalli <vi...@apache.org>
Authored: Thu Apr 30 21:41:07 2015 -0700
Committer: Vinod Kumar Vavilapalli <vi...@apache.org>
Committed: Thu Apr 30 21:41:07 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../hadoop/yarn/conf/YarnConfiguration.java     |  54 ++++--
 .../CGroupsBlkioResourceHandlerImpl.java        | 170 +++++++++++++++++++
 .../linux/resources/CGroupsHandler.java         |   4 +-
 .../linux/resources/CGroupsHandlerImpl.java     |  67 ++++----
 .../linux/resources/DiskResourceHandler.java    |  30 ++++
 .../linux/resources/ResourceHandlerModule.java  |  45 ++++-
 .../util/CgroupsLCEResourcesHandler.java        |   6 +
 .../TestCGroupsBlkioResourceHandlerImpl.java    | 116 +++++++++++++
 .../linux/resources/TestCGroupsHandlerImpl.java | 101 +++++++++--
 .../resources/TestResourceHandlerModule.java    |  26 ++-
 .../util/TestCgroupsLCEResourcesHandler.java    | 112 +++++-------
 12 files changed, 600 insertions(+), 134 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b3b9e5c/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 28fcae4..dcf3538 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -105,6 +105,9 @@ Release 2.8.0 - UNRELEASED
     YARN-2498. Respect labels in preemption policy of capacity scheduler for
     inter-queue preemption. (Wangda Tan via jianhe)
 
+    YARN-2619. Added NodeManager support for disk io isolation through cgroups.
+    (Varun Vasudev and Wei Yan via vinodkv)
+
   IMPROVEMENTS
 
     YARN-1880. Cleanup TestApplicationClientProtocolOnHA

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b3b9e5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index eb568b9..70b87f3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -823,38 +823,68 @@ public class YarnConfiguration extends Configuration {
   public static final int DEFAULT_NM_RESOURCE_PERCENTAGE_PHYSICAL_CPU_LIMIT =
       100;
 
+  /**
+   * Prefix for disk configurations. Work in progress: This configuration
+   * parameter may be changed/removed in the future.
+   */
+  @Private
+  public static final String NM_DISK_RESOURCE_PREFIX = NM_PREFIX
+      + "resource.disk.";
+  /**
+   * This setting controls if resource handling for disk operations is enabled.
+   * Work in progress: This configuration parameter may be changed/removed in
+   * the future
+   */
+  @Private
+  public static final String NM_DISK_RESOURCE_ENABLED = NM_DISK_RESOURCE_PREFIX
+      + "enabled";
+  /** Disk as a resource is disabled by default. **/
+  @Private
+  public static final boolean DEFAULT_NM_DISK_RESOURCE_ENABLED = false;
 
-  public static final String NM_NETWORK_RESOURCE_PREFIX = NM_PREFIX + "resource.network.";
+  public static final String NM_NETWORK_RESOURCE_PREFIX = NM_PREFIX
+      + "resource.network.";
 
-  /** This setting controls if resource handling for network bandwidth is enabled **/
-  /* Work in progress: This configuration parameter may be changed/removed in the future */
+  /**
+   * This setting controls if resource handling for network bandwidth is
+   * enabled. Work in progress: This configuration parameter may be
+   * changed/removed in the future
+   */
   @Private
   public static final String NM_NETWORK_RESOURCE_ENABLED =
       NM_NETWORK_RESOURCE_PREFIX + "enabled";
-  /** Network as a resource is disabled by default **/
+  /** Network as a resource is disabled by default. **/
   @Private
   public static final boolean DEFAULT_NM_NETWORK_RESOURCE_ENABLED = false;
 
-  /** Specifies the interface to be used for applying network throttling rules **/
-  /* Work in progress: This configuration parameter may be changed/removed in the future */
+  /**
+   * Specifies the interface to be used for applying network throttling rules.
+   * Work in progress: This configuration parameter may be changed/removed in
+   * the future
+   */
   @Private
   public static final String NM_NETWORK_RESOURCE_INTERFACE =
       NM_NETWORK_RESOURCE_PREFIX + "interface";
   @Private
   public static final String DEFAULT_NM_NETWORK_RESOURCE_INTERFACE = "eth0";
 
-  /** Specifies the total available outbound bandwidth on the node **/
-  /* Work in progress: This configuration parameter may be changed/removed in the future */
+  /**
+   * Specifies the total available outbound bandwidth on the node. Work in
+   * progress: This configuration parameter may be changed/removed in the future
+   */
   @Private
   public static final String NM_NETWORK_RESOURCE_OUTBOUND_BANDWIDTH_MBIT =
       NM_NETWORK_RESOURCE_PREFIX + "outbound-bandwidth-mbit";
   @Private
-  public static final int DEFAULT_NM_NETWORK_RESOURCE_OUTBOUND_BANDWIDTH_MBIT = 1000;
+  public static final int DEFAULT_NM_NETWORK_RESOURCE_OUTBOUND_BANDWIDTH_MBIT =
+      1000;
 
-  /** Specifies the total outbound bandwidth available to YARN containers. defaults to
-   * NM_NETWORK_RESOURCE_OUTBOUND_BANDWIDTH_MBIT if not specified.
+  /**
+   * Specifies the total outbound bandwidth available to YARN containers.
+   * defaults to NM_NETWORK_RESOURCE_OUTBOUND_BANDWIDTH_MBIT if not specified.
+   * Work in progress: This configuration parameter may be changed/removed in
+   * the future
    */
-  /* Work in progress: This configuration parameter may be changed/removed in the future */
   @Private
   public static final String NM_NETWORK_RESOURCE_OUTBOUND_BANDWIDTH_YARN_MBIT =
       NM_NETWORK_RESOURCE_PREFIX + "outbound-bandwidth-yarn-mbit";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b3b9e5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsBlkioResourceHandlerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsBlkioResourceHandlerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsBlkioResourceHandlerImpl.java
new file mode 100644
index 0000000..e7eea1f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsBlkioResourceHandlerImpl.java
@@ -0,0 +1,170 @@
+/**
+ * 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.containermanager.linux.resources;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperation;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Handler class to handle the blkio controller. Currently it splits resources
+ * evenly across all containers. Once we have scheduling sorted out, we can
+ * modify the function to represent the disk resources allocated.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class CGroupsBlkioResourceHandlerImpl implements DiskResourceHandler {
+
+  static final Log LOG = LogFactory
+      .getLog(CGroupsBlkioResourceHandlerImpl.class);
+
+  private CGroupsHandler cGroupsHandler;
+  // Arbitrarily choose a weight - all that matters is that all containers
+  // get the same weight assigned to them. Once we have scheduling support
+  // this number will be determined dynamically for each container.
+  @VisibleForTesting
+  static final String DEFAULT_WEIGHT = "500";
+  private static final String PARTITIONS_FILE = "/proc/partitions";
+
+  CGroupsBlkioResourceHandlerImpl(CGroupsHandler cGroupsHandler) {
+    this.cGroupsHandler = cGroupsHandler;
+    // check for linux so that we don't print messages for tests running on
+    // other platforms
+    if(Shell.LINUX) {
+      checkDiskScheduler();
+    }
+  }
+
+
+  private void checkDiskScheduler() {
+    String data;
+
+    // read /proc/partitions and check to make sure that sd* and hd*
+    // are using the CFQ scheduler. If they aren't print a warning
+    try {
+      byte[] contents = Files.readAllBytes(Paths.get(PARTITIONS_FILE));
+      data = new String(contents, "UTF-8").trim();
+    } catch (IOException e) {
+      String msg = "Couldn't read " + PARTITIONS_FILE +
+          "; can't determine disk scheduler type";
+      LOG.warn(msg, e);
+      return;
+    }
+    String[] lines = data.split(System.lineSeparator());
+    if (lines.length > 0) {
+      for (String line : lines) {
+        String[] columns = line.split("\\s+");
+        if (columns.length > 4) {
+          String partition = columns[4];
+          // check some known partitions to make sure  the disk scheduler
+          // is cfq - not meant to be comprehensive, more a sanity check
+          if (partition.startsWith("sd") || partition.startsWith("hd")
+              || partition.startsWith("vd") || partition.startsWith("xvd")) {
+            String schedulerPath =
+                "/sys/block/" + partition + "/queue/scheduler";
+            File schedulerFile = new File(schedulerPath);
+            if (schedulerFile.exists()) {
+              try {
+                byte[] contents = Files.readAllBytes(Paths.get(schedulerPath));
+                String schedulerString = new String(contents, "UTF-8").trim();
+                if (!schedulerString.contains("[cfq]")) {
+                  LOG.warn("Device " + partition + " does not use the CFQ"
+                      + " scheduler; disk isolation using "
+                      + "CGroups will not work on this partition.");
+                }
+              } catch (IOException ie) {
+                LOG.warn(
+                    "Unable to determine disk scheduler type for partition "
+                      + partition, ie);
+              }
+            }
+          }
+        }
+      }
+    }
+  }
+
+  @Override
+  public List<PrivilegedOperation> bootstrap(Configuration configuration)
+      throws ResourceHandlerException {
+    // if bootstrap is called on this class, disk is already enabled
+    // so no need to check again
+    this.cGroupsHandler
+      .mountCGroupController(CGroupsHandler.CGroupController.BLKIO);
+    return null;
+  }
+
+  @Override
+  public List<PrivilegedOperation> preStart(Container container)
+      throws ResourceHandlerException {
+
+    String cgroupId = container.getContainerId().toString();
+    cGroupsHandler
+      .createCGroup(CGroupsHandler.CGroupController.BLKIO, cgroupId);
+    try {
+      cGroupsHandler.updateCGroupParam(CGroupsHandler.CGroupController.BLKIO,
+          cgroupId, CGroupsHandler.CGROUP_PARAM_BLKIO_WEIGHT, DEFAULT_WEIGHT);
+    } catch (ResourceHandlerException re) {
+      cGroupsHandler.deleteCGroup(CGroupsHandler.CGroupController.BLKIO,
+          cgroupId);
+      LOG.warn("Could not update cgroup for container", re);
+      throw re;
+    }
+    List<PrivilegedOperation> ret = new ArrayList<>();
+    ret.add(new PrivilegedOperation(
+      PrivilegedOperation.OperationType.ADD_PID_TO_CGROUP,
+      PrivilegedOperation.CGROUP_ARG_PREFIX
+          + cGroupsHandler.getPathForCGroupTasks(
+            CGroupsHandler.CGroupController.BLKIO, cgroupId)));
+    return ret;
+  }
+
+  @Override
+  public List<PrivilegedOperation> reacquireContainer(ContainerId containerId)
+      throws ResourceHandlerException {
+    return null;
+  }
+
+  @Override
+  public List<PrivilegedOperation> postComplete(ContainerId containerId)
+      throws ResourceHandlerException {
+    cGroupsHandler.deleteCGroup(CGroupsHandler.CGroupController.BLKIO,
+        containerId.toString());
+    return null;
+  }
+
+  @Override
+  public List<PrivilegedOperation> teardown() throws ResourceHandlerException {
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b3b9e5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandler.java
index 34429d3..70dc818 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandler.java
@@ -33,7 +33,8 @@ import org.apache.hadoop.classification.InterfaceStability;
 public interface CGroupsHandler {
   public enum CGroupController {
     CPU("cpu"),
-    NET_CLS("net_cls");
+    NET_CLS("net_cls"),
+    BLKIO("blkio");
 
     private final String name;
 
@@ -48,6 +49,7 @@ public interface CGroupsHandler {
 
   public static final String CGROUP_FILE_TASKS = "tasks";
   public static final String CGROUP_PARAM_CLASSID = "classid";
+  public static final String CGROUP_PARAM_BLKIO_WEIGHT = "weight";
 
   /**
    * Mounts a cgroup controller

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b3b9e5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandlerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandlerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandlerImpl.java
index 9a4230f..ff56121 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandlerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandlerImpl.java
@@ -20,6 +20,7 @@
 
 package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -38,6 +39,7 @@ import java.io.*;
 import java.nio.file.Files;
 import java.nio.file.Paths;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -63,7 +65,7 @@ class CGroupsHandlerImpl implements CGroupsHandler {
   private final String cGroupMountPath;
   private final long deleteCGroupTimeout;
   private final long deleteCGroupDelay;
-  private final Map<CGroupController, String> controllerPaths;
+  private Map<CGroupController, String> controllerPaths;
   private final ReadWriteLock rwLock;
   private final PrivilegedOperationExecutor privilegedOperationExecutor;
   private final Clock clock;
@@ -106,55 +108,61 @@ class CGroupsHandlerImpl implements CGroupsHandler {
 
   private void initializeControllerPaths() throws ResourceHandlerException {
     if (enableCGroupMount) {
-      //nothing to do here - we support 'deferred' mounting of specific
-      //controllers - we'll populate the path for a given controller when an
-      //explicit mountCGroupController request is issued.
+      // nothing to do here - we support 'deferred' mounting of specific
+      // controllers - we'll populate the path for a given controller when an
+      // explicit mountCGroupController request is issued.
       LOG.info("CGroup controller mounting enabled.");
     } else {
-      //cluster admins are expected to have mounted controllers in specific
-      //locations - we'll attempt to figure out mount points
-      initializeControllerPathsFromMtab();
+      // cluster admins are expected to have mounted controllers in specific
+      // locations - we'll attempt to figure out mount points
+
+      Map<CGroupController, String> cPaths =
+          initializeControllerPathsFromMtab(MTAB_FILE, this.cGroupPrefix);
+      // we want to do a bulk update without the paths changing concurrently
+      try {
+        rwLock.writeLock().lock();
+        controllerPaths = cPaths;
+      } finally {
+        rwLock.writeLock().unlock();
+      }
     }
   }
 
-  private void initializeControllerPathsFromMtab()
-      throws ResourceHandlerException {
+  @VisibleForTesting
+  static Map<CGroupController, String> initializeControllerPathsFromMtab(
+      String mtab, String cGroupPrefix) throws ResourceHandlerException {
     try {
-      Map<String, List<String>> parsedMtab = parseMtab();
-
-      //we want to do a bulk update without the paths changing concurrently
-      rwLock.writeLock().lock();
+      Map<String, List<String>> parsedMtab = parseMtab(mtab);
+      Map<CGroupController, String> ret = new HashMap<>();
 
       for (CGroupController controller : CGroupController.values()) {
         String name = controller.getName();
         String controllerPath = findControllerInMtab(name, parsedMtab);
 
         if (controllerPath != null) {
-          File f = new File(controllerPath + "/" + this.cGroupPrefix);
+          File f = new File(controllerPath + "/" + cGroupPrefix);
 
           if (FileUtil.canWrite(f)) {
-            controllerPaths.put(controller, controllerPath);
+            ret.put(controller, controllerPath);
           } else {
             String error =
                 new StringBuffer("Mount point Based on mtab file: ")
-                    .append(MTAB_FILE).append(
-                    ". Controller mount point not writable for: ")
-                    .append(name).toString();
+                  .append(mtab)
+                  .append(". Controller mount point not writable for: ")
+                  .append(name).toString();
 
             LOG.error(error);
             throw new ResourceHandlerException(error);
           }
         } else {
-
-            LOG.warn("Controller not mounted but automount disabled: " + name);
+          LOG.warn("Controller not mounted but automount disabled: " + name);
         }
       }
+      return ret;
     } catch (IOException e) {
       LOG.warn("Failed to initialize controller paths! Exception: " + e);
       throw new ResourceHandlerException(
-          "Failed to initialize controller paths!");
-    } finally {
-      rwLock.writeLock().unlock();
+        "Failed to initialize controller paths!");
     }
   }
 
@@ -173,12 +181,13 @@ class CGroupsHandlerImpl implements CGroupsHandler {
    * for mounts with type "cgroup". Cgroup controllers will
    * appear in the list of options for a path.
    */
-  private Map<String, List<String>> parseMtab() throws IOException {
+  private static Map<String, List<String>> parseMtab(String mtab)
+      throws IOException {
     Map<String, List<String>> ret = new HashMap<String, List<String>>();
     BufferedReader in = null;
 
     try {
-      FileInputStream fis = new FileInputStream(new File(getMtabFileName()));
+      FileInputStream fis = new FileInputStream(new File(mtab));
       in = new BufferedReader(new InputStreamReader(fis, "UTF-8"));
 
       for (String str = in.readLine(); str != null;
@@ -197,7 +206,7 @@ class CGroupsHandlerImpl implements CGroupsHandler {
         }
       }
     } catch (IOException e) {
-      throw new IOException("Error while reading " + getMtabFileName(), e);
+      throw new IOException("Error while reading " + mtab, e);
     } finally {
       IOUtils.cleanup(LOG, in);
     }
@@ -205,7 +214,7 @@ class CGroupsHandlerImpl implements CGroupsHandler {
     return ret;
   }
 
-  private String findControllerInMtab(String controller,
+  private static String findControllerInMtab(String controller,
       Map<String, List<String>> entries) {
     for (Map.Entry<String, List<String>> e : entries.entrySet()) {
       if (e.getValue().contains(controller))
@@ -215,10 +224,6 @@ class CGroupsHandlerImpl implements CGroupsHandler {
     return null;
   }
 
-  String getMtabFileName() {
-    return MTAB_FILE;
-  }
-
   @Override
   public void mountCGroupController(CGroupController controller)
       throws ResourceHandlerException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b3b9e5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/DiskResourceHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/DiskResourceHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/DiskResourceHandler.java
new file mode 100644
index 0000000..ca08d89
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/DiskResourceHandler.java
@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Resource handler for disk resources.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public interface DiskResourceHandler extends ResourceHandler {
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b3b9e5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/ResourceHandlerModule.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/ResourceHandlerModule.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/ResourceHandlerModule.java
index 30fc951..5dfd78c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/ResourceHandlerModule.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/ResourceHandlerModule.java
@@ -20,6 +20,7 @@
 
 package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
@@ -31,25 +32,27 @@ import java.util.List;
 
 /**
  * Provides mechanisms to get various resource handlers - cpu, memory, network,
- * disk etc., - based on configuration
+ * disk etc., - based on configuration.
  */
 
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 public class ResourceHandlerModule {
-  private volatile static ResourceHandlerChain resourceHandlerChain;
+  private static volatile ResourceHandlerChain resourceHandlerChain;
 
   /**
    * This specific implementation might provide resource management as well
    * as resource metrics functionality. We need to ensure that the same
    * instance is used for both.
    */
-  private volatile static TrafficControlBandwidthHandlerImpl
+  private static volatile TrafficControlBandwidthHandlerImpl
       trafficControlBandwidthHandler;
-  private volatile static CGroupsHandler cGroupsHandler;
+  private static volatile CGroupsHandler cGroupsHandler;
+  private static volatile CGroupsBlkioResourceHandlerImpl
+      cGroupsBlkioResourceHandler;
 
   /**
-   * Returns an initialized, thread-safe CGroupsHandler instance
+   * Returns an initialized, thread-safe CGroupsHandler instance.
    */
   public static CGroupsHandler getCGroupsHandler(Configuration conf)
       throws ResourceHandlerException {
@@ -94,6 +97,28 @@ public class ResourceHandlerModule {
     return getTrafficControlBandwidthHandler(conf);
   }
 
+  public static DiskResourceHandler getDiskResourceHandler(Configuration conf)
+      throws ResourceHandlerException {
+    if (conf.getBoolean(YarnConfiguration.NM_DISK_RESOURCE_ENABLED,
+        YarnConfiguration.DEFAULT_NM_DISK_RESOURCE_ENABLED)) {
+      return getCgroupsBlkioResourceHandler(conf);
+    }
+    return null;
+  }
+
+  private static CGroupsBlkioResourceHandlerImpl getCgroupsBlkioResourceHandler(
+      Configuration conf) throws ResourceHandlerException {
+    if (cGroupsBlkioResourceHandler == null) {
+      synchronized (DiskResourceHandler.class) {
+        if (cGroupsBlkioResourceHandler == null) {
+          cGroupsBlkioResourceHandler =
+              new CGroupsBlkioResourceHandlerImpl(getCGroupsHandler(conf));
+        }
+      }
+    }
+    return cGroupsBlkioResourceHandler;
+  }
+
   private static void addHandlerIfNotNull(List<ResourceHandler> handlerList,
       ResourceHandler handler) {
     if (handler != null) {
@@ -106,11 +131,12 @@ public class ResourceHandlerModule {
     ArrayList<ResourceHandler> handlerList = new ArrayList<>();
 
     addHandlerIfNotNull(handlerList, getOutboundBandwidthResourceHandler(conf));
+    addHandlerIfNotNull(handlerList, getDiskResourceHandler(conf));
     resourceHandlerChain = new ResourceHandlerChain(handlerList);
   }
 
-  public static ResourceHandlerChain getConfiguredResourceHandlerChain
-      (Configuration conf) throws ResourceHandlerException {
+  public static ResourceHandlerChain getConfiguredResourceHandlerChain(
+      Configuration conf) throws ResourceHandlerException {
     if (resourceHandlerChain == null) {
       synchronized (ResourceHandlerModule.class) {
         if (resourceHandlerChain == null) {
@@ -125,4 +151,9 @@ public class ResourceHandlerModule {
       return null;
     }
   }
+
+  @VisibleForTesting
+  static void nullifyResourceHandlerChain() throws ResourceHandlerException {
+    resourceHandlerChain = null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b3b9e5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/CgroupsLCEResourcesHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/CgroupsLCEResourcesHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/CgroupsLCEResourcesHandler.java
index ffa17ac..176b63c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/CgroupsLCEResourcesHandler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/CgroupsLCEResourcesHandler.java
@@ -30,6 +30,7 @@ import java.io.PrintWriter;
 import java.io.Writer;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -503,4 +504,9 @@ public class CgroupsLCEResourcesHandler implements LCEResourcesHandler {
   String getMtabFileName() {
     return MTAB_FILE;
   }
+
+  @VisibleForTesting
+  Map<String, String> getControllerPaths() {
+    return Collections.unmodifiableMap(controllerPaths);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b3b9e5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsBlkioResourceHandlerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsBlkioResourceHandlerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsBlkioResourceHandlerImpl.java
new file mode 100644
index 0000000..20aab69
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsBlkioResourceHandlerImpl.java
@@ -0,0 +1,116 @@
+/**
+ * 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.containermanager.linux.resources;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperation;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.List;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.times;
+
+/**
+ * Tests for the cgroups disk handler implementation.
+ */
+public class TestCGroupsBlkioResourceHandlerImpl {
+
+  private CGroupsHandler mockCGroupsHandler;
+  private CGroupsBlkioResourceHandlerImpl cGroupsBlkioResourceHandlerImpl;
+
+  @Before
+  public void setup() {
+    mockCGroupsHandler = mock(CGroupsHandler.class);
+    cGroupsBlkioResourceHandlerImpl =
+        new CGroupsBlkioResourceHandlerImpl(mockCGroupsHandler);
+  }
+
+  @Test
+  public void testBootstrap() throws Exception {
+    Configuration conf = new YarnConfiguration();
+    List<PrivilegedOperation> ret =
+        cGroupsBlkioResourceHandlerImpl.bootstrap(conf);
+    verify(mockCGroupsHandler, times(1)).mountCGroupController(
+        CGroupsHandler.CGroupController.BLKIO);
+    Assert.assertNull(ret);
+  }
+
+  @Test
+  public void testPreStart() throws Exception {
+    String id = "container_01_01";
+    String path = "test-path/" + id;
+    ContainerId mockContainerId = mock(ContainerId.class);
+    when(mockContainerId.toString()).thenReturn(id);
+    Container mockContainer = mock(Container.class);
+    when(mockContainer.getContainerId()).thenReturn(mockContainerId);
+    when(
+      mockCGroupsHandler.getPathForCGroupTasks(
+        CGroupsHandler.CGroupController.BLKIO, id)).thenReturn(path);
+
+    List<PrivilegedOperation> ret =
+        cGroupsBlkioResourceHandlerImpl.preStart(mockContainer);
+    verify(mockCGroupsHandler, times(1)).createCGroup(
+        CGroupsHandler.CGroupController.BLKIO, id);
+    verify(mockCGroupsHandler, times(1)).updateCGroupParam(
+        CGroupsHandler.CGroupController.BLKIO, id,
+        CGroupsHandler.CGROUP_PARAM_BLKIO_WEIGHT,
+        CGroupsBlkioResourceHandlerImpl.DEFAULT_WEIGHT);
+    Assert.assertNotNull(ret);
+    Assert.assertEquals(1, ret.size());
+    PrivilegedOperation op = ret.get(0);
+    Assert.assertEquals(PrivilegedOperation.OperationType.ADD_PID_TO_CGROUP,
+        op.getOperationType());
+    List<String> args = op.getArguments();
+    Assert.assertEquals(1, args.size());
+    Assert.assertEquals(PrivilegedOperation.CGROUP_ARG_PREFIX + path,
+        args.get(0));
+  }
+
+  @Test
+  public void testReacquireContainer() throws Exception {
+    ContainerId containerIdMock = mock(ContainerId.class);
+    Assert.assertNull(cGroupsBlkioResourceHandlerImpl
+        .reacquireContainer(containerIdMock));
+  }
+
+  @Test
+  public void testPostComplete() throws Exception {
+    String id = "container_01_01";
+    ContainerId mockContainerId = mock(ContainerId.class);
+    when(mockContainerId.toString()).thenReturn(id);
+    Assert.assertNull(cGroupsBlkioResourceHandlerImpl
+        .postComplete(mockContainerId));
+    verify(mockCGroupsHandler, times(1)).deleteCGroup(
+        CGroupsHandler.CGroupController.BLKIO, id);
+  }
+
+  @Test
+  public void testTeardown() throws Exception {
+    Assert.assertNull(cGroupsBlkioResourceHandlerImpl.teardown());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b3b9e5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsHandlerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsHandlerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsHandlerImpl.java
index 0717447..50f8da6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsHandlerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsHandlerImpl.java
@@ -20,6 +20,7 @@
 
 package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources;
 
+import org.apache.commons.io.FileUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -35,18 +36,21 @@ import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 
 import java.io.File;
+import java.io.FileWriter;
 import java.io.IOException;
 import java.nio.file.Files;
-import java.util.List;
 import java.util.Map;
+import java.util.UUID;
 
 import static org.mockito.Matchers.eq;
-import static org.mockito.Matchers.isNull;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.verifyNoMoreInteractions;
 import static org.mockito.Mockito.verifyZeroInteractions;
 
+/**
+ * Tests for the CGroups handler implementation.
+ */
 public class TestCGroupsHandlerImpl {
   private static final Log LOG =
       LogFactory.getLog(TestCGroupsHandlerImpl.class);
@@ -84,8 +88,8 @@ public class TestCGroupsHandlerImpl {
     try {
       cGroupsHandler = new CGroupsHandlerImpl(conf,
           privilegedOperationExecutorMock);
-      PrivilegedOperation expectedOp = new PrivilegedOperation
-          (PrivilegedOperation.OperationType.MOUNT_CGROUPS, (String) null);
+      PrivilegedOperation expectedOp = new PrivilegedOperation(
+          PrivilegedOperation.OperationType.MOUNT_CGROUPS, (String) null);
       //This is expected to be of the form :
       //net_cls=<mount_path>/net_cls
       StringBuffer controllerKV = new StringBuffer(controller.getName())
@@ -94,8 +98,8 @@ public class TestCGroupsHandlerImpl {
 
       cGroupsHandler.mountCGroupController(controller);
       try {
-        ArgumentCaptor<PrivilegedOperation> opCaptor = ArgumentCaptor.forClass
-            (PrivilegedOperation.class);
+        ArgumentCaptor<PrivilegedOperation> opCaptor = ArgumentCaptor.forClass(
+            PrivilegedOperation.class);
         verify(privilegedOperationExecutorMock)
             .executePrivilegedOperation(opCaptor.capture(), eq(false));
 
@@ -200,17 +204,15 @@ public class TestCGroupsHandlerImpl {
 
       Assert.assertTrue(paramFile.exists());
       try {
-        Assert.assertEquals(paramValue, new String(Files.readAllBytes
-            (paramFile
-                .toPath())));
+        Assert.assertEquals(paramValue, new String(Files.readAllBytes(
+            paramFile.toPath())));
       } catch (IOException e) {
         LOG.error("Caught exception: " + e);
-        Assert.assertTrue("Unexpected IOException trying to read cgroup param!",
-            false);
+        Assert.fail("Unexpected IOException trying to read cgroup param!");
       }
 
-      Assert.assertEquals(paramValue, cGroupsHandler.getCGroupParam
-          (controller, testCGroup, param));
+      Assert.assertEquals(paramValue,
+          cGroupsHandler.getCGroupParam(controller, testCGroup, param));
 
       //We can't really do a delete test here. Linux cgroups
       //implementation provides additional semantics - the cgroup cannot be
@@ -222,10 +224,77 @@ public class TestCGroupsHandlerImpl {
       //delete is not possible with a regular non-empty directory.
     } catch (ResourceHandlerException e) {
       LOG.error("Caught exception: " + e);
-      Assert.assertTrue(
-          "Unexpected ResourceHandlerException during cgroup operations!",
-          false);
+      Assert
+        .fail("Unexpected ResourceHandlerException during cgroup operations!");
+    }
+  }
+
+  public static File createMockCgroupMount(File parentDir, String type)
+      throws IOException {
+    return createMockCgroupMount(parentDir, type, "hadoop-yarn");
+  }
+
+  public static File createMockCgroupMount(File parentDir, String type,
+      String hierarchy) throws IOException {
+    File cgroupMountDir =
+        new File(parentDir.getAbsolutePath(), type + "/" + hierarchy);
+    FileUtils.deleteQuietly(cgroupMountDir);
+    if (!cgroupMountDir.mkdirs()) {
+      String message =
+          "Could not create dir " + cgroupMountDir.getAbsolutePath();
+      throw new IOException(message);
     }
+    return cgroupMountDir;
+  }
+
+  public static File createMockMTab(File parentDir) throws IOException {
+    String cpuMtabContent =
+        "none " + parentDir.getAbsolutePath()
+            + "/cpu cgroup rw,relatime,cpu 0 0\n";
+    String blkioMtabContent =
+        "none " + parentDir.getAbsolutePath()
+            + "/blkio cgroup rw,relatime,blkio 0 0\n";
+
+    File mockMtab = new File(parentDir, UUID.randomUUID().toString());
+    if (!mockMtab.exists()) {
+      if (!mockMtab.createNewFile()) {
+        String message = "Could not create file " + mockMtab.getAbsolutePath();
+        throw new IOException(message);
+      }
+    }
+    FileWriter mtabWriter = new FileWriter(mockMtab.getAbsoluteFile());
+    mtabWriter.write(cpuMtabContent);
+    mtabWriter.write(blkioMtabContent);
+    mtabWriter.close();
+    mockMtab.deleteOnExit();
+    return mockMtab;
+  }
+
+
+  @Test
+  public void testMtabParsing() throws Exception {
+    File parentDir = new File(tmpPath);
+    // create mock cgroup
+    File cpuCgroupMountDir = createMockCgroupMount(parentDir, "cpu",
+        hierarchy);
+    Assert.assertTrue(cpuCgroupMountDir.exists());
+    File blkioCgroupMountDir = createMockCgroupMount(parentDir,
+        "blkio", hierarchy);
+    Assert.assertTrue(blkioCgroupMountDir.exists());
+    File mockMtabFile = createMockMTab(parentDir);
+    Map<CGroupsHandler.CGroupController, String> controllerPaths =
+        CGroupsHandlerImpl.initializeControllerPathsFromMtab(
+          mockMtabFile.getAbsolutePath(), hierarchy);
+    Assert.assertEquals(2, controllerPaths.size());
+    Assert.assertTrue(controllerPaths
+        .containsKey(CGroupsHandler.CGroupController.CPU));
+    Assert.assertTrue(controllerPaths
+        .containsKey(CGroupsHandler.CGroupController.BLKIO));
+    String cpuDir = controllerPaths.get(CGroupsHandler.CGroupController.CPU);
+    String blkioDir =
+        controllerPaths.get(CGroupsHandler.CGroupController.BLKIO);
+    Assert.assertEquals(parentDir.getAbsolutePath() + "/cpu", cpuDir);
+    Assert.assertEquals(parentDir.getAbsolutePath() + "/blkio", blkioDir);
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b3b9e5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestResourceHandlerModule.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestResourceHandlerModule.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestResourceHandlerModule.java
index 939dfe7..69479d1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestResourceHandlerModule.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestResourceHandlerModule.java
@@ -37,7 +37,7 @@ public class TestResourceHandlerModule {
   Configuration networkEnabledConf;
 
   @Before
-  public void setup() {
+  public void setup() throws Exception {
     emptyConf = new YarnConfiguration();
     networkEnabledConf = new YarnConfiguration();
 
@@ -46,6 +46,7 @@ public class TestResourceHandlerModule {
     //We need to bypass mtab parsing for figuring out cgroups mount locations
     networkEnabledConf.setBoolean(YarnConfiguration
         .NM_LINUX_CONTAINER_CGROUPS_MOUNT, true);
+    ResourceHandlerModule.nullifyResourceHandlerChain();
   }
 
   @Test
@@ -75,4 +76,27 @@ public class TestResourceHandlerModule {
       Assert.fail("Unexpected ResourceHandlerException: " + e);
     }
   }
+
+  @Test
+  public void testDiskResourceHandler() throws Exception {
+
+    DiskResourceHandler handler =
+        ResourceHandlerModule.getDiskResourceHandler(emptyConf);
+    Assert.assertNull(handler);
+
+    Configuration diskConf = new YarnConfiguration();
+    diskConf.setBoolean(YarnConfiguration.NM_DISK_RESOURCE_ENABLED, true);
+
+    handler = ResourceHandlerModule.getDiskResourceHandler(diskConf);
+    Assert.assertNotNull(handler);
+
+    ResourceHandlerChain resourceHandlerChain =
+        ResourceHandlerModule.getConfiguredResourceHandlerChain(diskConf);
+    List<ResourceHandler> resourceHandlers =
+        resourceHandlerChain.getResourceHandlerList();
+    // Exactly one resource handler in chain
+    Assert.assertEquals(resourceHandlers.size(), 1);
+    // Same instance is expected to be in the chain.
+    Assert.assertTrue(resourceHandlers.get(0) == handler);
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b3b9e5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/util/TestCgroupsLCEResourcesHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/util/TestCgroupsLCEResourcesHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/util/TestCgroupsLCEResourcesHandler.java
index 4e35169..8e9d787 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/util/TestCgroupsLCEResourcesHandler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/util/TestCgroupsLCEResourcesHandler.java
@@ -21,6 +21,7 @@ import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.nodemanager.LinuxContainerExecutor;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.TestCGroupsHandlerImpl;
 import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin;
 import org.junit.Assert;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -33,7 +34,6 @@ import org.mockito.Mockito;
 import java.io.*;
 import java.util.List;
 import java.util.Scanner;
-import java.util.UUID;
 import java.util.concurrent.CountDownLatch;
 
 public class TestCgroupsLCEResourcesHandler {
@@ -142,7 +142,7 @@ public class TestCgroupsLCEResourcesHandler {
 
     @Override
     int[] getOverallLimits(float x) {
-      if (generateLimitsMode == true) {
+      if (generateLimitsMode) {
         return super.getOverallLimits(x);
       }
       return limits;
@@ -172,10 +172,11 @@ public class TestCgroupsLCEResourcesHandler {
     handler.initConfig();
 
     // create mock cgroup
-    File cgroupMountDir = createMockCgroupMount(cgroupDir);
+    File cpuCgroupMountDir = TestCGroupsHandlerImpl.createMockCgroupMount(
+        cgroupDir, "cpu");
 
     // create mock mtab
-    File mockMtab = createMockMTab(cgroupDir);
+    File mockMtab = TestCGroupsHandlerImpl.createMockMTab(cgroupDir);
 
     // setup our handler and call init()
     handler.setMtabFile(mockMtab.getAbsolutePath());
@@ -184,8 +185,8 @@ public class TestCgroupsLCEResourcesHandler {
     // in this case, we're using all cpu so the files
     // shouldn't exist(because init won't create them
     handler.init(mockLCE, plugin);
-    File periodFile = new File(cgroupMountDir, "cpu.cfs_period_us");
-    File quotaFile = new File(cgroupMountDir, "cpu.cfs_quota_us");
+    File periodFile = new File(cpuCgroupMountDir, "cpu.cfs_period_us");
+    File quotaFile = new File(cpuCgroupMountDir, "cpu.cfs_quota_us");
     Assert.assertFalse(periodFile.exists());
     Assert.assertFalse(quotaFile.exists());
 
@@ -202,7 +203,7 @@ public class TestCgroupsLCEResourcesHandler {
 
     // set cpu back to 100, quota should be -1
     conf.setInt(YarnConfiguration.NM_RESOURCE_PERCENTAGE_PHYSICAL_CPU_LIMIT,
-      100);
+        100);
     handler.limits[0] = 100 * 1000;
     handler.limits[1] = 1000 * 1000;
     handler.init(mockLCE, plugin);
@@ -235,7 +236,7 @@ public class TestCgroupsLCEResourcesHandler {
     Assert.assertEquals(expectedQuota, ret[0]);
     Assert.assertEquals(-1, ret[1]);
 
-    int[] params = { 0, -1 };
+    int[] params = {0, -1};
     for (int cores : params) {
       try {
         handler.getOverallLimits(cores);
@@ -251,34 +252,6 @@ public class TestCgroupsLCEResourcesHandler {
     Assert.assertEquals(-1, ret[1]);
   }
 
-  private File createMockCgroupMount(File cgroupDir) throws IOException {
-    File cgroupMountDir = new File(cgroupDir.getAbsolutePath(), "hadoop-yarn");
-    FileUtils.deleteQuietly(cgroupDir);
-    if (!cgroupMountDir.mkdirs()) {
-      String message =
-          "Could not create dir " + cgroupMountDir.getAbsolutePath();
-      throw new IOException(message);
-    }
-    return cgroupMountDir;
-  }
-
-  private File createMockMTab(File cgroupDir) throws IOException {
-    String mtabContent =
-        "none " + cgroupDir.getAbsolutePath() + " cgroup rw,relatime,cpu 0 0";
-    File mockMtab = new File("target", UUID.randomUUID().toString());
-    if (!mockMtab.exists()) {
-      if (!mockMtab.createNewFile()) {
-        String message = "Could not create file " + mockMtab.getAbsolutePath();
-        throw new IOException(message);
-      }
-    }
-    FileWriter mtabWriter = new FileWriter(mockMtab.getAbsoluteFile());
-    mtabWriter.write(mtabContent);
-    mtabWriter.close();
-    mockMtab.deleteOnExit();
-    return mockMtab;
-  }
-
   @Test
   public void testContainerLimits() throws IOException {
     LinuxContainerExecutor mockLCE = new MockLinuxContainerExecutor();
@@ -286,6 +259,7 @@ public class TestCgroupsLCEResourcesHandler {
         new CustomCgroupsLCEResourceHandler();
     handler.generateLimitsMode = true;
     YarnConfiguration conf = new YarnConfiguration();
+    conf.setBoolean(YarnConfiguration.NM_DISK_RESOURCE_ENABLED, true);
     final int numProcessors = 4;
     ResourceCalculatorPlugin plugin =
         Mockito.mock(ResourceCalculatorPlugin.class);
@@ -294,71 +268,77 @@ public class TestCgroupsLCEResourcesHandler {
     handler.initConfig();
 
     // create mock cgroup
-    File cgroupMountDir = createMockCgroupMount(cgroupDir);
+    File cpuCgroupMountDir = TestCGroupsHandlerImpl.createMockCgroupMount(
+        cgroupDir, "cpu");
 
     // create mock mtab
-    File mockMtab = createMockMTab(cgroupDir);
+    File mockMtab = TestCGroupsHandlerImpl.createMockMTab(cgroupDir);
 
     // setup our handler and call init()
     handler.setMtabFile(mockMtab.getAbsolutePath());
     handler.init(mockLCE, plugin);
 
-    // check values
-    // default case - files shouldn't exist, strict mode off by default
+    // check the controller paths map isn't empty
     ContainerId id = ContainerId.fromString("container_1_1_1_1");
     handler.preExecute(id, Resource.newInstance(1024, 1));
-    File containerDir = new File(cgroupMountDir, id.toString());
-    Assert.assertTrue(containerDir.exists());
-    Assert.assertTrue(containerDir.isDirectory());
-    File periodFile = new File(containerDir, "cpu.cfs_period_us");
-    File quotaFile = new File(containerDir, "cpu.cfs_quota_us");
+    Assert.assertNotNull(handler.getControllerPaths());
+    // check values
+    // default case - files shouldn't exist, strict mode off by default
+    File containerCpuDir = new File(cpuCgroupMountDir, id.toString());
+    Assert.assertTrue(containerCpuDir.exists());
+    Assert.assertTrue(containerCpuDir.isDirectory());
+    File periodFile = new File(containerCpuDir, "cpu.cfs_period_us");
+    File quotaFile = new File(containerCpuDir, "cpu.cfs_quota_us");
     Assert.assertFalse(periodFile.exists());
     Assert.assertFalse(quotaFile.exists());
 
     // no files created because we're using all cpu
-    FileUtils.deleteQuietly(containerDir);
+    FileUtils.deleteQuietly(containerCpuDir);
     conf.setBoolean(
-      YarnConfiguration.NM_LINUX_CONTAINER_CGROUPS_STRICT_RESOURCE_USAGE, true);
+        YarnConfiguration.NM_LINUX_CONTAINER_CGROUPS_STRICT_RESOURCE_USAGE,
+        true);
     handler.initConfig();
     handler.preExecute(id,
-      Resource.newInstance(1024, YarnConfiguration.DEFAULT_NM_VCORES));
-    Assert.assertTrue(containerDir.exists());
-    Assert.assertTrue(containerDir.isDirectory());
-    periodFile = new File(containerDir, "cpu.cfs_period_us");
-    quotaFile = new File(containerDir, "cpu.cfs_quota_us");
+        Resource.newInstance(1024, YarnConfiguration.DEFAULT_NM_VCORES));
+    Assert.assertTrue(containerCpuDir.exists());
+    Assert.assertTrue(containerCpuDir.isDirectory());
+    periodFile = new File(containerCpuDir, "cpu.cfs_period_us");
+    quotaFile = new File(containerCpuDir, "cpu.cfs_quota_us");
     Assert.assertFalse(periodFile.exists());
     Assert.assertFalse(quotaFile.exists());
 
     // 50% of CPU
-    FileUtils.deleteQuietly(containerDir);
+    FileUtils.deleteQuietly(containerCpuDir);
     conf.setBoolean(
-      YarnConfiguration.NM_LINUX_CONTAINER_CGROUPS_STRICT_RESOURCE_USAGE, true);
+        YarnConfiguration.NM_LINUX_CONTAINER_CGROUPS_STRICT_RESOURCE_USAGE,
+        true);
     handler.initConfig();
     handler.preExecute(id,
-      Resource.newInstance(1024, YarnConfiguration.DEFAULT_NM_VCORES / 2));
-    Assert.assertTrue(containerDir.exists());
-    Assert.assertTrue(containerDir.isDirectory());
-    periodFile = new File(containerDir, "cpu.cfs_period_us");
-    quotaFile = new File(containerDir, "cpu.cfs_quota_us");
+        Resource.newInstance(1024, YarnConfiguration.DEFAULT_NM_VCORES / 2));
+    Assert.assertTrue(containerCpuDir.exists());
+    Assert.assertTrue(containerCpuDir.isDirectory());
+    periodFile = new File(containerCpuDir, "cpu.cfs_period_us");
+    quotaFile = new File(containerCpuDir, "cpu.cfs_quota_us");
     Assert.assertTrue(periodFile.exists());
     Assert.assertTrue(quotaFile.exists());
     Assert.assertEquals(500 * 1000, readIntFromFile(periodFile));
     Assert.assertEquals(1000 * 1000, readIntFromFile(quotaFile));
 
     // CGroups set to 50% of CPU, container set to 50% of YARN CPU
-    FileUtils.deleteQuietly(containerDir);
+    FileUtils.deleteQuietly(containerCpuDir);
     conf.setBoolean(
-      YarnConfiguration.NM_LINUX_CONTAINER_CGROUPS_STRICT_RESOURCE_USAGE, true);
+        YarnConfiguration.NM_LINUX_CONTAINER_CGROUPS_STRICT_RESOURCE_USAGE,
+        true);
     conf
       .setInt(YarnConfiguration.NM_RESOURCE_PERCENTAGE_PHYSICAL_CPU_LIMIT, 50);
     handler.initConfig();
     handler.init(mockLCE, plugin);
     handler.preExecute(id,
-      Resource.newInstance(1024, YarnConfiguration.DEFAULT_NM_VCORES / 2));
-    Assert.assertTrue(containerDir.exists());
-    Assert.assertTrue(containerDir.isDirectory());
-    periodFile = new File(containerDir, "cpu.cfs_period_us");
-    quotaFile = new File(containerDir, "cpu.cfs_quota_us");
+        Resource.newInstance(1024, YarnConfiguration.DEFAULT_NM_VCORES / 2));
+    Assert.assertTrue(containerCpuDir.exists());
+    Assert.assertTrue(containerCpuDir.isDirectory());
+    periodFile = new File(containerCpuDir, "cpu.cfs_period_us");
+    quotaFile = new File(containerCpuDir, "cpu.cfs_quota_us");
     Assert.assertTrue(periodFile.exists());
     Assert.assertTrue(quotaFile.exists());
     Assert.assertEquals(1000 * 1000, readIntFromFile(periodFile));


[44/50] [abbrv] hadoop git commit: YARN-3544. Got back AM logs link on the RM web UI for a completed app. Contributed by Xuan Gong.

Posted by ji...@apache.org.
YARN-3544. Got back AM logs link on the RM web UI for a completed app. Contributed by Xuan Gong.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/7e8639fd
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7e8639fd
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7e8639fd

Branch: refs/heads/HDFS-7240
Commit: 7e8639fda40c13fe163128d7a725fcd0f2fce3c5
Parents: e2e8f77
Author: Zhijie Shen <zj...@apache.org>
Authored: Wed Apr 29 17:12:52 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Apr 30 13:05:55 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../resourcemanager/webapp/RMAppBlock.java      | 83 +++++++-------------
 2 files changed, 30 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8639fd/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index cf3d910..28fcae4 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -321,6 +321,9 @@ Release 2.7.1 - UNRELEASED
     YARN-3485. FairScheduler headroom calculation doesn't consider 
     maxResources for Fifo and FairShare policies. (kasha)
 
+    YARN-3544. Got back AM logs link on the RM web UI for a completed app.
+    (Xuan Gong via zjshen)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e8639fd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.java
index 3779b91..43e26be 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.java
@@ -25,28 +25,24 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerReport;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.exceptions.ContainerNotFoundException;
 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.RMAppMetrics;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptMetrics;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppAttemptInfo;
 import org.apache.hadoop.yarn.server.webapp.AppBlock;
-import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo;
-import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo;
-import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.hadoop.yarn.util.Times;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV;
+import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 import org.apache.hadoop.yarn.webapp.view.InfoBlock;
 
 import com.google.inject.Inject;
 
-import java.security.PrivilegedExceptionAction;
 import java.util.Collection;
 import java.util.Set;
 
@@ -54,11 +50,13 @@ public class RMAppBlock extends AppBlock{
 
   private static final Log LOG = LogFactory.getLog(RMAppBlock.class);
   private final ResourceManager rm;
+  private final Configuration conf;
 
 
   @Inject
   RMAppBlock(ViewContext ctx, Configuration conf, ResourceManager rm) {
     super(rm.getClientRMService(), ctx, conf);
+    this.conf = conf;
     this.rm = rm;
   }
 
@@ -120,65 +118,38 @@ public class RMAppBlock extends AppBlock{
             .th(".started", "Started").th(".node", "Node").th(".logs", "Logs")
             .th(".blacklistednodes", "Blacklisted Nodes")._()._().tbody();
 
+    RMApp rmApp = this.rm.getRMContext().getRMApps().get(this.appID);
+    if (rmApp == null) {
+      return;
+    }
     StringBuilder attemptsTableData = new StringBuilder("[\n");
     for (final ApplicationAttemptReport appAttemptReport : attempts) {
-      AppAttemptInfo appAttempt = new AppAttemptInfo(appAttemptReport);
-      ContainerReport containerReport = null;
-      try {
-        // AM container is always the first container of the attempt
-        final GetContainerReportRequest request =
-            GetContainerReportRequest.newInstance(ContainerId.newContainerId(
-                appAttemptReport.getApplicationAttemptId(), 1));
-        if (callerUGI == null) {
-          containerReport =
-              appBaseProt.getContainerReport(request).getContainerReport();
-        } else {
-          containerReport = callerUGI.doAs(
-              new PrivilegedExceptionAction<ContainerReport>() {
-                @Override
-                public ContainerReport run() throws Exception {
-                  ContainerReport report = null;
-                  try {
-                    report = appBaseProt.getContainerReport(request)
-                        .getContainerReport();
-                  } catch (ContainerNotFoundException ex) {
-                    LOG.warn(ex.getMessage());
-                  }
-                  return report;
-                }
-              });
-        }
-      } catch (Exception e) {
-        String message =
-            "Failed to read the AM container of the application attempt "
-                + appAttemptReport.getApplicationAttemptId() + ".";
-        LOG.error(message, e);
-        html.p()._(message)._();
-        return;
-      }
-      long startTime = 0L;
-      String logsLink = null;
-      String nodeLink = null;
-      if (containerReport != null) {
-        ContainerInfo container = new ContainerInfo(containerReport);
-        startTime = container.getStartedTime();
-        logsLink = containerReport.getLogUrl();
-        nodeLink = containerReport.getNodeHttpAddress();
+      RMAppAttempt rmAppAttempt =
+          rmApp.getRMAppAttempt(appAttemptReport.getApplicationAttemptId());
+      if (rmAppAttempt == null) {
+        continue;
       }
+      AppAttemptInfo attemptInfo =
+          new AppAttemptInfo(this.rm, rmAppAttempt, rmApp.getUser());
       String blacklistedNodesCount = "N/A";
-      Set<String> nodes = RMAppAttemptBlock.getBlacklistedNodes(rm,
-          ConverterUtils.toApplicationAttemptId(appAttempt.getAppAttemptId()));
+      Set<String> nodes =
+          RMAppAttemptBlock.getBlacklistedNodes(rm,
+            rmAppAttempt.getAppAttemptId());
       if(nodes != null) {
         blacklistedNodesCount = String.valueOf(nodes.size());
       }
-
+      String nodeLink = attemptInfo.getNodeHttpAddress();
+      if (nodeLink != null) {
+        nodeLink = WebAppUtils.getHttpSchemePrefix(conf) + nodeLink;
+      }
+      String logsLink = attemptInfo.getLogsLink();
       attemptsTableData
           .append("[\"<a href='")
-          .append(url("appattempt", appAttempt.getAppAttemptId()))
+          .append(url("appattempt", rmAppAttempt.getAppAttemptId().toString()))
           .append("'>")
-          .append(appAttempt.getAppAttemptId())
+          .append(String.valueOf(rmAppAttempt.getAppAttemptId()))
           .append("</a>\",\"")
-          .append(startTime)
+          .append(attemptInfo.getStartTime())
           .append("\",\"<a ")
           .append(nodeLink == null ? "#" : "href='" + nodeLink)
           .append("'>")


[05/50] [abbrv] hadoop git commit: Moving YARN-3351, YARN-3382, YARN-3472, MAPREDUCE-6238 to the 2.7.1 CHANGES.txt sections given the recent merge into branch-2.7.

Posted by ji...@apache.org.
Moving YARN-3351, YARN-3382, YARN-3472, MAPREDUCE-6238 to the 2.7.1 CHANGES.txt
sections given the recent merge into branch-2.7.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/2f82ae04
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2f82ae04
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2f82ae04

Branch: refs/heads/HDFS-7240
Commit: 2f82ae042a6f3110742aaa57c076bb9ebd7888d1
Parents: d497f6e
Author: Vinod Kumar Vavilapalli <vi...@apache.org>
Authored: Fri Apr 24 17:18:46 2015 -0700
Committer: Vinod Kumar Vavilapalli <vi...@apache.org>
Committed: Fri Apr 24 17:18:46 2015 -0700

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt |  6 +++---
 hadoop-yarn-project/CHANGES.txt      | 17 +++++++++--------
 2 files changed, 12 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f82ae04/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index e5acd1e..5b26910 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -334,9 +334,6 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6266. Job#getTrackingURL should consistently return a proper URL
     (rchiang via rkanter)
 
-    MAPREDUCE-6238. MR2 can't run local jobs with -libjars command options
-    which is a regression from MR1 (zxu via rkanter)
-
     MAPREDUCE-6293. Set job classloader on uber-job's LocalContainerLauncher
     event thread. (Sangjin Lee via gera)
 
@@ -360,6 +357,9 @@ Release 2.7.1 - UNRELEASED
 
     MAPREDUCE-6300. Task list sort by task id broken. (Siqi Li via aajisaka)
 
+    MAPREDUCE-6238. MR2 can't run local jobs with -libjars command options
+    which is a regression from MR1 (zxu via rkanter)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f82ae04/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index a830771..a626f82 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -193,8 +193,6 @@ Release 2.8.0 - UNRELEASED
     YARN-3205 FileSystemRMStateStore should disable FileSystem Cache to avoid
     get a Filesystem with an old configuration. (Zhihai Xu via ozawa)
 
-    YARN-3351. AppMaster tracking URL is broken in HA. (Anubhav Dhoot via kasha)
-
     YARN-3269. Yarn.nodemanager.remote-app-log-dir could not be configured to 
     fully qualified path. (Xuan Gong via junping_du)
 
@@ -238,12 +236,6 @@ Release 2.8.0 - UNRELEASED
     YARN-3465. Use LinkedHashMap to preserve order of resource requests. 
     (Zhihai Xu via kasha)
 
-    YARN-3382. Some of UserMetricsInfo metrics are incorrectly set to root
-    queue metrics. (Rohit Agarwal via jianhe)
-
-    YARN-3472. Fixed possible leak in DelegationTokenRenewer#allTokens.
-    (Rohith Sharmaks via jianhe)
-
     YARN-3266. RMContext#inactiveNodes should have NodeId as map key.
     (Chengbing Liu via jianhe)
 
@@ -287,6 +279,7 @@ Release 2.7.1 - UNRELEASED
   OPTIMIZATIONS
 
   BUG FIXES
+
     YARN-3487. CapacityScheduler scheduler lock obtained unnecessarily when 
     calling getQueue (Jason Lowe via wangda)
 
@@ -299,6 +292,14 @@ Release 2.7.1 - UNRELEASED
     YARN-3522. Fixed DistributedShell to instantiate TimeLineClient as the
     correct user. (Zhijie Shen via jianhe)
 
+    YARN-3351. AppMaster tracking URL is broken in HA. (Anubhav Dhoot via kasha)
+
+    YARN-3382. Some of UserMetricsInfo metrics are incorrectly set to root
+    queue metrics. (Rohit Agarwal via jianhe)
+
+    YARN-3472. Fixed possible leak in DelegationTokenRenewer#allTokens.
+    (Rohith Sharmaks via jianhe)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES


[29/50] [abbrv] hadoop git commit: MAPREDUCE-6334. Fetcher#copyMapOutput is leaking usedMemory upon IOException during InMemoryMapOutput shuffle handler. Contributed by Eric Payne

Posted by ji...@apache.org.
MAPREDUCE-6334. Fetcher#copyMapOutput is leaking usedMemory upon IOException during InMemoryMapOutput shuffle handler. Contributed by Eric Payne


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/bc1bd7e5
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/bc1bd7e5
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/bc1bd7e5

Branch: refs/heads/HDFS-7240
Commit: bc1bd7e5c4047b374420683d36a8c30eda6d75b6
Parents: 5639bf0
Author: Jason Lowe <jl...@apache.org>
Authored: Tue Apr 28 20:17:52 2015 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Tue Apr 28 20:19:05 2015 +0000

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |  3 ++
 .../hadoop/mapreduce/task/reduce/Fetcher.java   |  6 ++--
 .../mapreduce/task/reduce/TestFetcher.java      | 34 ++++++++++++++++++++
 3 files changed, 41 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc1bd7e5/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index d27a022..2090007 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -376,6 +376,9 @@ Release 2.7.1 - UNRELEASED
     MAPREDUCE-6252. JobHistoryServer should not fail when encountering a 
     missing directory. (Craig Welch via devaraj)
 
+    MAPREDUCE-6334. Fetcher#copyMapOutput is leaking usedMemory upon
+    IOException during InMemoryMapOutput shuffle handler (Eric Payne via jlowe)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc1bd7e5/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Fetcher.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Fetcher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Fetcher.java
index d867e4b..4b80dc9 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Fetcher.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Fetcher.java
@@ -553,7 +553,10 @@ class Fetcher<K,V> extends Thread {
       metrics.successFetch();
       return null;
     } catch (IOException ioe) {
-      
+      if (mapOutput != null) {
+        mapOutput.abort();
+      }
+
       if (canRetry) {
         checkTimeoutOrRetry(host, ioe);
       } 
@@ -574,7 +577,6 @@ class Fetcher<K,V> extends Thread {
                " from " + host.getHostName(), ioe); 
 
       // Inform the shuffle-scheduler
-      mapOutput.abort();
       metrics.failedFetch();
       return new TaskAttemptID[] {mapId};
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc1bd7e5/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestFetcher.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestFetcher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestFetcher.java
index 723df17..a9cd33e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestFetcher.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestFetcher.java
@@ -628,6 +628,40 @@ public class TestFetcher {
     verify(odmo).abort();
   }
 
+  @SuppressWarnings("unchecked")
+  @Test(timeout=10000)
+  public void testCopyFromHostWithRetryUnreserve() throws Exception {
+    InMemoryMapOutput<Text, Text> immo = mock(InMemoryMapOutput.class);
+    Fetcher<Text,Text> underTest = new FakeFetcher<Text,Text>(jobWithRetry,
+        id, ss, mm, r, metrics, except, key, connection);
+
+    String replyHash = SecureShuffleUtils.generateHash(encHash.getBytes(), key);
+
+    when(connection.getResponseCode()).thenReturn(200);
+    when(connection.getHeaderField(SecureShuffleUtils.HTTP_HEADER_REPLY_URL_HASH))
+        .thenReturn(replyHash);
+    ShuffleHeader header = new ShuffleHeader(map1ID.toString(), 10, 10, 1);
+    ByteArrayOutputStream bout = new ByteArrayOutputStream();
+    header.write(new DataOutputStream(bout));
+    ByteArrayInputStream in = new ByteArrayInputStream(bout.toByteArray());
+    when(connection.getInputStream()).thenReturn(in);
+    when(connection.getHeaderField(ShuffleHeader.HTTP_HEADER_NAME))
+        .thenReturn(ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
+    when(connection.getHeaderField(ShuffleHeader.HTTP_HEADER_VERSION))
+        .thenReturn(ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);
+
+    // Verify that unreserve occurs if an exception happens after shuffle
+    // buffer is reserved.
+    when(mm.reserve(any(TaskAttemptID.class), anyLong(), anyInt()))
+        .thenReturn(immo);
+    doThrow(new IOException("forced error")).when(immo).shuffle(
+        any(MapHost.class), any(InputStream.class), anyLong(),
+        anyLong(), any(ShuffleClientMetrics.class), any(Reporter.class));
+
+    underTest.copyFromHost(host);
+    verify(immo).abort();
+  }
+
   public static class FakeFetcher<K,V> extends Fetcher<K,V> {
 
     // If connection need to be reopen.


[09/50] [abbrv] hadoop git commit: HDFS-8206. Fix the typos in hadoop-hdfs-httpfs. (Brahma Reddy Battula via xyao)

Posted by ji...@apache.org.
HDFS-8206. Fix the typos in hadoop-hdfs-httpfs. (Brahma Reddy Battula via xyao)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/8f3946cd
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8f3946cd
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8f3946cd

Branch: refs/heads/HDFS-7240
Commit: 8f3946cd4013eaeaafbaf7d038f3920f74c8457e
Parents: a00e001
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Sat Apr 25 21:41:35 2015 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Sat Apr 25 21:41:35 2015 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/fs/http/client/HttpFSFileSystem.java     | 6 +++---
 .../hadoop/fs/http/server/CheckUploadContentTypeFilter.java    | 4 ++--
 .../java/org/apache/hadoop/fs/http/server/HttpFSServer.java    | 2 +-
 .../org/apache/hadoop/lib/servlet/FileSystemReleaseFilter.java | 4 ++--
 .../java/org/apache/hadoop/lib/servlet/HostnameFilter.java     | 4 ++--
 .../src/main/java/org/apache/hadoop/lib/servlet/MDCFilter.java | 4 ++--
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                    | 2 ++
 7 files changed, 14 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8f3946cd/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
index e797d12..3a6ce7d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
@@ -238,7 +238,7 @@ public class HttpFSFileSystem extends FileSystem
    * @return a <code>HttpURLConnection</code> for the HttpFSServer server,
    *         authenticated and ready to use for the specified path and file system operation.
    *
-   * @throws IOException thrown if an IO error occurrs.
+   * @throws IOException thrown if an IO error occurs.
    */
   private HttpURLConnection getConnection(final String method,
       Map<String, String> params, Path path, boolean makeQualified)
@@ -263,7 +263,7 @@ public class HttpFSFileSystem extends FileSystem
    *         HttpFSServer server, authenticated and ready to use for the
    *         specified path and file system operation.
    *
-   * @throws IOException thrown if an IO error occurrs.
+   * @throws IOException thrown if an IO error occurs.
    */
   private HttpURLConnection getConnection(final String method,
       Map<String, String> params, Map<String, List<String>> multiValuedParams,
@@ -301,7 +301,7 @@ public class HttpFSFileSystem extends FileSystem
    * @return a <code>HttpURLConnection</code> for the HttpFSServer server, authenticated and ready to use for
    *         the specified path and file system operation.
    *
-   * @throws IOException thrown if an IO error occurrs.
+   * @throws IOException thrown if an IO error occurs.
    */
   private HttpURLConnection getConnection(URL url, String method) throws IOException {
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8f3946cd/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/CheckUploadContentTypeFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/CheckUploadContentTypeFilter.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/CheckUploadContentTypeFilter.java
index 81b0b7a..e96bfa1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/CheckUploadContentTypeFilter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/CheckUploadContentTypeFilter.java
@@ -70,8 +70,8 @@ public class CheckUploadContentTypeFilter implements Filter {
    * @param response servlet response.
    * @param chain filter chain.
    *
-   * @throws IOException thrown if an IO error occurrs.
-   * @throws ServletException thrown if a servet error occurrs.
+   * @throws IOException thrown if an IO error occurs.
+   * @throws ServletException thrown if a servlet error occurs.
    */
   @Override
   public void doFilter(ServletRequest request, ServletResponse response,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8f3946cd/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
index 1f903ba..b7b63fa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
@@ -104,7 +104,7 @@ public class HttpFSServer {
    *
    * @return FileSystemExecutor response
    *
-   * @throws IOException thrown if an IO error occurrs.
+   * @throws IOException thrown if an IO error occurs.
    * @throws FileSystemAccessException thrown if a FileSystemAccess releated error occurred. Thrown
    * exceptions are handled by {@link HttpFSExceptionProvider}.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8f3946cd/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/servlet/FileSystemReleaseFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/servlet/FileSystemReleaseFilter.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/servlet/FileSystemReleaseFilter.java
index cf73979..ec559f9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/servlet/FileSystemReleaseFilter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/servlet/FileSystemReleaseFilter.java
@@ -64,8 +64,8 @@ public abstract class FileSystemReleaseFilter implements Filter {
    * @param servletResponse servlet response.
    * @param filterChain filter chain.
    *
-   * @throws IOException thrown if an IO error occurrs.
-   * @throws ServletException thrown if a servet error occurrs.
+   * @throws IOException thrown if an IO error occurs.
+   * @throws ServletException thrown if a servlet error occurs.
    */
   @Override
   public void doFilter(ServletRequest servletRequest, ServletResponse servletResponse, FilterChain filterChain)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8f3946cd/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/servlet/HostnameFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/servlet/HostnameFilter.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/servlet/HostnameFilter.java
index 64f4926..ed116e6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/servlet/HostnameFilter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/servlet/HostnameFilter.java
@@ -63,8 +63,8 @@ public class HostnameFilter implements Filter {
    * @param response servlet response.
    * @param chain filter chain.
    *
-   * @throws IOException thrown if an IO error occurrs.
-   * @throws ServletException thrown if a servet error occurrs.
+   * @throws IOException thrown if an IO error occurs.
+   * @throws ServletException thrown if a servlet error occurs.
    */
   @Override
   public void doFilter(ServletRequest request, ServletResponse response, FilterChain chain)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8f3946cd/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/servlet/MDCFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/servlet/MDCFilter.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/servlet/MDCFilter.java
index 156cf64..0f34652 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/servlet/MDCFilter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/servlet/MDCFilter.java
@@ -66,8 +66,8 @@ public class MDCFilter implements Filter {
    * @param response servlet response.
    * @param chain filter chain.
    *
-   * @throws IOException thrown if an IO error occurrs.
-   * @throws ServletException thrown if a servet error occurrs.
+   * @throws IOException thrown if an IO error occurs.
+   * @throws ServletException thrown if a servlet error occurs.
    */
   @Override
   public void doFilter(ServletRequest request, ServletResponse response, FilterChain chain)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8f3946cd/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index c1aac54..a0a81c0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -321,6 +321,8 @@ Trunk (Unreleased)
     HDFS-7673. synthetic load generator docs give incorrect/incomplete commands
     (Brahma Reddy Battula via aw)
 
+    HDFS-8206. Fix the typos in hadoop-hdfs-httpfs. (Brahma Reddy Battula via xyao)
+
 Release 2.8.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES


[04/50] [abbrv] hadoop git commit: YARN-2498. Respect labels in preemption policy of capacity scheduler for inter-queue preemption. Contributed by Wangda Tan

Posted by ji...@apache.org.
YARN-2498. Respect labels in preemption policy of capacity scheduler for inter-queue preemption. Contributed by Wangda Tan


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/d497f6ea
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d497f6ea
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d497f6ea

Branch: refs/heads/HDFS-7240
Commit: d497f6ea2be559aa31ed76f37ae949dbfabe2a51
Parents: dcc5455
Author: Jian He <ji...@apache.org>
Authored: Fri Apr 24 17:03:13 2015 -0700
Committer: Jian He <ji...@apache.org>
Committed: Fri Apr 24 17:03:13 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |    3 +
 .../ProportionalCapacityPreemptionPolicy.java   |  585 +++++----
 .../rmcontainer/RMContainerImpl.java            |   28 +-
 .../scheduler/capacity/CapacityScheduler.java   |    2 +-
 .../scheduler/capacity/LeafQueue.java           |   70 +-
 .../scheduler/common/AssignmentInformation.java |   31 +-
 ...estProportionalCapacityPreemptionPolicy.java |   94 +-
 ...pacityPreemptionPolicyForNodePartitions.java | 1211 ++++++++++++++++++
 .../scheduler/capacity/TestChildQueueOrder.java |    2 +-
 .../scheduler/capacity/TestLeafQueue.java       |    4 +-
 .../TestNodeLabelContainerAllocation.java       |   16 +
 .../scheduler/capacity/TestParentQueue.java     |    2 +-
 12 files changed, 1750 insertions(+), 298 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d497f6ea/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 44b87e5..a830771 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -102,6 +102,9 @@ Release 2.8.0 - UNRELEASED
 
     YARN-3319. Implement a FairOrderingPolicy. (Craig Welch via wangda)
 
+    YARN-2498. Respect labels in preemption policy of capacity scheduler for
+    inter-queue preemption. (Wangda Tan via jianhe)
+
   IMPROVEMENTS
 
     YARN-1880. Cleanup TestApplicationClientProtocolOnHA

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d497f6ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
index 2ab4197..1f47b5f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -26,11 +27,10 @@ import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.NavigableSet;
 import java.util.PriorityQueue;
 import java.util.Set;
+import java.util.TreeSet;
 
-import org.apache.commons.collections.map.HashedMap;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -40,7 +40,6 @@ import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
-import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingEditPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
@@ -49,7 +48,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerPreemptE
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.SystemClock;
@@ -57,6 +58,7 @@ import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableSet;
 
 /**
  * This class implement a {@link SchedulingEditPolicy} that is designed to be
@@ -130,7 +132,9 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
   private float percentageClusterPreemptionAllowed;
   private double naturalTerminationFactor;
   private boolean observeOnly;
-  private Map<NodeId, Set<String>> labels;
+  private Map<String, Map<String, TempQueuePerPartition>> queueToPartitions =
+      new HashMap<>();
+  private RMNodeLabelsManager nlm;
 
   public ProportionalCapacityPreemptionPolicy() {
     clock = new SystemClock();
@@ -170,7 +174,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       config.getFloat(TOTAL_PREEMPTION_PER_ROUND, (float) 0.1);
     observeOnly = config.getBoolean(OBSERVE_ONLY, false);
     rc = scheduler.getResourceCalculator();
-    labels = null;
+    nlm = scheduler.getRMContext().getNodeLabelManager();
   }
   
   @VisibleForTesting
@@ -182,34 +186,8 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
   public void editSchedule() {
     CSQueue root = scheduler.getRootQueue();
     Resource clusterResources = Resources.clone(scheduler.getClusterResource());
-    clusterResources = getNonLabeledResources(clusterResources);
-    setNodeLabels(scheduler.getRMContext().getNodeLabelManager()
-        .getNodeLabels());
     containerBasedPreemptOrKill(root, clusterResources);
   }
-
-  /**
-   * Setting Node Labels
-   * 
-   * @param nodelabels
-   */
-  public void setNodeLabels(Map<NodeId, Set<String>> nodelabels) {
-    labels = nodelabels;
-  }
-
-  /**
-   * This method returns all non labeled resources.
-   * 
-   * @param clusterResources
-   * @return Resources
-   */
-  private Resource getNonLabeledResources(Resource clusterResources) {
-    RMContext rmcontext = scheduler.getRMContext();
-    RMNodeLabelsManager lm = rmcontext.getNodeLabelManager();
-    Resource res = lm.getResourceByLabel(RMNodeLabelsManager.NO_LABEL,
-        clusterResources);
-    return res == null ? clusterResources : res;
-  }
   
   /**
    * This method selects and tracks containers to be preempted. If a container
@@ -220,28 +198,46 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
    */
   private void containerBasedPreemptOrKill(CSQueue root,
       Resource clusterResources) {
+    // All partitions to look at
+    Set<String> allPartitions = new HashSet<>();
+    allPartitions.addAll(scheduler.getRMContext()
+        .getNodeLabelManager().getClusterNodeLabelNames());
+    allPartitions.add(RMNodeLabelsManager.NO_LABEL);
 
     // extract a summary of the queues from scheduler
-    TempQueue tRoot;
     synchronized (scheduler) {
-      tRoot = cloneQueues(root, clusterResources);
+      queueToPartitions.clear();
+
+      for (String partitionToLookAt : allPartitions) {
+        cloneQueues(root,
+            nlm.getResourceByLabel(partitionToLookAt, clusterResources),
+            partitionToLookAt);
+      }
     }
 
-    // compute the ideal distribution of resources among queues
-    // updates cloned queues state accordingly
-    tRoot.idealAssigned = tRoot.guaranteed;
+    // compute total preemption allowed
     Resource totalPreemptionAllowed = Resources.multiply(clusterResources,
         percentageClusterPreemptionAllowed);
-    List<TempQueue> queues =
-      recursivelyComputeIdealAssignment(tRoot, totalPreemptionAllowed);
+
+    Set<String> leafQueueNames = null;
+    for (String partition : allPartitions) {
+      TempQueuePerPartition tRoot =
+          getQueueByPartition(CapacitySchedulerConfiguration.ROOT, partition);
+      // compute the ideal distribution of resources among queues
+      // updates cloned queues state accordingly
+      tRoot.idealAssigned = tRoot.guaranteed;
+
+      leafQueueNames =
+          recursivelyComputeIdealAssignment(tRoot, totalPreemptionAllowed);
+    }
 
     // based on ideal allocation select containers to be preempted from each
     // queue and each application
     Map<ApplicationAttemptId,Set<RMContainer>> toPreempt =
-        getContainersToPreempt(queues, clusterResources);
+        getContainersToPreempt(leafQueueNames, clusterResources);
 
     if (LOG.isDebugEnabled()) {
-      logToCSV(queues);
+      logToCSV(new ArrayList<String>(leafQueueNames));
     }
 
     // if we are in observeOnly mode return before any action is taken
@@ -252,6 +248,10 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     // preempt (or kill) the selected containers
     for (Map.Entry<ApplicationAttemptId,Set<RMContainer>> e
          : toPreempt.entrySet()) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Send to scheduler: in app=" + e.getKey()
+            + " #containers-to-be-preempted=" + e.getValue().size());
+      }
       for (RMContainer container : e.getValue()) {
         // if we tried to preempt this for more than maxWaitTime
         if (preempted.get(container) != null &&
@@ -291,23 +291,24 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
    * @param totalPreemptionAllowed maximum amount of preemption allowed
    * @return a list of leaf queues updated with preemption targets
    */
-  private List<TempQueue> recursivelyComputeIdealAssignment(
-      TempQueue root, Resource totalPreemptionAllowed) {
-    List<TempQueue> leafs = new ArrayList<TempQueue>();
+  private Set<String> recursivelyComputeIdealAssignment(
+      TempQueuePerPartition root, Resource totalPreemptionAllowed) {
+    Set<String> leafQueueNames = new HashSet<>();
     if (root.getChildren() != null &&
         root.getChildren().size() > 0) {
       // compute ideal distribution at this level
       computeIdealResourceDistribution(rc, root.getChildren(),
           totalPreemptionAllowed, root.idealAssigned);
       // compute recursively for lower levels and build list of leafs
-      for(TempQueue t : root.getChildren()) {
-        leafs.addAll(recursivelyComputeIdealAssignment(t, totalPreemptionAllowed));
+      for(TempQueuePerPartition t : root.getChildren()) {
+        leafQueueNames.addAll(recursivelyComputeIdealAssignment(t,
+            totalPreemptionAllowed));
       }
     } else {
       // we are in a leaf nothing to do, just return yourself
-      return Collections.singletonList(root);
+      return ImmutableSet.of(root.queueName);
     }
-    return leafs;
+    return leafQueueNames;
   }
 
   /**
@@ -324,20 +325,21 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
    * @param tot_guarant the amount of capacity assigned to this pool of queues
    */
   private void computeIdealResourceDistribution(ResourceCalculator rc,
-      List<TempQueue> queues, Resource totalPreemptionAllowed, Resource tot_guarant) {
+      List<TempQueuePerPartition> queues, Resource totalPreemptionAllowed,
+      Resource tot_guarant) {
 
     // qAlloc tracks currently active queues (will decrease progressively as
     // demand is met)
-    List<TempQueue> qAlloc = new ArrayList<TempQueue>(queues);
+    List<TempQueuePerPartition> qAlloc = new ArrayList<TempQueuePerPartition>(queues);
     // unassigned tracks how much resources are still to assign, initialized
     // with the total capacity for this set of queues
     Resource unassigned = Resources.clone(tot_guarant);
 
     // group queues based on whether they have non-zero guaranteed capacity
-    Set<TempQueue> nonZeroGuarQueues = new HashSet<TempQueue>();
-    Set<TempQueue> zeroGuarQueues = new HashSet<TempQueue>();
+    Set<TempQueuePerPartition> nonZeroGuarQueues = new HashSet<TempQueuePerPartition>();
+    Set<TempQueuePerPartition> zeroGuarQueues = new HashSet<TempQueuePerPartition>();
 
-    for (TempQueue q : qAlloc) {
+    for (TempQueuePerPartition q : qAlloc) {
       if (Resources
           .greaterThan(rc, tot_guarant, q.guaranteed, Resources.none())) {
         nonZeroGuarQueues.add(q);
@@ -361,7 +363,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     // based on ideal assignment computed above and current assignment we derive
     // how much preemption is required overall
     Resource totPreemptionNeeded = Resource.newInstance(0, 0);
-    for (TempQueue t:queues) {
+    for (TempQueuePerPartition t:queues) {
       if (Resources.greaterThan(rc, tot_guarant, t.current, t.idealAssigned)) {
         Resources.addTo(totPreemptionNeeded,
             Resources.subtract(t.current, t.idealAssigned));
@@ -379,12 +381,12 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
 
     // assign to each queue the amount of actual preemption based on local
     // information of ideal preemption and scaling factor
-    for (TempQueue t : queues) {
+    for (TempQueuePerPartition t : queues) {
       t.assignPreemption(scalingFactor, rc, tot_guarant);
     }
     if (LOG.isDebugEnabled()) {
       long time = clock.getTime();
-      for (TempQueue t : queues) {
+      for (TempQueuePerPartition t : queues) {
         LOG.debug(time + ": " + t);
       }
     }
@@ -400,8 +402,8 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
    * distributed uniformly.
    */
   private void computeFixpointAllocation(ResourceCalculator rc,
-      Resource tot_guarant, Collection<TempQueue> qAlloc, Resource unassigned, 
-      boolean ignoreGuarantee) {
+      Resource tot_guarant, Collection<TempQueuePerPartition> qAlloc,
+      Resource unassigned, boolean ignoreGuarantee) {
     // Prior to assigning the unused resources, process each queue as follows:
     // If current > guaranteed, idealAssigned = guaranteed + untouchable extra
     // Else idealAssigned = current;
@@ -410,10 +412,10 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     // idealAssigned >= current + pending), remove it from consideration.
     // Sort queues from most under-guaranteed to most over-guaranteed.
     TQComparator tqComparator = new TQComparator(rc, tot_guarant);
-    PriorityQueue<TempQueue> orderedByNeed =
-                                 new PriorityQueue<TempQueue>(10,tqComparator);
-    for (Iterator<TempQueue> i = qAlloc.iterator(); i.hasNext();) {
-      TempQueue q = i.next();
+    PriorityQueue<TempQueuePerPartition> orderedByNeed =
+        new PriorityQueue<TempQueuePerPartition>(10, tqComparator);
+    for (Iterator<TempQueuePerPartition> i = qAlloc.iterator(); i.hasNext();) {
+      TempQueuePerPartition q = i.next();
       if (Resources.greaterThan(rc, tot_guarant, q.current, q.guaranteed)) {
         q.idealAssigned = Resources.add(q.guaranteed, q.untouchableExtra);
       } else {
@@ -442,10 +444,11 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       // place it back in the ordered list of queues, recalculating its place
       // in the order of most under-guaranteed to most over-guaranteed. In this
       // way, the most underserved queue(s) are always given resources first.
-      Collection<TempQueue> underserved =
+      Collection<TempQueuePerPartition> underserved =
           getMostUnderservedQueues(orderedByNeed, tqComparator);
-      for (Iterator<TempQueue> i = underserved.iterator(); i.hasNext();) {
-        TempQueue sub = i.next();
+      for (Iterator<TempQueuePerPartition> i = underserved.iterator(); i
+          .hasNext();) {
+        TempQueuePerPartition sub = i.next();
         Resource wQavail = Resources.multiplyAndNormalizeUp(rc,
             unassigned, sub.normalizedGuarantee, Resource.newInstance(1, 1));
         Resource wQidle = sub.offer(wQavail, rc, tot_guarant);
@@ -466,13 +469,13 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
   // Take the most underserved TempQueue (the one on the head). Collect and
   // return the list of all queues that have the same idealAssigned
   // percentage of guaranteed.
-  protected Collection<TempQueue> getMostUnderservedQueues(
-      PriorityQueue<TempQueue> orderedByNeed, TQComparator tqComparator) {
-    ArrayList<TempQueue> underserved = new ArrayList<TempQueue>();
+  protected Collection<TempQueuePerPartition> getMostUnderservedQueues(
+      PriorityQueue<TempQueuePerPartition> orderedByNeed, TQComparator tqComparator) {
+    ArrayList<TempQueuePerPartition> underserved = new ArrayList<TempQueuePerPartition>();
     while (!orderedByNeed.isEmpty()) {
-      TempQueue q1 = orderedByNeed.remove();
+      TempQueuePerPartition q1 = orderedByNeed.remove();
       underserved.add(q1);
-      TempQueue q2 = orderedByNeed.peek();
+      TempQueuePerPartition q2 = orderedByNeed.peek();
       // q1's pct of guaranteed won't be larger than q2's. If it's less, then
       // return what has already been collected. Otherwise, q1's pct of
       // guaranteed == that of q2, so add q2 to underserved list during the
@@ -491,24 +494,90 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
    * @param queues the list of queues to consider
    */
   private void resetCapacity(ResourceCalculator rc, Resource clusterResource,
-      Collection<TempQueue> queues, boolean ignoreGuar) {
+      Collection<TempQueuePerPartition> queues, boolean ignoreGuar) {
     Resource activeCap = Resource.newInstance(0, 0);
     
     if (ignoreGuar) {
-      for (TempQueue q : queues) {
+      for (TempQueuePerPartition q : queues) {
         q.normalizedGuarantee = (float)  1.0f / ((float) queues.size());
       }
     } else {
-      for (TempQueue q : queues) {
+      for (TempQueuePerPartition q : queues) {
         Resources.addTo(activeCap, q.guaranteed);
       }
-      for (TempQueue q : queues) {
+      for (TempQueuePerPartition q : queues) {
         q.normalizedGuarantee = Resources.divide(rc, clusterResource,
             q.guaranteed, activeCap);
       }
     }
   }
 
+  private String getPartitionByNodeId(NodeId nodeId) {
+    return scheduler.getSchedulerNode(nodeId).getPartition();
+  }
+
+  /**
+   * Return should we preempt rmContainer. If we should, deduct from
+   * <code>resourceToObtainByPartition</code>
+   */
+  private boolean tryPreemptContainerAndDeductResToObtain(
+      Map<String, Resource> resourceToObtainByPartitions,
+      RMContainer rmContainer, Resource clusterResource,
+      Map<ApplicationAttemptId, Set<RMContainer>> preemptMap) {
+    ApplicationAttemptId attemptId = rmContainer.getApplicationAttemptId();
+
+    // We will not account resource of a container twice or more
+    if (preemptMapContains(preemptMap, attemptId, rmContainer)) {
+      return false;
+    }
+
+    String nodePartition = getPartitionByNodeId(rmContainer.getAllocatedNode());
+    Resource toObtainByPartition =
+        resourceToObtainByPartitions.get(nodePartition);
+
+    if (null != toObtainByPartition
+        && Resources.greaterThan(rc, clusterResource, toObtainByPartition,
+            Resources.none())) {
+      Resources.subtractFrom(toObtainByPartition,
+          rmContainer.getAllocatedResource());
+      // When we have no more resource need to obtain, remove from map.
+      if (Resources.lessThanOrEqual(rc, clusterResource, toObtainByPartition,
+          Resources.none())) {
+        resourceToObtainByPartitions.remove(nodePartition);
+      }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Marked container=" + rmContainer.getContainerId()
+            + " in partition=" + nodePartition + " will be preempted");
+      }
+      // Add to preemptMap
+      addToPreemptMap(preemptMap, attemptId, rmContainer);
+      return true;
+    }
+
+    return false;
+  }
+
+  private boolean preemptMapContains(
+      Map<ApplicationAttemptId, Set<RMContainer>> preemptMap,
+      ApplicationAttemptId attemptId, RMContainer rmContainer) {
+    Set<RMContainer> rmContainers;
+    if (null == (rmContainers = preemptMap.get(attemptId))) {
+      return false;
+    }
+    return rmContainers.contains(rmContainer);
+  }
+
+  private void addToPreemptMap(
+      Map<ApplicationAttemptId, Set<RMContainer>> preemptMap,
+      ApplicationAttemptId appAttemptId, RMContainer containerToPreempt) {
+    Set<RMContainer> set;
+    if (null == (set = preemptMap.get(appAttemptId))) {
+      set = new HashSet<RMContainer>();
+      preemptMap.put(appAttemptId, set);
+    }
+    set.add(containerToPreempt);
+  }
+
   /**
    * Based a resource preemption target drop reservations of containers and
    * if necessary select containers for preemption from applications in each
@@ -520,64 +589,106 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
    * @return a map of applciationID to set of containers to preempt
    */
   private Map<ApplicationAttemptId,Set<RMContainer>> getContainersToPreempt(
-      List<TempQueue> queues, Resource clusterResource) {
+      Set<String> leafQueueNames, Resource clusterResource) {
 
-    Map<ApplicationAttemptId,Set<RMContainer>> preemptMap =
-        new HashMap<ApplicationAttemptId,Set<RMContainer>>();
+    Map<ApplicationAttemptId, Set<RMContainer>> preemptMap =
+        new HashMap<ApplicationAttemptId, Set<RMContainer>>();
     List<RMContainer> skippedAMContainerlist = new ArrayList<RMContainer>();
 
-    for (TempQueue qT : queues) {
-      if (qT.preemptionDisabled && qT.leafQueue != null) {
+    // Loop all leaf queues
+    for (String queueName : leafQueueNames) {
+      // check if preemption disabled for the queue
+      if (getQueueByPartition(queueName,
+          RMNodeLabelsManager.NO_LABEL).preemptionDisabled) {
         if (LOG.isDebugEnabled()) {
-          if (Resources.greaterThan(rc, clusterResource,
-              qT.toBePreempted, Resource.newInstance(0, 0))) {
-            LOG.debug("Tried to preempt the following "
-                      + "resources from non-preemptable queue: "
-                      + qT.queueName + " - Resources: " + qT.toBePreempted);
-          }
+          LOG.debug("skipping from queue=" + queueName
+              + " because it's a non-preemptable queue");
         }
         continue;
       }
-      // we act only if we are violating balance by more than
-      // maxIgnoredOverCapacity
-      if (Resources.greaterThan(rc, clusterResource, qT.current,
-          Resources.multiply(qT.guaranteed, 1.0 + maxIgnoredOverCapacity))) {
-        // we introduce a dampening factor naturalTerminationFactor that
-        // accounts for natural termination of containers
-        Resource resToObtain =
-          Resources.multiply(qT.toBePreempted, naturalTerminationFactor);
-        Resource skippedAMSize = Resource.newInstance(0, 0);
 
-        // lock the leafqueue while we scan applications and unreserve
-        synchronized (qT.leafQueue) {
-          Iterator<FiCaSchedulerApp> desc =   
-            qT.leafQueue.getOrderingPolicy().getPreemptionIterator();
+      // compute resToObtainByPartition considered inter-queue preemption
+      LeafQueue leafQueue = null;
+
+      Map<String, Resource> resToObtainByPartition =
+          new HashMap<String, Resource>();
+      for (TempQueuePerPartition qT : getQueuePartitions(queueName)) {
+        leafQueue = qT.leafQueue;
+        // we act only if we are violating balance by more than
+        // maxIgnoredOverCapacity
+        if (Resources.greaterThan(rc, clusterResource, qT.current,
+            Resources.multiply(qT.guaranteed, 1.0 + maxIgnoredOverCapacity))) {
+          // we introduce a dampening factor naturalTerminationFactor that
+          // accounts for natural termination of containers
+          Resource resToObtain =
+              Resources.multiply(qT.toBePreempted, naturalTerminationFactor);
+          // Only add resToObtain when it >= 0
+          if (Resources.greaterThan(rc, clusterResource, resToObtain,
+              Resources.none())) {
+            resToObtainByPartition.put(qT.partition, resToObtain);
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Queue=" + queueName + " partition=" + qT.partition
+                  + " resource-to-obtain=" + resToObtain);
+            }
+          }
           qT.actuallyPreempted = Resources.clone(resToObtain);
-          while (desc.hasNext()) {
-            FiCaSchedulerApp fc = desc.next();
-            if (Resources.lessThanOrEqual(rc, clusterResource, resToObtain,
-                Resources.none())) {
-              break;
+        } else {
+          qT.actuallyPreempted = Resources.none();
+        }
+      }
+
+      synchronized (leafQueue) {
+        // go through all ignore-partition-exclusivity containers first to make
+        // sure such containers will be preempted first
+        Map<String, TreeSet<RMContainer>> ignorePartitionExclusivityContainers =
+            leafQueue.getIgnoreExclusivityRMContainers();
+        for (String partition : resToObtainByPartition.keySet()) {
+          if (ignorePartitionExclusivityContainers.containsKey(partition)) {
+            TreeSet<RMContainer> rmContainers =
+                ignorePartitionExclusivityContainers.get(partition);
+            // We will check container from reverse order, so latter submitted
+            // application's containers will be preempted first.
+            for (RMContainer c : rmContainers.descendingSet()) {
+              boolean preempted =
+                  tryPreemptContainerAndDeductResToObtain(
+                      resToObtainByPartition, c, clusterResource, preemptMap);
+              if (!preempted) {
+                break;
+              }
             }
-            preemptMap.put(
-                fc.getApplicationAttemptId(),
-                preemptFrom(fc, clusterResource, resToObtain,
-                    skippedAMContainerlist, skippedAMSize));
           }
-          Resource maxAMCapacityForThisQueue = Resources.multiply(
-              Resources.multiply(clusterResource,
-                  qT.leafQueue.getAbsoluteCapacity()),
-              qT.leafQueue.getMaxAMResourcePerQueuePercent());
-
-          // Can try preempting AMContainers (still saving atmost
-          // maxAMCapacityForThisQueue AMResource's) if more resources are
-          // required to be preempted from this Queue.
-          preemptAMContainers(clusterResource, preemptMap,
-              skippedAMContainerlist, resToObtain, skippedAMSize,
-              maxAMCapacityForThisQueue);
         }
+
+        // preempt other containers
+        Resource skippedAMSize = Resource.newInstance(0, 0);
+        Iterator<FiCaSchedulerApp> desc =
+            leafQueue.getOrderingPolicy().getPreemptionIterator();
+        while (desc.hasNext()) {
+          FiCaSchedulerApp fc = desc.next();
+          // When we complete preempt from one partition, we will remove from
+          // resToObtainByPartition, so when it becomes empty, we can get no
+          // more preemption is needed
+          if (resToObtainByPartition.isEmpty()) {
+            break;
+          }
+
+          preemptFrom(fc, clusterResource, resToObtainByPartition,
+              skippedAMContainerlist, skippedAMSize, preemptMap);
+        }
+
+        // Can try preempting AMContainers (still saving atmost
+        // maxAMCapacityForThisQueue AMResource's) if more resources are
+        // required to be preempted from this Queue.
+        Resource maxAMCapacityForThisQueue = Resources.multiply(
+            Resources.multiply(clusterResource,
+                leafQueue.getAbsoluteCapacity()),
+            leafQueue.getMaxAMResourcePerQueuePercent());
+
+        preemptAMContainers(clusterResource, preemptMap, skippedAMContainerlist,
+            resToObtainByPartition, skippedAMSize, maxAMCapacityForThisQueue);
       }
     }
+
     return preemptMap;
   }
 
@@ -595,31 +706,27 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
    */
   private void preemptAMContainers(Resource clusterResource,
       Map<ApplicationAttemptId, Set<RMContainer>> preemptMap,
-      List<RMContainer> skippedAMContainerlist, Resource resToObtain,
-      Resource skippedAMSize, Resource maxAMCapacityForThisQueue) {
+      List<RMContainer> skippedAMContainerlist,
+      Map<String, Resource> resToObtainByPartition, Resource skippedAMSize,
+      Resource maxAMCapacityForThisQueue) {
     for (RMContainer c : skippedAMContainerlist) {
       // Got required amount of resources for preemption, can stop now
-      if (Resources.lessThanOrEqual(rc, clusterResource, resToObtain,
-          Resources.none())) {
+      if (resToObtainByPartition.isEmpty()) {
         break;
       }
       // Once skippedAMSize reaches down to maxAMCapacityForThisQueue,
-      // container selection iteration for preemption will be stopped. 
+      // container selection iteration for preemption will be stopped.
       if (Resources.lessThanOrEqual(rc, clusterResource, skippedAMSize,
           maxAMCapacityForThisQueue)) {
         break;
       }
-      Set<RMContainer> contToPrempt = preemptMap.get(c
-          .getApplicationAttemptId());
-      if (null == contToPrempt) {
-        contToPrempt = new HashSet<RMContainer>();
-        preemptMap.put(c.getApplicationAttemptId(), contToPrempt);
+
+      boolean preempted =
+          tryPreemptContainerAndDeductResToObtain(resToObtainByPartition, c,
+              clusterResource, preemptMap);
+      if (preempted) {
+        Resources.subtractFrom(skippedAMSize, c.getAllocatedResource());
       }
-      contToPrempt.add(c);
-      
-      Resources.subtractFrom(resToObtain, c.getContainer().getResource());
-      Resources.subtractFrom(skippedAMSize, c.getContainer()
-          .getResource());
     }
     skippedAMContainerlist.clear();
   }
@@ -627,71 +734,59 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
   /**
    * Given a target preemption for a specific application, select containers
    * to preempt (after unreserving all reservation for that app).
-   *
-   * @param app
-   * @param clusterResource
-   * @param rsrcPreempt
-   * @return Set<RMContainer> Set of RMContainers
    */
-  private Set<RMContainer> preemptFrom(FiCaSchedulerApp app,
-      Resource clusterResource, Resource rsrcPreempt,
-      List<RMContainer> skippedAMContainerlist, Resource skippedAMSize) {
-    Set<RMContainer> ret = new HashSet<RMContainer>();
+  private void preemptFrom(FiCaSchedulerApp app,
+      Resource clusterResource, Map<String, Resource> resToObtainByPartition,
+      List<RMContainer> skippedAMContainerlist, Resource skippedAMSize,
+      Map<ApplicationAttemptId, Set<RMContainer>> preemptMap) {
     ApplicationAttemptId appId = app.getApplicationAttemptId();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Looking at application=" + app.getApplicationAttemptId()
+          + " resourceToObtain=" + resToObtainByPartition);
+    }
 
     // first drop reserved containers towards rsrcPreempt
-    List<RMContainer> reservations =
+    List<RMContainer> reservedContainers =
         new ArrayList<RMContainer>(app.getReservedContainers());
-    for (RMContainer c : reservations) {
-      if (Resources.lessThanOrEqual(rc, clusterResource,
-          rsrcPreempt, Resources.none())) {
-        return ret;
+    for (RMContainer c : reservedContainers) {
+      if (resToObtainByPartition.isEmpty()) {
+        return;
       }
+
+      // Try to preempt this container
+      tryPreemptContainerAndDeductResToObtain(resToObtainByPartition, c,
+          clusterResource, preemptMap);
+
       if (!observeOnly) {
         dispatcher.handle(new ContainerPreemptEvent(appId, c,
             ContainerPreemptEventType.DROP_RESERVATION));
       }
-      Resources.subtractFrom(rsrcPreempt, c.getContainer().getResource());
     }
 
     // if more resources are to be freed go through all live containers in
     // reverse priority and reverse allocation order and mark them for
     // preemption
-    List<RMContainer> containers =
+    List<RMContainer> liveContainers =
       new ArrayList<RMContainer>(app.getLiveContainers());
 
-    sortContainers(containers);
+    sortContainers(liveContainers);
 
-    for (RMContainer c : containers) {
-      if (Resources.lessThanOrEqual(rc, clusterResource,
-            rsrcPreempt, Resources.none())) {
-        return ret;
+    for (RMContainer c : liveContainers) {
+      if (resToObtainByPartition.isEmpty()) {
+        return;
       }
+
       // Skip AM Container from preemption for now.
       if (c.isAMContainer()) {
         skippedAMContainerlist.add(c);
-        Resources.addTo(skippedAMSize, c.getContainer().getResource());
-        continue;
-      }
-      // skip Labeled resource
-      if(isLabeledContainer(c)){
+        Resources.addTo(skippedAMSize, c.getAllocatedResource());
         continue;
       }
-      ret.add(c);
-      Resources.subtractFrom(rsrcPreempt, c.getContainer().getResource());
-    }
 
-    return ret;
-  }
-  
-  /**
-   * Checking if given container is a labeled container
-   * 
-   * @param c
-   * @return true/false
-   */
-  private boolean isLabeledContainer(RMContainer c) {
-    return labels.containsKey(c.getAllocatedNode());
+      // Try to preempt this container
+      tryPreemptContainerAndDeductResToObtain(resToObtainByPartition, c,
+          clusterResource, preemptMap);
+    }
   }
 
   /**
@@ -733,32 +828,48 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
    * the leaves. Finally it aggregates pending resources in each queue and rolls
    * it up to higher levels.
    *
-   * @param root the root of the CapacityScheduler queue hierarchy
-   * @param clusterResources the total amount of resources in the cluster
+   * @param curQueue current queue which I'm looking at now
+   * @param partitionResource the total amount of resources in the cluster
    * @return the root of the cloned queue hierarchy
    */
-  private TempQueue cloneQueues(CSQueue root, Resource clusterResources) {
-    TempQueue ret;
-    synchronized (root) {
-      String queueName = root.getQueueName();
-      float absUsed = root.getAbsoluteUsedCapacity();
-      float absCap = root.getAbsoluteCapacity();
-      float absMaxCap = root.getAbsoluteMaximumCapacity();
-      boolean preemptionDisabled = root.getPreemptionDisabled();
-
-      Resource current = Resources.multiply(clusterResources, absUsed);
-      Resource guaranteed = Resources.multiply(clusterResources, absCap);
-      Resource maxCapacity = Resources.multiply(clusterResources, absMaxCap);
+  private TempQueuePerPartition cloneQueues(CSQueue curQueue,
+      Resource partitionResource, String partitionToLookAt) {
+    TempQueuePerPartition ret;
+    synchronized (curQueue) {
+      String queueName = curQueue.getQueueName();
+      QueueCapacities qc = curQueue.getQueueCapacities();
+      float absUsed = qc.getAbsoluteUsedCapacity(partitionToLookAt);
+      float absCap = qc.getAbsoluteCapacity(partitionToLookAt);
+      float absMaxCap = qc.getAbsoluteMaximumCapacity(partitionToLookAt);
+      boolean preemptionDisabled = curQueue.getPreemptionDisabled();
+
+      Resource current = Resources.multiply(partitionResource, absUsed);
+      Resource guaranteed = Resources.multiply(partitionResource, absCap);
+      Resource maxCapacity = Resources.multiply(partitionResource, absMaxCap);
+
+      // when partition is a non-exclusive partition, the actual maxCapacity
+      // could more than specified maxCapacity
+      try {
+        if (!scheduler.getRMContext().getNodeLabelManager()
+            .isExclusiveNodeLabel(partitionToLookAt)) {
+          maxCapacity =
+              Resources.max(rc, partitionResource, maxCapacity, current);
+        }
+      } catch (IOException e) {
+        // This may cause by partition removed when running capacity monitor,
+        // just ignore the error, this will be corrected when doing next check.
+      }
 
       Resource extra = Resource.newInstance(0, 0);
-      if (Resources.greaterThan(rc, clusterResources, current, guaranteed)) {
+      if (Resources.greaterThan(rc, partitionResource, current, guaranteed)) {
         extra = Resources.subtract(current, guaranteed);
       }
-      if (root instanceof LeafQueue) {
-        LeafQueue l = (LeafQueue) root;
-        Resource pending = l.getTotalResourcePending();
-        ret = new TempQueue(queueName, current, pending, guaranteed,
-            maxCapacity, preemptionDisabled);
+      if (curQueue instanceof LeafQueue) {
+        LeafQueue l = (LeafQueue) curQueue;
+        Resource pending =
+            l.getQueueResourceUsage().getPending(partitionToLookAt);
+        ret = new TempQueuePerPartition(queueName, current, pending, guaranteed,
+            maxCapacity, preemptionDisabled, partitionToLookAt);
         if (preemptionDisabled) {
           ret.untouchableExtra = extra;
         } else {
@@ -767,17 +878,19 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
         ret.setLeafQueue(l);
       } else {
         Resource pending = Resource.newInstance(0, 0);
-        ret = new TempQueue(root.getQueueName(), current, pending, guaranteed,
-            maxCapacity, false);
+        ret =
+            new TempQueuePerPartition(curQueue.getQueueName(), current, pending,
+                guaranteed, maxCapacity, false, partitionToLookAt);
         Resource childrensPreemptable = Resource.newInstance(0, 0);
-        for (CSQueue c : root.getChildQueues()) {
-          TempQueue subq = cloneQueues(c, clusterResources);
+        for (CSQueue c : curQueue.getChildQueues()) {
+          TempQueuePerPartition subq =
+              cloneQueues(c, partitionResource, partitionToLookAt);
           Resources.addTo(childrensPreemptable, subq.preemptableExtra);
           ret.addChild(subq);
         }
         // untouchableExtra = max(extra - childrenPreemptable, 0)
         if (Resources.greaterThanOrEqual(
-              rc, clusterResources, childrensPreemptable, extra)) {
+              rc, partitionResource, childrensPreemptable, extra)) {
           ret.untouchableExtra = Resource.newInstance(0, 0);
         } else {
           ret.untouchableExtra =
@@ -785,52 +898,87 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
         }
       }
     }
+    addTempQueuePartition(ret);
     return ret;
   }
 
   // simple printout function that reports internal queue state (useful for
   // plotting)
-  private void logToCSV(List<TempQueue> unorderedqueues){
-    List<TempQueue> queues = new ArrayList<TempQueue>(unorderedqueues);
-    Collections.sort(queues, new Comparator<TempQueue>(){
-      @Override
-      public int compare(TempQueue o1, TempQueue o2) {
-        return o1.queueName.compareTo(o2.queueName);
-      }});
+  private void logToCSV(List<String> leafQueueNames){
+    Collections.sort(leafQueueNames);
     String queueState = " QUEUESTATE: " + clock.getTime();
     StringBuilder sb = new StringBuilder();
     sb.append(queueState);
-    for (TempQueue tq : queues) {
+
+    for (String queueName : leafQueueNames) {
+      TempQueuePerPartition tq =
+          getQueueByPartition(queueName, RMNodeLabelsManager.NO_LABEL);
       sb.append(", ");
       tq.appendLogString(sb);
     }
     LOG.debug(sb.toString());
   }
 
+  private void addTempQueuePartition(TempQueuePerPartition queuePartition) {
+    String queueName = queuePartition.queueName;
+
+    Map<String, TempQueuePerPartition> queuePartitions;
+    if (null == (queuePartitions = queueToPartitions.get(queueName))) {
+      queuePartitions = new HashMap<String, TempQueuePerPartition>();
+      queueToPartitions.put(queueName, queuePartitions);
+    }
+    queuePartitions.put(queuePartition.partition, queuePartition);
+  }
+
+  /**
+   * Get queue partition by given queueName and partitionName
+   */
+  private TempQueuePerPartition getQueueByPartition(String queueName,
+      String partition) {
+    Map<String, TempQueuePerPartition> partitionToQueues = null;
+    if (null == (partitionToQueues = queueToPartitions.get(queueName))) {
+      return null;
+    }
+    return partitionToQueues.get(partition);
+  }
+
+  /**
+   * Get all queue partitions by given queueName
+   */
+  private Collection<TempQueuePerPartition> getQueuePartitions(String queueName) {
+    if (!queueToPartitions.containsKey(queueName)) {
+      return null;
+    }
+    return queueToPartitions.get(queueName).values();
+  }
+
   /**
    * Temporary data-structure tracking resource availability, pending resource
-   * need, current utilization. Used to clone {@link CSQueue}.
+   * need, current utilization. This is per-queue-per-partition data structure
    */
-  static class TempQueue {
+  static class TempQueuePerPartition {
     final String queueName;
     final Resource current;
     final Resource pending;
     final Resource guaranteed;
     final Resource maxCapacity;
+    final String partition;
     Resource idealAssigned;
     Resource toBePreempted;
+    // For logging purpose
     Resource actuallyPreempted;
     Resource untouchableExtra;
     Resource preemptableExtra;
 
     double normalizedGuarantee;
 
-    final ArrayList<TempQueue> children;
+    final ArrayList<TempQueuePerPartition> children;
     LeafQueue leafQueue;
     boolean preemptionDisabled;
 
-    TempQueue(String queueName, Resource current, Resource pending,
-        Resource guaranteed, Resource maxCapacity, boolean preemptionDisabled) {
+    TempQueuePerPartition(String queueName, Resource current, Resource pending,
+        Resource guaranteed, Resource maxCapacity, boolean preemptionDisabled,
+        String partition) {
       this.queueName = queueName;
       this.current = current;
       this.pending = pending;
@@ -840,10 +988,11 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       this.actuallyPreempted = Resource.newInstance(0, 0);
       this.toBePreempted = Resource.newInstance(0, 0);
       this.normalizedGuarantee = Float.NaN;
-      this.children = new ArrayList<TempQueue>();
+      this.children = new ArrayList<TempQueuePerPartition>();
       this.untouchableExtra = Resource.newInstance(0, 0);
       this.preemptableExtra = Resource.newInstance(0, 0);
       this.preemptionDisabled = preemptionDisabled;
+      this.partition = partition;
     }
 
     public void setLeafQueue(LeafQueue l){
@@ -855,19 +1004,19 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
      * When adding a child we also aggregate its pending resource needs.
      * @param q the child queue to add to this queue
      */
-    public void addChild(TempQueue q) {
+    public void addChild(TempQueuePerPartition q) {
       assert leafQueue == null;
       children.add(q);
       Resources.addTo(pending, q.pending);
     }
 
-    public void addChildren(ArrayList<TempQueue> queues) {
+    public void addChildren(ArrayList<TempQueuePerPartition> queues) {
       assert leafQueue == null;
       children.addAll(queues);
     }
 
 
-    public ArrayList<TempQueue> getChildren(){
+    public ArrayList<TempQueuePerPartition> getChildren(){
       return children;
     }
 
@@ -909,7 +1058,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
 
     public void printAll() {
       LOG.info(this.toString());
-      for (TempQueue sub : this.getChildren()) {
+      for (TempQueuePerPartition sub : this.getChildren()) {
         sub.printAll();
       }
     }
@@ -942,7 +1091,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
 
   }
 
-  static class TQComparator implements Comparator<TempQueue> {
+  static class TQComparator implements Comparator<TempQueuePerPartition> {
     private ResourceCalculator rc;
     private Resource clusterRes;
 
@@ -952,7 +1101,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     }
 
     @Override
-    public int compare(TempQueue tq1, TempQueue tq2) {
+    public int compare(TempQueuePerPartition tq1, TempQueuePerPartition tq2) {
       if (getIdealPctOfGuaranteed(tq1) < getIdealPctOfGuaranteed(tq2)) {
         return -1;
       }
@@ -965,7 +1114,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
     // Calculates idealAssigned / guaranteed
     // TempQueues with 0 guarantees are always considered the most over
     // capacity and therefore considered last for resources.
-    private double getIdealPctOfGuaranteed(TempQueue q) {
+    private double getIdealPctOfGuaranteed(TempQueuePerPartition q) {
       double pctOver = Integer.MAX_VALUE;
       if (q != null && Resources.greaterThan(
           rc, clusterRes, q.guaranteed, Resources.none())) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d497f6ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
index 2750d4e..316a450 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
@@ -56,7 +56,7 @@ import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 
 @SuppressWarnings({"unchecked", "rawtypes"})
-public class RMContainerImpl implements RMContainer {
+public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
 
   private static final Log LOG = LogFactory.getLog(RMContainerImpl.class);
 
@@ -615,4 +615,30 @@ public class RMContainerImpl implements RMContainer {
     }
     return nodeLabelExpression;
   }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof RMContainer) {
+      if (null != getContainerId()) {
+        return getContainerId().equals(((RMContainer) obj).getContainerId());
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    if (null != getContainerId()) {
+      return getContainerId().hashCode();
+    }
+    return super.hashCode();
+  }
+
+  @Override
+  public int compareTo(RMContainer o) {
+    if (containerId != null && o.getContainerId() != null) {
+      return containerId.compareTo(o.getContainerId());
+    }
+    return -1;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d497f6ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
index 1e1623d..48c7f2f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
@@ -153,7 +153,7 @@ public class CapacityScheduler extends
   static final PartitionedQueueComparator partitionedQueueComparator =
       new PartitionedQueueComparator();
 
-  static final Comparator<FiCaSchedulerApp> applicationComparator = 
+  public static final Comparator<FiCaSchedulerApp> applicationComparator =
     new Comparator<FiCaSchedulerApp>() {
     @Override
     public int compare(FiCaSchedulerApp a1, FiCaSchedulerApp a2) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d497f6ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
index 22aafaa..56ade84 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
@@ -68,9 +68,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.*;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.FifoOrderingPolicy;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.OrderingPolicy;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.server.utils.Lock;
 import org.apache.hadoop.yarn.server.utils.Lock.NoLock;
@@ -118,11 +119,16 @@ public class LeafQueue extends AbstractCSQueue {
   
   private final QueueResourceLimitsInfo queueResourceLimitsInfo =
       new QueueResourceLimitsInfo();
-  
+
   private volatile ResourceLimits cachedResourceLimitsForHeadroom = null;
 
   private OrderingPolicy<FiCaSchedulerApp> 
     orderingPolicy = new FifoOrderingPolicy<FiCaSchedulerApp>();
+
+  // record all ignore partition exclusivityRMContainer, this will be used to do
+  // preemption, key is the partition of the RMContainer allocated on
+  private Map<String, TreeSet<RMContainer>> ignorePartitionExclusivityRMContainers =
+      new HashMap<>();
   
   public LeafQueue(CapacitySchedulerContext cs, 
       String queueName, CSQueue parent, CSQueue old) throws IOException {
@@ -921,11 +927,16 @@ public class LeafQueue extends AbstractCSQueue {
           Resource assigned = assignment.getResource();
           if (Resources.greaterThan(
               resourceCalculator, clusterResource, assigned, Resources.none())) {
+            // Get reserved or allocated container from application
+            RMContainer reservedOrAllocatedRMContainer =
+                application.getRMContainer(assignment
+                    .getAssignmentInformation()
+                    .getFirstAllocatedOrReservedContainerId());
 
             // Book-keeping 
             // Note: Update headroom to account for current allocation too...
             allocateResource(clusterResource, application, assigned,
-                node.getPartition());
+                node.getPartition(), reservedOrAllocatedRMContainer);
             
             // Don't reset scheduling opportunities for offswitch assignments
             // otherwise the app will be delayed for each non-local assignment.
@@ -1720,7 +1731,7 @@ public class LeafQueue extends AbstractCSQueue {
           orderingPolicy.containerReleased(application, rmContainer);
           
           releaseResource(clusterResource, application,
-              container.getResource(), node.getPartition());
+              container.getResource(), node.getPartition(), rmContainer);
           LOG.info("completedContainer" +
               " container=" + container +
               " queue=" + this +
@@ -1738,9 +1749,22 @@ public class LeafQueue extends AbstractCSQueue {
 
   synchronized void allocateResource(Resource clusterResource,
       SchedulerApplicationAttempt application, Resource resource,
-      String nodePartition) {
+      String nodePartition, RMContainer rmContainer) {
     super.allocateResource(clusterResource, resource, nodePartition);
     
+    // handle ignore exclusivity container
+    if (null != rmContainer && rmContainer.getNodeLabelExpression().equals(
+        RMNodeLabelsManager.NO_LABEL)
+        && !nodePartition.equals(RMNodeLabelsManager.NO_LABEL)) {
+      TreeSet<RMContainer> rmContainers = null;
+      if (null == (rmContainers =
+          ignorePartitionExclusivityRMContainers.get(nodePartition))) {
+        rmContainers = new TreeSet<>();
+        ignorePartitionExclusivityRMContainers.put(nodePartition, rmContainers);
+      }
+      rmContainers.add(rmContainer);
+    }
+
     // Update user metrics
     String userName = application.getUser();
     User user = getUser(userName);
@@ -1760,10 +1784,25 @@ public class LeafQueue extends AbstractCSQueue {
     }
   }
 
-  synchronized void releaseResource(Resource clusterResource, 
-      FiCaSchedulerApp application, Resource resource, String nodePartition) {
+  synchronized void releaseResource(Resource clusterResource,
+      FiCaSchedulerApp application, Resource resource, String nodePartition,
+      RMContainer rmContainer) {
     super.releaseResource(clusterResource, resource, nodePartition);
     
+    // handle ignore exclusivity container
+    if (null != rmContainer && rmContainer.getNodeLabelExpression().equals(
+        RMNodeLabelsManager.NO_LABEL)
+        && !nodePartition.equals(RMNodeLabelsManager.NO_LABEL)) {
+      if (ignorePartitionExclusivityRMContainers.containsKey(nodePartition)) {
+        Set<RMContainer> rmContainers =
+            ignorePartitionExclusivityRMContainers.get(nodePartition);
+        rmContainers.remove(rmContainer);
+        if (rmContainers.isEmpty()) {
+          ignorePartitionExclusivityRMContainers.remove(nodePartition);
+        }
+      }
+    }
+
     // Update user metrics
     String userName = application.getUser();
     User user = getUser(userName);
@@ -1912,7 +1951,7 @@ public class LeafQueue extends AbstractCSQueue {
       FiCaSchedulerNode node =
           scheduler.getNode(rmContainer.getContainer().getNodeId());
       allocateResource(clusterResource, attempt, rmContainer.getContainer()
-          .getResource(), node.getPartition());
+          .getResource(), node.getPartition(), rmContainer);
     }
     getParent().recoverContainer(clusterResource, attempt, rmContainer);
   }
@@ -1953,7 +1992,7 @@ public class LeafQueue extends AbstractCSQueue {
       FiCaSchedulerNode node =
           scheduler.getNode(rmContainer.getContainer().getNodeId());
       allocateResource(clusterResource, application, rmContainer.getContainer()
-          .getResource(), node.getPartition());
+          .getResource(), node.getPartition(), rmContainer);
       LOG.info("movedContainer" + " container=" + rmContainer.getContainer()
           + " resource=" + rmContainer.getContainer().getResource()
           + " queueMoveIn=" + this + " usedCapacity=" + getUsedCapacity()
@@ -1971,7 +2010,7 @@ public class LeafQueue extends AbstractCSQueue {
       FiCaSchedulerNode node =
           scheduler.getNode(rmContainer.getContainer().getNodeId());
       releaseResource(clusterResource, application, rmContainer.getContainer()
-          .getResource(), node.getPartition());
+          .getResource(), node.getPartition(), rmContainer);
       LOG.info("movedContainer" + " container=" + rmContainer.getContainer()
           + " resource=" + rmContainer.getContainer().getResource()
           + " queueMoveOut=" + this + " usedCapacity=" + getUsedCapacity()
@@ -1982,6 +2021,17 @@ public class LeafQueue extends AbstractCSQueue {
     }
   }
   
+  /**
+   * return all ignored partition exclusivity RMContainers in the LeafQueue, this
+   * will be used by preemption policy, and use of return
+   * ignorePartitionExclusivityRMContainer should protected by LeafQueue
+   * synchronized lock
+   */
+  public synchronized Map<String, TreeSet<RMContainer>>
+      getIgnoreExclusivityRMContainers() {
+    return ignorePartitionExclusivityRMContainers;
+  }
+
   public void setCapacity(float capacity) {
     queueCapacities.setCapacity(capacity);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d497f6ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/AssignmentInformation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/AssignmentInformation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/AssignmentInformation.java
index c5c067d..5158255 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/AssignmentInformation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/AssignmentInformation.java
@@ -18,16 +18,17 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.common;
 
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.Resource;
-
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 public class AssignmentInformation {
@@ -117,4 +118,24 @@ public class AssignmentInformation {
   public List<AssignmentDetails> getReservationDetails() {
     return operationDetails.get(Operation.RESERVATION);
   }
+
+  private ContainerId getFirstContainerIdFromOperation(Operation op) {
+    if (null != operationDetails.get(Operation.ALLOCATION)) {
+      List<AssignmentDetails> assignDetails =
+          operationDetails.get(Operation.ALLOCATION);
+      if (!assignDetails.isEmpty()) {
+        return assignDetails.get(0).containerId;
+      }
+    }
+    return null;
+  }
+
+  public ContainerId getFirstAllocatedOrReservedContainerId() {
+    ContainerId containerId = null;
+    containerId = getFirstContainerIdFromOperation(Operation.ALLOCATION);
+    if (null != containerId) {
+      return containerId;
+    }
+    return getFirstContainerIdFromOperation(Operation.RESERVATION);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d497f6ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
index 9e8b769..6c0ed6c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
@@ -25,11 +25,12 @@ import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.Pro
 import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.WAIT_TIME_BEFORE_KILL;
 import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerPreemptEventType.KILL_CONTAINER;
 import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerPreemptEventType.PREEMPT_CONTAINER;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.*;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyString;
 import static org.mockito.Matchers.argThat;
 import static org.mockito.Matchers.isA;
 import static org.mockito.Mockito.mock;
@@ -37,27 +38,17 @@ import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyString;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer; 
 
 import java.util.ArrayList;
 import java.util.Comparator;
 import java.util.Deque;
-import java.util.HashMap;
-import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
-import java.util.Iterator;
-import java.util.Map;
 import java.util.NavigableSet;
 import java.util.Random;
-import java.util.Set;
 import java.util.StringTokenizer;
 import java.util.TreeSet;
 
-import org.apache.commons.collections.map.HashedMap;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.Service;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -76,23 +67,27 @@ import org.apache.hadoop.yarn.server.resourcemanager.resource.Priority;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerPreemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerPreemptEventType;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.OrderingPolicy;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
-import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TestName;
 import org.mockito.ArgumentCaptor;
 import org.mockito.ArgumentMatcher;
-import org.mortbay.log.Log;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 
 public class TestProportionalCapacityPreemptionPolicy {
 
@@ -799,50 +794,6 @@ public class TestProportionalCapacityPreemptionPolicy {
   }
 
   @Test
-  public void testIdealAllocationForLabels() {
-    int[][] qData = new int[][] {
-    // / A B
-        { 80, 40, 40 }, // abs
-        { 80, 80, 80 }, // maxcap
-        { 80, 80, 0 }, // used
-        { 70, 20, 50 }, // pending
-        { 0, 0, 0 }, // reserved
-        { 5, 4, 1 }, // apps
-        { -1, 1, 1 }, // req granularity
-        { 2, 0, 0 }, // subqueues
-    };
-    setAMContainer = true;
-    setLabeledContainer = true;
-    Map<NodeId, Set<String>> labels = new HashMap<NodeId, Set<String>>();
-    NodeId node = NodeId.newInstance("node1", 0);
-    Set<String> labelSet = new HashSet<String>();
-    labelSet.add("x");
-    labels.put(node, labelSet);
-    when(lm.getNodeLabels()).thenReturn(labels);
-    ProportionalCapacityPreemptionPolicy policy = buildPolicy(qData);
-    // Subtracting Label X resources from cluster resources
-    when(lm.getResourceByLabel(anyString(), any(Resource.class))).thenReturn(
-        Resources.clone(Resource.newInstance(80, 0)));
-    clusterResources.setMemory(100);
-    policy.editSchedule();
-
-    // By skipping AM Container and Labeled container, all other 18 containers
-    // of appD will be
-    // preempted
-    verify(mDisp, times(19)).handle(argThat(new IsPreemptionRequestFor(appD)));
-
-    // By skipping AM Container and Labeled container, all other 18 containers
-    // of appC will be
-    // preempted
-    verify(mDisp, times(19)).handle(argThat(new IsPreemptionRequestFor(appC)));
-
-    // rest 4 containers from appB will be preempted
-    verify(mDisp, times(2)).handle(argThat(new IsPreemptionRequestFor(appB)));
-    setAMContainer = false;
-    setLabeledContainer = false;
-  }
-
-  @Test
   public void testPreemptSkippedAMContainers() {
     int[][] qData = new int[][] {
         //  /   A   B
@@ -944,6 +895,12 @@ public class TestProportionalCapacityPreemptionPolicy {
     clusterResources =
       Resource.newInstance(leafAbsCapacities(qData[0], qData[7]), 0);
     when(mCS.getClusterResource()).thenReturn(clusterResources);
+    when(lm.getResourceByLabel(anyString(), any(Resource.class))).thenReturn(
+        clusterResources);
+
+    SchedulerNode mNode = mock(SchedulerNode.class);
+    when(mNode.getPartition()).thenReturn(RMNodeLabelsManager.NO_LABEL);
+    when(mCS.getSchedulerNode(any(NodeId.class))).thenReturn(mNode);
     return policy;
   }
 
@@ -965,11 +922,16 @@ public class TestProportionalCapacityPreemptionPolicy {
     float tot = leafAbsCapacities(abs, queues);
     Deque<ParentQueue> pqs = new LinkedList<ParentQueue>();
     ParentQueue root = mockParentQueue(null, queues[0], pqs);
-    when(root.getQueueName()).thenReturn("/");
+    when(root.getQueueName()).thenReturn(CapacitySchedulerConfiguration.ROOT);
     when(root.getAbsoluteUsedCapacity()).thenReturn(used[0] / tot);
     when(root.getAbsoluteCapacity()).thenReturn(abs[0] / tot);
     when(root.getAbsoluteMaximumCapacity()).thenReturn(maxCap[0] / tot);
-    when(root.getQueuePath()).thenReturn("root");
+    QueueCapacities rootQc = new QueueCapacities(true);
+    rootQc.setAbsoluteUsedCapacity(used[0] / tot);
+    rootQc.setAbsoluteCapacity(abs[0] / tot);
+    rootQc.setAbsoluteMaximumCapacity(maxCap[0] / tot);
+    when(root.getQueueCapacities()).thenReturn(rootQc);
+    when(root.getQueuePath()).thenReturn(CapacitySchedulerConfiguration.ROOT);
     boolean preemptionDisabled = mockPreemptionStatus("root");
     when(root.getPreemptionDisabled()).thenReturn(preemptionDisabled);
 
@@ -987,6 +949,14 @@ public class TestProportionalCapacityPreemptionPolicy {
       when(q.getAbsoluteUsedCapacity()).thenReturn(used[i] / tot);
       when(q.getAbsoluteCapacity()).thenReturn(abs[i] / tot);
       when(q.getAbsoluteMaximumCapacity()).thenReturn(maxCap[i] / tot);
+
+      // We need to make these fields to QueueCapacities
+      QueueCapacities qc = new QueueCapacities(false);
+      qc.setAbsoluteUsedCapacity(used[i] / tot);
+      qc.setAbsoluteCapacity(abs[i] / tot);
+      qc.setAbsoluteMaximumCapacity(maxCap[i] / tot);
+      when(q.getQueueCapacities()).thenReturn(qc);
+
       String parentPathName = p.getQueuePath();
       parentPathName = (parentPathName == null) ? "root" : parentPathName;
       String queuePathName = (parentPathName+"."+queueName).replace("/","root");
@@ -1028,6 +998,7 @@ public class TestProportionalCapacityPreemptionPolicy {
     return pq;
   }
 
+  @SuppressWarnings("rawtypes")
   LeafQueue mockLeafQueue(ParentQueue p, float tot, int i, int[] abs, 
       int[] used, int[] pending, int[] reserved, int[] apps, int[] gran) {
     LeafQueue lq = mock(LeafQueue.class);
@@ -1035,6 +1006,10 @@ public class TestProportionalCapacityPreemptionPolicy {
         new ArrayList<ApplicationAttemptId>();
     when(lq.getTotalResourcePending()).thenReturn(
         Resource.newInstance(pending[i], 0));
+    // need to set pending resource in resource usage as well
+    ResourceUsage ru = new ResourceUsage();
+    ru.setPending(Resource.newInstance(pending[i], 0));
+    when(lq.getQueueResourceUsage()).thenReturn(ru);
     // consider moving where CapacityScheduler::comparator accessible
     final NavigableSet<FiCaSchedulerApp> qApps = new TreeSet<FiCaSchedulerApp>(
       new Comparator<FiCaSchedulerApp>() {
@@ -1124,6 +1099,7 @@ public class TestProportionalCapacityPreemptionPolicy {
     when(mC.getContainerId()).thenReturn(cId);
     when(mC.getContainer()).thenReturn(c);
     when(mC.getApplicationAttemptId()).thenReturn(appAttId);
+    when(mC.getAllocatedResource()).thenReturn(r);
     if (priority.AMCONTAINER.getValue() == cpriority) {
       when(mC.isAMContainer()).thenReturn(true);
     }


[10/50] [abbrv] hadoop git commit: Updated CHANGES.TXT for correct version of HDFS-8206

Posted by ji...@apache.org.
Updated CHANGES.TXT for correct version of HDFS-8206


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/22b70e7c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/22b70e7c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/22b70e7c

Branch: refs/heads/HDFS-7240
Commit: 22b70e7c5a005b553610820d866763d8096aeca5
Parents: 8f3946c
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Sat Apr 25 22:16:06 2015 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Sat Apr 25 22:16:06 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/22b70e7c/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index a0a81c0..b7199c7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -321,8 +321,6 @@ Trunk (Unreleased)
     HDFS-7673. synthetic load generator docs give incorrect/incomplete commands
     (Brahma Reddy Battula via aw)
 
-    HDFS-8206. Fix the typos in hadoop-hdfs-httpfs. (Brahma Reddy Battula via xyao)
-
 Release 2.8.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -571,6 +569,8 @@ Release 2.8.0 - UNRELEASED
     HDFS-8247. TestDiskspaceQuotaUpdate#testAppendOverTypeQuota is failing.
     (Xiaoyu Yao via cnauroth)
 
+    HDFS-8206. Fix the typos in hadoop-hdfs-httpfs. (Brahma Reddy Battula via xyao)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES


[48/50] [abbrv] hadoop git commit: HDFS-8292. Move conditional in fmt_time from dfs-dust.js to status.html. Contributed by Charles Lamb.

Posted by ji...@apache.org.
HDFS-8292. Move conditional in fmt_time from dfs-dust.js to status.html. Contributed by Charles Lamb.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/87e99782
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/87e99782
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/87e99782

Branch: refs/heads/HDFS-7240
Commit: 87e997823581790cce8d82d20e5e82ef9dd80670
Parents: 5f8112f
Author: Andrew Wang <wa...@apache.org>
Authored: Thu Apr 30 18:23:13 2015 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Thu Apr 30 18:23:13 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                       | 3 +++
 .../hadoop-hdfs/src/main/webapps/secondary/status.html            | 2 +-
 .../hadoop-hdfs/src/main/webapps/static/dfs-dust.js               | 3 ---
 3 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/87e99782/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 29e3f85..7e4a6f2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -488,6 +488,9 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-8200. Refactor FSDirStatAndListingOp. (wheat9)
 
+    HDFS-8292. Move conditional in fmt_time from dfs-dust.js to status.html.
+    (Charles Lamb via wang)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/87e99782/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/status.html
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/status.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/status.html
index 2ef58ad..72f3ff7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/status.html
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/status.html
@@ -65,7 +65,7 @@
   <tr><th>Compiled</th><td>{CompileInfo}</td></tr>
   <tr><th>NameNode Address</th><td>{HostAndPort}</td></tr>
   <tr><th>Started</th><td>{StartTime|date_tostring}</td></tr>
-  <tr><th>Last Checkpoint</th><td>{@if cond="{LastCheckpointTime} === 0"}Never{:else}{LastCheckpointTime|date_tostring} ({LastCheckpointDeltaMs|fmt_time} ago){/if}</td></tr>
+  <tr><th>Last Checkpoint</th><td>{@if cond="{LastCheckpointTime} === 0"}Never{:else}{LastCheckpointTime|date_tostring}{/if} {@if cond="{LastCheckpointDeltaMs} >= 0"} ({LastCheckpointDeltaMs|fmt_time} ago){/if}</td></tr>
   <tr><th>Checkpoint Period</th><td>{CheckpointPeriod} seconds</td></tr>
   <tr><th>Checkpoint Transactions</th><td>{TxnCount}</td></tr>
 </table>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/87e99782/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dfs-dust.js
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dfs-dust.js b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dfs-dust.js
index f483d23..3c8efd9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dfs-dust.js
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dfs-dust.js
@@ -40,9 +40,6 @@
     },
 
     'fmt_time': function (v) {
-      if (v < 0) {
-        return "unknown";
-      }
       var s = Math.floor(v / 1000), h = Math.floor(s / 3600);
       s -= h * 3600;
       var m = Math.floor(s / 60);


[34/50] [abbrv] hadoop git commit: HDFS-8283. DataStreamer cleanup and some minor improvement. Contributed by Tsz Wo Nicholas Sze.

Posted by ji...@apache.org.
HDFS-8283. DataStreamer cleanup and some minor improvement. Contributed by Tsz Wo Nicholas Sze.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/7947e5b5
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7947e5b5
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7947e5b5

Branch: refs/heads/HDFS-7240
Commit: 7947e5b53b9ac9524b535b0384c1c355b74723ff
Parents: 8f82970
Author: Jing Zhao <ji...@apache.org>
Authored: Wed Apr 29 10:41:46 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 10:41:46 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/io/MultipleIOException.java   |  26 ++
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../org/apache/hadoop/hdfs/DFSOutputStream.java |  30 +--
 .../org/apache/hadoop/hdfs/DataStreamer.java    | 235 ++++++++++---------
 .../apache/hadoop/hdfs/TestDFSOutputStream.java |   3 +-
 5 files changed, 162 insertions(+), 135 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7947e5b5/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MultipleIOException.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MultipleIOException.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MultipleIOException.java
index 5e584c9c..66c1ab1 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MultipleIOException.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MultipleIOException.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.io;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -51,4 +52,29 @@ public class MultipleIOException extends IOException {
     }
     return new MultipleIOException(exceptions);
   }
+
+  /**
+   * Build an {@link IOException} using {@link MultipleIOException}
+   * if there are more than one.
+   */
+  public static class Builder {
+    private List<IOException> exceptions;
+    
+    /** Add the given {@link Throwable} to the exception list. */
+    public void add(Throwable t) {
+      if (exceptions == null) {
+        exceptions = new ArrayList<>();
+      }
+      exceptions.add(t instanceof IOException? (IOException)t
+          : new IOException(t));
+    }
+
+    /**
+     * @return null if nothing is added to this builder;
+     *         otherwise, return an {@link IOException}
+     */
+    public IOException build() {
+      return createIOException(exceptions);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7947e5b5/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index e7fa8fd..2dde356 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -480,6 +480,9 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-8280. Code Cleanup in DFSInputStream. (Jing Zhao via wheat9)
 
+    HDFS-8283. DataStreamer cleanup and some minor improvement. (szetszwo via
+    jing9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7947e5b5/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
index d9b8ee7..4646b60 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
@@ -139,8 +139,7 @@ public class DFSOutputStream extends FSOutputSummer
   @Override
   protected void checkClosed() throws IOException {
     if (isClosed()) {
-      IOException e = streamer.getLastException().get();
-      throw e != null ? e : new ClosedChannelException();
+      streamer.getLastException().throwException4Close();
     }
   }
 
@@ -216,10 +215,7 @@ public class DFSOutputStream extends FSOutputSummer
     computePacketChunkSize(dfsClient.getConf().getWritePacketSize(), bytesPerChecksum);
 
     streamer = new DataStreamer(stat, null, dfsClient, src, progress, checksum,
-        cachingStrategy, byteArrayManager);
-    if (favoredNodes != null && favoredNodes.length != 0) {
-      streamer.setFavoredNodes(favoredNodes);
-    }
+        cachingStrategy, byteArrayManager, favoredNodes);
   }
 
   static DFSOutputStream newStreamForCreate(DFSClient dfsClient, String src,
@@ -282,7 +278,8 @@ public class DFSOutputStream extends FSOutputSummer
   /** Construct a new output stream for append. */
   private DFSOutputStream(DFSClient dfsClient, String src,
       EnumSet<CreateFlag> flags, Progressable progress, LocatedBlock lastBlock,
-      HdfsFileStatus stat, DataChecksum checksum) throws IOException {
+      HdfsFileStatus stat, DataChecksum checksum, String[] favoredNodes)
+          throws IOException {
     this(dfsClient, src, progress, stat, checksum);
     initialFileSize = stat.getLen(); // length of file when opened
     this.shouldSyncBlock = flags.contains(CreateFlag.SYNC_BLOCK);
@@ -303,7 +300,8 @@ public class DFSOutputStream extends FSOutputSummer
       computePacketChunkSize(dfsClient.getConf().getWritePacketSize(),
           bytesPerChecksum);
       streamer = new DataStreamer(stat, lastBlock != null ? lastBlock.getBlock() : null,
-          dfsClient, src, progress, checksum, cachingStrategy, byteArrayManager);
+          dfsClient, src, progress, checksum, cachingStrategy, byteArrayManager,
+          favoredNodes);
     }
   }
 
@@ -351,10 +349,7 @@ public class DFSOutputStream extends FSOutputSummer
         dfsClient.getPathTraceScope("newStreamForAppend", src);
     try {
       final DFSOutputStream out = new DFSOutputStream(dfsClient, src, flags,
-          progress, lastBlock, stat, checksum);
-      if (favoredNodes != null && favoredNodes.length != 0) {
-        out.streamer.setFavoredNodes(favoredNodes);
-      }
+          progress, lastBlock, stat, checksum, favoredNodes);
       out.start();
       return out;
     } finally {
@@ -653,7 +648,7 @@ public class DFSOutputStream extends FSOutputSummer
       DFSClient.LOG.warn("Error while syncing", e);
       synchronized (this) {
         if (!isClosed()) {
-          streamer.getLastException().set(new IOException("IOException flush: " + e));
+          streamer.getLastException().set(e);
           closeThreads(true);
         }
       }
@@ -720,7 +715,7 @@ public class DFSOutputStream extends FSOutputSummer
     if (isClosed()) {
       return;
     }
-    streamer.setLastException(new IOException("Lease timeout of "
+    streamer.getLastException().set(new IOException("Lease timeout of "
         + (dfsClient.getConf().getHdfsTimeout()/1000) + " seconds expired."));
     closeThreads(true);
     dfsClient.endFileLease(fileId);
@@ -767,11 +762,8 @@ public class DFSOutputStream extends FSOutputSummer
 
   protected synchronized void closeImpl() throws IOException {
     if (isClosed()) {
-      IOException e = streamer.getLastException().getAndSet(null);
-      if (e == null)
-        return;
-      else
-        throw e;
+      streamer.getLastException().check();
+      return;
     }
 
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7947e5b5/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
index 5f0c9ac..3727d20 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
@@ -41,6 +41,8 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
@@ -73,6 +75,7 @@ import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
 import org.apache.hadoop.hdfs.util.ByteArrayManager;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.MultipleIOException;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.AccessControlException;
@@ -88,6 +91,7 @@ import org.apache.htrace.Trace;
 import org.apache.htrace.TraceInfo;
 import org.apache.htrace.TraceScope;
 
+import com.google.common.base.Preconditions;
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
@@ -117,6 +121,7 @@ import com.google.common.cache.RemovalNotification;
 
 @InterfaceAudience.Private
 class DataStreamer extends Daemon {
+  static final Log LOG = LogFactory.getLog(DataStreamer.class);
   /**
    * Create a socket for a write pipeline
    *
@@ -129,8 +134,8 @@ class DataStreamer extends Daemon {
       final int length, final DFSClient client) throws IOException {
     final DfsClientConf conf = client.getConf();
     final String dnAddr = first.getXferAddr(conf.isConnectToDnViaHostname());
-    if (DFSClient.LOG.isDebugEnabled()) {
-      DFSClient.LOG.debug("Connecting to datanode " + dnAddr);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Connecting to datanode " + dnAddr);
     }
     final InetSocketAddress isa = NetUtils.createSocketAddr(dnAddr);
     final Socket sock = client.socketFactory.createSocket();
@@ -138,8 +143,8 @@ class DataStreamer extends Daemon {
     NetUtils.connect(sock, isa, client.getRandomLocalInterfaceAddr(), conf.getSocketTimeout());
     sock.setSoTimeout(timeout);
     sock.setSendBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
-    if(DFSClient.LOG.isDebugEnabled()) {
-      DFSClient.LOG.debug("Send buf size " + sock.getSendBufferSize());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Send buf size " + sock.getSendBufferSize());
     }
     return sock;
   }
@@ -168,6 +173,34 @@ class DataStreamer extends Daemon {
     }
     packets.clear();
   }
+  
+  static class LastException {
+    private Throwable thrown;
+
+    synchronized void set(Throwable t) {
+      Preconditions.checkNotNull(t);
+      Preconditions.checkState(thrown == null);
+      this.thrown = t;
+    }
+
+    synchronized void clear() {
+      thrown = null;
+    }
+
+    /** Check if there already is an exception. */
+    synchronized void check() throws IOException {
+      if (thrown != null) {
+        throw new IOException(thrown);
+      }
+    }
+
+    synchronized void throwException4Close() throws IOException {
+      check();
+      final IOException ioe = new ClosedChannelException();
+      thrown = ioe;
+      throw ioe;
+    }
+  }
 
   private volatile boolean streamerClosed = false;
   private ExtendedBlock block; // its length is number of bytes acked
@@ -178,7 +211,6 @@ class DataStreamer extends Daemon {
   private volatile DatanodeInfo[] nodes = null; // list of targets for current block
   private volatile StorageType[] storageTypes = null;
   private volatile String[] storageIDs = null;
-  private String[] favoredNodes;
   volatile boolean hasError = false;
   volatile int errorIndex = -1;
   // Restarting node index
@@ -196,13 +228,13 @@ class DataStreamer extends Daemon {
   /** Has the current block been hflushed? */
   private boolean isHflushed = false;
   /** Append on an existing block? */
-  private boolean isAppend;
+  private final boolean isAppend;
 
   private long currentSeqno = 0;
   private long lastQueuedSeqno = -1;
   private long lastAckedSeqno = -1;
   private long bytesCurBlock = 0; // bytes written in current block
-  private final AtomicReference<IOException> lastException = new AtomicReference<>();
+  private final LastException lastException = new LastException();
   private Socket s;
 
   private final DFSClient dfsClient;
@@ -227,18 +259,20 @@ class DataStreamer extends Daemon {
   private long artificialSlowdown = 0;
   // List of congested data nodes. The stream will back off if the DataNodes
   // are congested
-  private final ArrayList<DatanodeInfo> congestedNodes = new ArrayList<>();
+  private final List<DatanodeInfo> congestedNodes = new ArrayList<>();
   private static final int CONGESTION_BACKOFF_MEAN_TIME_IN_MS = 5000;
   private static final int CONGESTION_BACK_OFF_MAX_TIME_IN_MS =
       CONGESTION_BACKOFF_MEAN_TIME_IN_MS * 10;
   private int lastCongestionBackoffTime;
 
   private final LoadingCache<DatanodeInfo, DatanodeInfo> excludedNodes;
+  private final String[] favoredNodes;
 
   private DataStreamer(HdfsFileStatus stat, DFSClient dfsClient, String src,
                        Progressable progress, DataChecksum checksum,
                        AtomicReference<CachingStrategy> cachingStrategy,
-                       ByteArrayManager byteArrayManage){
+                       ByteArrayManager byteArrayManage,
+                       boolean isAppend, String[] favoredNodes) {
     this.dfsClient = dfsClient;
     this.src = src;
     this.progress = progress;
@@ -246,10 +280,12 @@ class DataStreamer extends Daemon {
     this.checksum4WriteBlock = checksum;
     this.cachingStrategy = cachingStrategy;
     this.byteArrayManager = byteArrayManage;
-    isLazyPersistFile = isLazyPersist(stat);
+    this.isLazyPersistFile = isLazyPersist(stat);
     this.dfsclientSlowLogThresholdMs =
         dfsClient.getConf().getSlowIoWarningThresholdMs();
-    excludedNodes = initExcludedNodes();
+    this.excludedNodes = initExcludedNodes();
+    this.isAppend = isAppend;
+    this.favoredNodes = favoredNodes;
   }
 
   /**
@@ -258,10 +294,9 @@ class DataStreamer extends Daemon {
   DataStreamer(HdfsFileStatus stat, ExtendedBlock block, DFSClient dfsClient,
                String src, Progressable progress, DataChecksum checksum,
                AtomicReference<CachingStrategy> cachingStrategy,
-               ByteArrayManager byteArrayManage) {
+               ByteArrayManager byteArrayManage, String[] favoredNodes) {
     this(stat, dfsClient, src, progress, checksum, cachingStrategy,
-        byteArrayManage);
-    isAppend = false;
+        byteArrayManage, false, favoredNodes);
     this.block = block;
     stage = BlockConstructionStage.PIPELINE_SETUP_CREATE;
   }
@@ -277,8 +312,7 @@ class DataStreamer extends Daemon {
                AtomicReference<CachingStrategy> cachingStrategy,
                ByteArrayManager byteArrayManage) throws IOException {
     this(stat, dfsClient, src, progress, checksum, cachingStrategy,
-        byteArrayManage);
-    isAppend = true;
+        byteArrayManage, true, null);
     stage = BlockConstructionStage.PIPELINE_SETUP_APPEND;
     block = lastBlock.getBlock();
     bytesSent = block.getNumBytes();
@@ -314,15 +348,6 @@ class DataStreamer extends Daemon {
   }
 
   /**
-   * Set favored nodes
-   *
-   * @param favoredNodes favored nodes
-   */
-  void setFavoredNodes(String[] favoredNodes) {
-    this.favoredNodes = favoredNodes;
-  }
-
-  /**
    * Initialize for data streaming
    */
   private void initDataStreaming() {
@@ -334,8 +359,8 @@ class DataStreamer extends Daemon {
   }
 
   private void endBlock() {
-    if(DFSClient.LOG.isDebugEnabled()) {
-      DFSClient.LOG.debug("Closing old block " + block);
+    if(LOG.isDebugEnabled()) {
+      LOG.debug("Closing old block " + block);
     }
     this.setName("DataStreamer for file " + src);
     closeResponder();
@@ -360,7 +385,7 @@ class DataStreamer extends Daemon {
           response.join();
           response = null;
         } catch (InterruptedException  e) {
-          DFSClient.LOG.warn("Caught exception ", e);
+          LOG.warn("Caught exception", e);
         }
       }
 
@@ -388,7 +413,7 @@ class DataStreamer extends Daemon {
             try {
               dataQueue.wait(timeout);
             } catch (InterruptedException  e) {
-              DFSClient.LOG.warn("Caught exception ", e);
+              LOG.warn("Caught exception", e);
             }
             doSleep = false;
             now = Time.monotonicNow();
@@ -404,7 +429,7 @@ class DataStreamer extends Daemon {
             try {
               backOffIfNecessary();
             } catch (InterruptedException e) {
-              DFSClient.LOG.warn("Caught exception ", e);
+              LOG.warn("Caught exception", e);
             }
             one = dataQueue.getFirst(); // regular data packet
             long parents[] = one.getTraceParents();
@@ -419,14 +444,14 @@ class DataStreamer extends Daemon {
 
         // get new block from namenode.
         if (stage == BlockConstructionStage.PIPELINE_SETUP_CREATE) {
-          if(DFSClient.LOG.isDebugEnabled()) {
-            DFSClient.LOG.debug("Allocating new block");
+          if(LOG.isDebugEnabled()) {
+            LOG.debug("Allocating new block");
           }
           setPipeline(nextBlockOutputStream());
           initDataStreaming();
         } else if (stage == BlockConstructionStage.PIPELINE_SETUP_APPEND) {
-          if(DFSClient.LOG.isDebugEnabled()) {
-            DFSClient.LOG.debug("Append to block " + block);
+          if(LOG.isDebugEnabled()) {
+            LOG.debug("Append to block " + block);
           }
           setupPipelineForAppendOrRecovery();
           initDataStreaming();
@@ -450,7 +475,7 @@ class DataStreamer extends Daemon {
                 // wait for acks to arrive from datanodes
                 dataQueue.wait(1000);
               } catch (InterruptedException  e) {
-                DFSClient.LOG.warn("Caught exception ", e);
+                LOG.warn("Caught exception", e);
               }
             }
           }
@@ -473,8 +498,8 @@ class DataStreamer extends Daemon {
           }
         }
 
-        if (DFSClient.LOG.isDebugEnabled()) {
-          DFSClient.LOG.debug("DataStreamer block " + block +
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("DataStreamer block " + block +
               " sending packet " + one);
         }
 
@@ -534,16 +559,12 @@ class DataStreamer extends Daemon {
           // Since their messages are descriptive enough, do not always
           // log a verbose stack-trace WARN for quota exceptions.
           if (e instanceof QuotaExceededException) {
-            DFSClient.LOG.debug("DataStreamer Quota Exception", e);
+            LOG.debug("DataStreamer Quota Exception", e);
           } else {
-            DFSClient.LOG.warn("DataStreamer Exception", e);
+            LOG.warn("DataStreamer Exception", e);
           }
         }
-        if (e instanceof IOException) {
-          setLastException((IOException)e);
-        } else {
-          setLastException(new IOException("DataStreamer Exception: ",e));
-        }
+        lastException.set(e);
         hasError = true;
         if (errorIndex == -1 && restartingNodeIndex.get() == -1) {
           // Not a datanode issue
@@ -586,8 +607,8 @@ class DataStreamer extends Daemon {
   void waitForAckedSeqno(long seqno) throws IOException {
     TraceScope scope = Trace.startSpan("waitForAckedSeqno", Sampler.NEVER);
     try {
-      if (DFSClient.LOG.isDebugEnabled()) {
-        DFSClient.LOG.debug("Waiting for ack for: " + seqno);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Waiting for ack for: " + seqno);
       }
       long begin = Time.monotonicNow();
       try {
@@ -611,7 +632,7 @@ class DataStreamer extends Daemon {
       }
       long duration = Time.monotonicNow() - begin;
       if (duration > dfsclientSlowLogThresholdMs) {
-        DFSClient.LOG.warn("Slow waitForAckedSeqno took " + duration
+        LOG.warn("Slow waitForAckedSeqno took " + duration
             + "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms)");
       }
     } finally {
@@ -688,8 +709,7 @@ class DataStreamer extends Daemon {
 
   private void checkClosed() throws IOException {
     if (streamerClosed) {
-      IOException e = lastException.get();
-      throw e != null ? e : new ClosedChannelException();
+      lastException.throwException4Close();
     }
   }
 
@@ -699,7 +719,7 @@ class DataStreamer extends Daemon {
         response.close();
         response.join();
       } catch (InterruptedException  e) {
-        DFSClient.LOG.warn("Caught exception ", e);
+        LOG.warn("Caught exception", e);
       } finally {
         response = null;
       }
@@ -707,11 +727,13 @@ class DataStreamer extends Daemon {
   }
 
   private void closeStream() {
+    final MultipleIOException.Builder b = new MultipleIOException.Builder();
+
     if (blockStream != null) {
       try {
         blockStream.close();
       } catch (IOException e) {
-        setLastException(e);
+        b.add(e);
       } finally {
         blockStream = null;
       }
@@ -720,7 +742,7 @@ class DataStreamer extends Daemon {
       try {
         blockReplyStream.close();
       } catch (IOException e) {
-        setLastException(e);
+        b.add(e);
       } finally {
         blockReplyStream = null;
       }
@@ -729,11 +751,16 @@ class DataStreamer extends Daemon {
       try {
         s.close();
       } catch (IOException e) {
-        setLastException(e);
+        b.add(e);
       } finally {
         s = null;
       }
     }
+
+    final IOException ioe = b.build();
+    if (ioe != null) {
+      lastException.set(ioe);
+    }
   }
 
   // The following synchronized methods are used whenever
@@ -825,12 +852,11 @@ class DataStreamer extends Daemon {
           long duration = Time.monotonicNow() - begin;
           if (duration > dfsclientSlowLogThresholdMs
               && ack.getSeqno() != DFSPacket.HEART_BEAT_SEQNO) {
-            DFSClient.LOG
-                .warn("Slow ReadProcessor read fields took " + duration
-                    + "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms); ack: "
-                    + ack + ", targets: " + Arrays.asList(targets));
-          } else if (DFSClient.LOG.isDebugEnabled()) {
-            DFSClient.LOG.debug("DFSClient " + ack);
+            LOG.warn("Slow ReadProcessor read fields took " + duration
+                + "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms); ack: "
+                + ack + ", targets: " + Arrays.asList(targets));
+          } else if (LOG.isDebugEnabled()) {
+            LOG.debug("DFSClient " + ack);
           }
 
           long seqno = ack.getSeqno();
@@ -851,7 +877,7 @@ class DataStreamer extends Daemon {
                   + Time.monotonicNow();
               setRestartingNodeIndex(i);
               String message = "A datanode is restarting: " + targets[i];
-              DFSClient.LOG.info(message);
+              LOG.info(message);
               throw new IOException(message);
             }
             // node error
@@ -917,9 +943,7 @@ class DataStreamer extends Daemon {
           }
         } catch (Exception e) {
           if (!responderClosed) {
-            if (e instanceof IOException) {
-              setLastException((IOException)e);
-            }
+            lastException.set(e);
             hasError = true;
             // If no explicit error report was received, mark the primary
             // node as failed.
@@ -928,8 +952,7 @@ class DataStreamer extends Daemon {
               dataQueue.notifyAll();
             }
             if (restartingNodeIndex.get() == -1) {
-              DFSClient.LOG.warn("DataStreamer ResponseProcessor exception "
-                  + " for block " + block, e);
+              LOG.warn("Exception for " + block, e);
             }
             responderClosed = true;
           }
@@ -951,7 +974,7 @@ class DataStreamer extends Daemon {
   //
   private boolean processDatanodeError() throws IOException {
     if (response != null) {
-      DFSClient.LOG.info("Error Recovery for " + block +
+      LOG.info("Error Recovery for " + block +
           " waiting for responder to exit. ");
       return true;
     }
@@ -972,7 +995,7 @@ class DataStreamer extends Daemon {
       // same packet, this client likely has corrupt data or corrupting
       // during transmission.
       if (++pipelineRecoveryCount > 5) {
-        DFSClient.LOG.warn("Error recovering pipeline for writing " +
+        LOG.warn("Error recovering pipeline for writing " +
             block + ". Already retried 5 times for the same packet.");
         lastException.set(new IOException("Failing write. Tried pipeline " +
             "recovery 5 times without success."));
@@ -1147,8 +1170,8 @@ class DataStreamer extends Daemon {
     if (nodes == null || nodes.length == 0) {
       String msg = "Could not get block locations. " + "Source file \""
           + src + "\" - Aborting...";
-      DFSClient.LOG.warn(msg);
-      setLastException(new IOException(msg));
+      LOG.warn(msg);
+      lastException.set(new IOException(msg));
       streamerClosed = true;
       return false;
     }
@@ -1193,7 +1216,7 @@ class DataStreamer extends Daemon {
           streamerClosed = true;
           return false;
         }
-        DFSClient.LOG.warn("Error Recovery for block " + block +
+        LOG.warn("Error Recovery for block " + block +
             " in pipeline " + pipelineMsg +
             ": bad datanode " + nodes[errorIndex]);
         failed.add(nodes[errorIndex]);
@@ -1227,7 +1250,7 @@ class DataStreamer extends Daemon {
         if (restartingNodeIndex.get() == -1) {
           hasError = false;
         }
-        lastException.set(null);
+        lastException.clear();
         errorIndex = -1;
       }
 
@@ -1240,7 +1263,7 @@ class DataStreamer extends Daemon {
           if (!dfsClient.dtpReplaceDatanodeOnFailure.isBestEffort()) {
             throw ioe;
           }
-          DFSClient.LOG.warn("Failed to replace datanode."
+          LOG.warn("Failed to replace datanode."
               + " Continue with the remaining datanodes since "
               + HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_KEY
               + " is set to true.", ioe);
@@ -1281,7 +1304,7 @@ class DataStreamer extends Daemon {
         restartDeadline = 0;
         int expiredNodeIndex = restartingNodeIndex.get();
         restartingNodeIndex.set(-1);
-        DFSClient.LOG.warn("Datanode did not restart in time: " +
+        LOG.warn("Datanode did not restart in time: " +
             nodes[expiredNodeIndex]);
         // Mark the restarting node as failed. If there is any other failed
         // node during the last pipeline construction attempt, it will not be
@@ -1321,7 +1344,7 @@ class DataStreamer extends Daemon {
     ExtendedBlock oldBlock = block;
     do {
       hasError = false;
-      lastException.set(null);
+      lastException.clear();
       errorIndex = -1;
       success = false;
 
@@ -1344,11 +1367,11 @@ class DataStreamer extends Daemon {
       success = createBlockOutputStream(nodes, storageTypes, 0L, false);
 
       if (!success) {
-        DFSClient.LOG.info("Abandoning " + block);
+        LOG.info("Abandoning " + block);
         dfsClient.namenode.abandonBlock(block, stat.getFileId(), src,
             dfsClient.clientName);
         block = null;
-        DFSClient.LOG.info("Excluding datanode " + nodes[errorIndex]);
+        LOG.info("Excluding datanode " + nodes[errorIndex]);
         excludedNodes.put(nodes[errorIndex], nodes[errorIndex]);
       }
     } while (!success && --count >= 0);
@@ -1365,17 +1388,14 @@ class DataStreamer extends Daemon {
   private boolean createBlockOutputStream(DatanodeInfo[] nodes,
       StorageType[] nodeStorageTypes, long newGS, boolean recoveryFlag) {
     if (nodes.length == 0) {
-      DFSClient.LOG.info("nodes are empty for write pipeline of block "
-          + block);
+      LOG.info("nodes are empty for write pipeline of " + block);
       return false;
     }
     Status pipelineStatus = SUCCESS;
     String firstBadLink = "";
     boolean checkRestart = false;
-    if (DFSClient.LOG.isDebugEnabled()) {
-      for (int i = 0; i < nodes.length; i++) {
-        DFSClient.LOG.debug("pipeline = " + nodes[i]);
-      }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("pipeline = " + Arrays.asList(nodes));
     }
 
     // persist blocks on namenode on next flush
@@ -1447,10 +1467,10 @@ class DataStreamer extends Daemon {
         hasError = false;
       } catch (IOException ie) {
         if (restartingNodeIndex.get() == -1) {
-          DFSClient.LOG.info("Exception in createBlockOutputStream", ie);
+          LOG.info("Exception in createBlockOutputStream", ie);
         }
         if (ie instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
-          DFSClient.LOG.info("Will fetch a new encryption key and retry, "
+          LOG.info("Will fetch a new encryption key and retry, "
               + "encryption key was invalid when connecting to "
               + nodes[0] + " : " + ie);
           // The encryption key used is invalid.
@@ -1480,11 +1500,11 @@ class DataStreamer extends Daemon {
               + Time.monotonicNow();
           restartingNodeIndex.set(errorIndex);
           errorIndex = -1;
-          DFSClient.LOG.info("Waiting for the datanode to be restarted: " +
+          LOG.info("Waiting for the datanode to be restarted: " +
               nodes[restartingNodeIndex.get()]);
         }
         hasError = true;
-        setLastException(ie);
+        lastException.set(ie);
         result =  false;  // error
       } finally {
         if (!result) {
@@ -1509,18 +1529,16 @@ class DataStreamer extends Daemon {
           new HashSet<String>(Arrays.asList(favoredNodes));
       for (int i = 0; i < nodes.length; i++) {
         pinnings[i] = favoredSet.remove(nodes[i].getXferAddrWithHostname());
-        if (DFSClient.LOG.isDebugEnabled()) {
-          DFSClient.LOG.debug(nodes[i].getXferAddrWithHostname() +
-              " was chosen by name node (favored=" + pinnings[i] +
-              ").");
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(nodes[i].getXferAddrWithHostname() +
+              " was chosen by name node (favored=" + pinnings[i] + ").");
         }
       }
       if (shouldLog && !favoredSet.isEmpty()) {
         // There is one or more favored nodes that were not allocated.
-        DFSClient.LOG.warn(
-            "These favored nodes were specified but not chosen: " +
-                favoredSet +
-                " Specified favored nodes: " + Arrays.toString(favoredNodes));
+        LOG.warn("These favored nodes were specified but not chosen: "
+            + favoredSet + " Specified favored nodes: "
+            + Arrays.toString(favoredNodes));
 
       }
       return pinnings;
@@ -1557,19 +1575,19 @@ class DataStreamer extends Daemon {
               throw e;
             } else {
               --retries;
-              DFSClient.LOG.info("Exception while adding a block", e);
+              LOG.info("Exception while adding a block", e);
               long elapsed = Time.monotonicNow() - localstart;
               if (elapsed > 5000) {
-                DFSClient.LOG.info("Waiting for replication for "
+                LOG.info("Waiting for replication for "
                     + (elapsed / 1000) + " seconds");
               }
               try {
-                DFSClient.LOG.warn("NotReplicatedYetException sleeping " + src
+                LOG.warn("NotReplicatedYetException sleeping " + src
                     + " retries left " + retries);
                 Thread.sleep(sleeptime);
                 sleeptime *= 2;
               } catch (InterruptedException ie) {
-                DFSClient.LOG.warn("Caught exception ", ie);
+                LOG.warn("Caught exception", ie);
               }
             }
           } else {
@@ -1606,7 +1624,7 @@ class DataStreamer extends Daemon {
                      (int)(base + Math.random() * range));
         lastCongestionBackoffTime = t;
         sb.append(" are congested. Backing off for ").append(t).append(" ms");
-        DFSClient.LOG.info(sb.toString());
+        LOG.info(sb.toString());
         congestedNodes.clear();
       }
     }
@@ -1643,15 +1661,6 @@ class DataStreamer extends Daemon {
   }
 
   /**
-   * set last exception
-   *
-   * @param e an exception
-   */
-  void setLastException(IOException e) {
-    lastException.compareAndSet(null, e);
-  }
-
-  /**
    * Put a packet to the data queue
    *
    * @param packet the packet to be put into the data queued
@@ -1662,8 +1671,8 @@ class DataStreamer extends Daemon {
       packet.addTraceParent(Trace.currentSpan());
       dataQueue.addLast(packet);
       lastQueuedSeqno = packet.getSeqno();
-      if (DFSClient.LOG.isDebugEnabled()) {
-        DFSClient.LOG.debug("Queued packet " + packet.getSeqno());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Queued packet " + packet.getSeqno());
       }
       dataQueue.notifyAll();
     }
@@ -1686,7 +1695,7 @@ class DataStreamer extends Daemon {
           @Override
           public void onRemoval(
               RemovalNotification<DatanodeInfo, DatanodeInfo> notification) {
-            DFSClient.LOG.info("Removing node " + notification.getKey()
+            LOG.info("Removing node " + notification.getKey()
                 + " from the excluded nodes list");
           }
         }).build(new CacheLoader<DatanodeInfo, DatanodeInfo>() {
@@ -1730,11 +1739,9 @@ class DataStreamer extends Daemon {
   }
 
   /**
-   * get the last exception
-   *
    * @return the last exception
    */
-  AtomicReference<IOException> getLastException(){
+  LastException getLastException(){
     return lastException;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7947e5b5/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
index 478f7e5..eac1fcd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
@@ -62,7 +62,6 @@ public class TestDFSOutputStream {
     FSDataOutputStream os = fs.create(new Path("/test"));
     DFSOutputStream dos = (DFSOutputStream) Whitebox.getInternalState(os,
         "wrappedStream");
-    @SuppressWarnings("unchecked")
     DataStreamer streamer = (DataStreamer) Whitebox
         .getInternalState(dos, "streamer");
     @SuppressWarnings("unchecked")
@@ -122,7 +121,7 @@ public class TestDFSOutputStream {
         mock(HdfsFileStatus.class),
         mock(ExtendedBlock.class),
         client,
-        "foo", null, null, null, null);
+        "foo", null, null, null, null, null);
 
     DataOutputStream blockStream = mock(DataOutputStream.class);
     doThrow(new IOException()).when(blockStream).flush();


[03/50] [abbrv] hadoop git commit: YARN-2498. Respect labels in preemption policy of capacity scheduler for inter-queue preemption. Contributed by Wangda Tan

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d497f6ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java
new file mode 100644
index 0000000..e13320c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java
@@ -0,0 +1,1211 @@
+/**
+ * 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.resourcemanager.monitor.capacity;
+
+import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.MONITORING_INTERVAL;
+import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.NATURAL_TERMINATION_FACTOR;
+import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.TOTAL_PREEMPTION_PER_ROUND;
+import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.WAIT_TIME_BEFORE_KILL;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.argThat;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeSet;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+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.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerPreemptEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.OrderingPolicy;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.Resources;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+public class TestProportionalCapacityPreemptionPolicyForNodePartitions {
+  private static final Log LOG =
+      LogFactory.getLog(TestProportionalCapacityPreemptionPolicyForNodePartitions.class);
+  static final String ROOT = CapacitySchedulerConfiguration.ROOT;
+
+  private Map<String, CSQueue> nameToCSQueues = null;
+  private Map<String, Resource> partitionToResource = null;
+  private Map<NodeId, SchedulerNode> nodeIdToSchedulerNodes = null;
+  private RMNodeLabelsManager nlm = null;
+  private RMContext rmContext = null;
+
+  private ResourceCalculator rc = new DefaultResourceCalculator();
+  private Clock mClock = null;
+  private Configuration conf = null;
+  private CapacitySchedulerConfiguration csConf = null;
+  private CapacityScheduler cs = null;
+  private EventHandler<ContainerPreemptEvent> mDisp = null;
+  private ProportionalCapacityPreemptionPolicy policy = null;
+  private Resource clusterResource = null;
+
+  @SuppressWarnings("unchecked")
+  @Before
+  public void setup() {
+    org.apache.log4j.Logger.getRootLogger().setLevel(
+        org.apache.log4j.Level.DEBUG);
+
+    conf = new Configuration(false);
+    conf.setLong(WAIT_TIME_BEFORE_KILL, 10000);
+    conf.setLong(MONITORING_INTERVAL, 3000);
+    // report "ideal" preempt
+    conf.setFloat(TOTAL_PREEMPTION_PER_ROUND, (float) 1.0);
+    conf.setFloat(NATURAL_TERMINATION_FACTOR, (float) 1.0);
+    conf.set(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES,
+        ProportionalCapacityPreemptionPolicy.class.getCanonicalName());
+    conf.setBoolean(YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS, true);
+    // FairScheduler doesn't support this test,
+    // Set CapacityScheduler as the scheduler for this test.
+    conf.set("yarn.resourcemanager.scheduler.class",
+        CapacityScheduler.class.getName());
+
+    mClock = mock(Clock.class);
+    cs = mock(CapacityScheduler.class);
+    when(cs.getResourceCalculator()).thenReturn(rc);
+
+    nlm = mock(RMNodeLabelsManager.class);
+    mDisp = mock(EventHandler.class);
+
+    rmContext = mock(RMContext.class);
+    when(rmContext.getNodeLabelManager()).thenReturn(nlm);
+    csConf = new CapacitySchedulerConfiguration();
+    when(cs.getConfiguration()).thenReturn(csConf);
+    when(cs.getRMContext()).thenReturn(rmContext);
+
+    policy = new ProportionalCapacityPreemptionPolicy(conf, mDisp, cs, mClock);
+    partitionToResource = new HashMap<>();
+    nodeIdToSchedulerNodes = new HashMap<>();
+    nameToCSQueues = new HashMap<>();
+  }
+
+  @Test
+  public void testBuilder() throws Exception {
+    /**
+     * Test of test, make sure we build expected mock schedulable objects
+     */
+    String labelsConfig =
+        "=200,true;" + // default partition
+        "red=100,false;" + // partition=red
+        "blue=200,true"; // partition=blue
+    String nodesConfig =
+        "n1=red;" + // n1 has partition=red
+        "n2=blue;" + // n2 has partition=blue
+        "n3="; // n3 doesn't have partition
+    String queuesConfig =
+        // guaranteed,max,used,pending
+        "root(=[200 200 100 100],red=[100 100 100 100],blue=[200 200 200 200]);" + //root
+        "-a(=[100 200 100 100],red=[0 0 0 0],blue=[200 200 200 200]);" + // a
+        "--a1(=[50 100 50 100],red=[0 0 0 0],blue=[100 200 200 0]);" + // a1
+        "--a2(=[50 200 50 0],red=[0 0 0 0],blue=[100 200 0 200]);" + // a2
+        "-b(=[100 200 0 0],red=[100 100 100 100],blue=[0 0 0 0])";
+    String appsConfig=
+        //queueName\t(priority,resource,host,expression,#repeat,reserved)
+        // app1 in a1, , 50 in n2 (reserved), 50 in n2 (allocated)
+        "a1\t" // app1 in a1
+        + "(1,1,n3,red,50,false);" + // 50 * default in n3
+
+        "a1\t" // app2 in a1
+        + "(2,1,n2,,50,true)(2,1,n2,,50,false)" // 50 * ignore-exclusivity (reserved),
+                                                // 50 * ignore-exclusivity (allocated)
+        + "(2,1,n2,blue,50,true)(2,1,n2,blue,50,true);" + // 50 in n2 (reserved),
+                                                          // 50 in n2 (allocated)
+        "a2\t" // app3 in a2
+        + "(1,1,n3,red,50,false);" + // 50 * default in n3
+
+        "b\t" // app4 in b
+        + "(1,1,n1,red,100,false);";
+
+    buildEnv(labelsConfig, nodesConfig, queuesConfig, appsConfig);
+
+    // Check queues:
+    // root
+    checkAbsCapacities(cs.getQueue("root"), "", 1f, 1f, 0.5f);
+    checkPendingResource(cs.getQueue("root"), "", 100);
+    checkAbsCapacities(cs.getQueue("root"), "red", 1f, 1f, 1f);
+    checkPendingResource(cs.getQueue("root"), "red", 100);
+    checkAbsCapacities(cs.getQueue("root"), "blue", 1f, 1f, 1f);
+    checkPendingResource(cs.getQueue("root"), "blue", 200);
+
+    // a
+    checkAbsCapacities(cs.getQueue("a"), "", 0.5f, 1f, 0.5f);
+    checkPendingResource(cs.getQueue("a"), "", 100);
+    checkAbsCapacities(cs.getQueue("a"), "red", 0f, 0f, 0f);
+    checkPendingResource(cs.getQueue("a"), "red", 0);
+    checkAbsCapacities(cs.getQueue("a"), "blue", 1f, 1f, 1f);
+    checkPendingResource(cs.getQueue("a"), "blue", 200);
+
+    // a1
+    checkAbsCapacities(cs.getQueue("a1"), "", 0.25f, 0.5f, 0.25f);
+    checkPendingResource(cs.getQueue("a1"), "", 100);
+    checkAbsCapacities(cs.getQueue("a1"), "red", 0f, 0f, 0f);
+    checkPendingResource(cs.getQueue("a1"), "red", 0);
+    checkAbsCapacities(cs.getQueue("a1"), "blue", 0.5f, 1f, 1f);
+    checkPendingResource(cs.getQueue("a1"), "blue", 0);
+
+    // a2
+    checkAbsCapacities(cs.getQueue("a2"), "", 0.25f, 1f, 0.25f);
+    checkPendingResource(cs.getQueue("a2"), "", 0);
+    checkAbsCapacities(cs.getQueue("a2"), "red", 0f, 0f, 0f);
+    checkPendingResource(cs.getQueue("a2"), "red", 0);
+    checkAbsCapacities(cs.getQueue("a2"), "blue", 0.5f, 1f, 0f);
+    checkPendingResource(cs.getQueue("a2"), "blue", 200);
+
+    // b1
+    checkAbsCapacities(cs.getQueue("b"), "", 0.5f, 1f, 0f);
+    checkPendingResource(cs.getQueue("b"), "", 0);
+    checkAbsCapacities(cs.getQueue("b"), "red", 1f, 1f, 1f);
+    checkPendingResource(cs.getQueue("b"), "red", 100);
+    checkAbsCapacities(cs.getQueue("b"), "blue", 0f, 0f, 0f);
+    checkPendingResource(cs.getQueue("b"), "blue", 0);
+
+    // Check ignored partitioned containers in queue
+    Assert.assertEquals(100, ((LeafQueue) cs.getQueue("a1"))
+        .getIgnoreExclusivityRMContainers().get("blue").size());
+
+    // Check applications
+    Assert.assertEquals(2, ((LeafQueue)cs.getQueue("a1")).getApplications().size());
+    Assert.assertEquals(1, ((LeafQueue)cs.getQueue("a2")).getApplications().size());
+    Assert.assertEquals(1, ((LeafQueue)cs.getQueue("b")).getApplications().size());
+
+    // Check #containers
+    FiCaSchedulerApp app1 = getApp("a1", 1);
+    FiCaSchedulerApp app2 = getApp("a1", 2);
+    FiCaSchedulerApp app3 = getApp("a2", 3);
+    FiCaSchedulerApp app4 = getApp("b", 4);
+
+    Assert.assertEquals(50, app1.getLiveContainers().size());
+    checkContainerNodesInApp(app1, 50, "n3");
+
+    Assert.assertEquals(50, app2.getLiveContainers().size());
+    Assert.assertEquals(150, app2.getReservedContainers().size());
+    checkContainerNodesInApp(app2, 200, "n2");
+
+    Assert.assertEquals(50, app3.getLiveContainers().size());
+    checkContainerNodesInApp(app3, 50, "n3");
+
+    Assert.assertEquals(100, app4.getLiveContainers().size());
+    checkContainerNodesInApp(app4, 100, "n1");
+  }
+
+  @Test
+  public void testNodePartitionPreemptionRespectGuaranteedCapacity()
+      throws IOException {
+    /**
+     * The simplest test of node label, Queue structure is:
+     *
+     * <pre>
+     *       root
+     *       /  \
+     *      a    b
+     * </pre>
+     *
+     * Both a/b can access x, and guaranteed capacity of them is 50:50. Two
+     * nodes, n1 has 100 x, n2 has 100 NO_LABEL 4 applications in the cluster,
+     * app1/app2 in a, and app3/app4 in b.
+     * app1 uses 80 x, app2 uses 20 NO_LABEL, app3 uses 20 x, app4 uses 80 NO_LABEL.
+     * Both a/b have 50 pending resource for x and NO_LABEL
+     *
+     * After preemption, it should preempt 30 from app1, and 30 from app4.
+     */
+    String labelsConfig =
+        "=100,true;" + // default partition
+        "x=100,true"; // partition=x
+    String nodesConfig =
+        "n1=x;" + // n1 has partition=x
+        "n2="; // n2 is default partition
+    String queuesConfig =
+        // guaranteed,max,used,pending
+        "root(=[100 100 100 100],x=[100 100 100 100]);" + //root
+        "-a(=[50 100 20 50],x=[50 100 80 50]);" + // a
+        "-b(=[50 100 80 50],x=[50 100 20 50])"; // b
+    String appsConfig=
+        //queueName\t(priority,resource,host,expression,#repeat,reserved)
+        "a\t" // app1 in a
+        + "(1,1,n1,x,80,false);" + // 80 * x in n1
+        "a\t" // app2 in a
+        + "(1,1,n2,,20,false);" + // 20 default in n2
+        "b\t" // app3 in b
+        + "(1,1,n1,x,20,false);" + // 80 * x in n1
+        "b\t" // app4 in b
+        + "(1,1,n2,,80,false)"; // 20 default in n2
+
+    buildEnv(labelsConfig, nodesConfig, queuesConfig, appsConfig);
+    policy.editSchedule();
+
+    // 30 preempted from app1, 30 preempted from app4, and nothing preempted
+    // from app2/app3
+    verify(mDisp, times(30)).handle(
+        argThat(new IsPreemptionRequestFor(getAppAttemptId(1))));
+    verify(mDisp, times(30)).handle(
+        argThat(new IsPreemptionRequestFor(getAppAttemptId(4))));
+    verify(mDisp, never()).handle(
+        argThat(new IsPreemptionRequestFor(getAppAttemptId(2))));
+    verify(mDisp, never()).handle(
+        argThat(new IsPreemptionRequestFor(getAppAttemptId(3))));
+  }
+
+  @Test
+  public void testNodePartitionPreemptionRespectMaximumCapacity()
+      throws IOException {
+    /**
+     * Queue structure is:
+     *
+     * <pre>
+     *         root
+     *       /  |  \
+     *      a   b   c
+     * </pre>
+     *
+     * Both a/b/c can access x, and guaranteed_capacity(x) of them is 80:10:10.
+     * a/b's max resource is 100, and c's max resource is 30.
+     *
+     * Two nodes, n1 has 100 x, n2 has 100 NO_LABEL.
+     *
+     * 2 apps in cluster.
+     * app1 in b and app2 in c.
+     *
+     * app1 uses 90x, and app2 use 10x. After preemption, app2 will preempt 10x
+     * from app1 because of max capacity.
+     */
+    String labelsConfig =
+        "=100,true;" + // default partition
+        "x=100,true"; // partition=x
+    String nodesConfig =
+        "n1=x;" + // n1 has partition=x
+        "n2="; // n2 is default partition
+    String queuesConfig =
+        // guaranteed,max,used,pending
+        "root(=[100 100 100 100],x=[100 100 100 100]);" + //root
+        "-a(=[80 80 0 0],x=[80 80 0 0]);" + // a
+        "-b(=[10 100 0 0],x=[10 100 90 50]);" + // b
+        "-c(=[10 100 0 0],x=[10 30 10 50])"; //c
+    String appsConfig=
+        //queueName\t(priority,resource,host,expression,#repeat,reserved)
+        "b\t" // app1 in b
+        + "(1,1,n1,x,90,false);" + // 80 * x in n1
+        "c\t" // app2 in c
+        + "(1,1,n1,x,10,false)"; // 20 default in n2
+
+    buildEnv(labelsConfig, nodesConfig, queuesConfig, appsConfig);
+    policy.editSchedule();
+
+    // 30 preempted from app1, 30 preempted from app4, and nothing preempted
+    // from app2/app3
+    verify(mDisp, times(20)).handle(
+        argThat(new IsPreemptionRequestFor(getAppAttemptId(1))));
+    verify(mDisp, never()).handle(
+        argThat(new IsPreemptionRequestFor(getAppAttemptId(2))));
+  }
+
+  @Test
+  public void testNodePartitionPreemptionOfIgnoreExclusivityAndRespectCapacity()
+      throws IOException {
+    /**
+     * <pre>
+     *       root
+     *       /  \
+     *      a    b
+     * </pre>
+     *
+     * Both a/b can access x, and guaranteed capacity of them is 50:50. Two
+     * nodes, n1 has 100 x, n2 has 100 NO_LABEL and 2 applications in the cluster,
+     * app1/app2 in a
+     * app1 uses 20x (ignoreExclusivity), app2 uses 80x (respectExclusivity).
+     *
+     * b has 100 pending resource of x
+     *
+     * After preemption, it should preempt 20 from app1, and 30 from app2.
+     */
+    String labelsConfig =
+        "=100,true;" + // default partition
+        "x=100,false"; // partition=x
+    String nodesConfig =
+        "n1=x;" + // n1 has partition=x
+        "n2="; // n2 is default partition
+    String queuesConfig =
+        // guaranteed,max,used,pending
+        "root(=[100 100 100 100],x=[100 100 100 100]);" + //root
+        "-a(=[50 100 0 0],x=[50 100 100 50]);" + // a
+        "-b(=[50 100 0 0],x=[50 100 0 100])"; // b
+    String appsConfig=
+        //queueName\t(priority,resource,host,expression,#repeat,reserved)
+        "a\t" // app1 in a
+        + "(1,1,n1,x,1,false)"  // 1 * x in n1 (it's AM container)
+        + "(1,1,n1,,20,false);" + // 20 * x in n1 (ignoreExclusivity)
+        "a\t" // app2 in a
+        + "(1,1,n1,x,79,false)"; // 79 * x
+
+    buildEnv(labelsConfig, nodesConfig, queuesConfig, appsConfig);
+    policy.editSchedule();
+
+    // 30 preempted from app1, 30 preempted from app4, and nothing preempted
+    // from app2/app3
+    verify(mDisp, times(20)).handle(
+        argThat(new IsPreemptionRequestFor(getAppAttemptId(1))));
+    verify(mDisp, times(30)).handle(
+        argThat(new IsPreemptionRequestFor(getAppAttemptId(2))));
+  }
+
+  @Test
+  public void testNodePartitionPreemptionOfSkippingAMContainer()
+      throws IOException {
+    /**
+     * <pre>
+     *       root
+     *       /  \
+     *      a    b
+     * </pre>
+     *
+     * Both a/b can access x, and guaranteed capacity of them is 20:80. Two
+     * nodes, n1 has 100 x, n2 has 100 NO_LABEL and 2 applications in the cluster,
+     * app1/app2/app3/app4/app5 in a, both uses 20 resources.
+     *
+     * b has 100 pending resource of x
+     *
+     * After preemption, it should preempt 19 from app[5-2] an 4 from app1
+     */
+    String labelsConfig =
+        "=100,true;" + // default partition
+        "x=100,true"; // partition=x
+    String nodesConfig =
+        "n1=x;" + // n1 has partition=x
+        "n2="; // n2 is default partition
+    String queuesConfig =
+        // guaranteed,max,used,pending
+        "root(=[100 100 100 100],x=[100 100 100 100]);" + //root
+        "-a(=[50 100 0 0],x=[20 100 100 50]);" + // a
+        "-b(=[50 100 0 0],x=[80 100 0 100])"; // b
+    String appsConfig=
+        //queueName\t(priority,resource,host,expression,#repeat,reserved)
+        "a\t" // app1 in a
+        + "(1,1,n1,x,20,false);" + // uses 20 resource
+        "a\t" // app2 in a
+        + "(1,1,n1,x,20,false);" + // uses 20 resource
+        "a\t" // app3 in a
+        + "(1,1,n1,x,20,false);" + // uses 20 resource
+        "a\t" // app4 in a
+        + "(1,1,n1,x,20,false);" + // uses 20 resource
+        "a\t" // app5 in a
+        + "(1,1,n1,x,20,false);";  // uses 20 resource
+
+    buildEnv(labelsConfig, nodesConfig, queuesConfig, appsConfig);
+    policy.editSchedule();
+
+    // 4 from app1
+    verify(mDisp, times(4)).handle(
+        argThat(new IsPreemptionRequestFor(getAppAttemptId(1))));
+    // 19 from app2-app5
+    verify(mDisp, times(19)).handle(
+        argThat(new IsPreemptionRequestFor(getAppAttemptId(2))));
+    verify(mDisp, times(19)).handle(
+        argThat(new IsPreemptionRequestFor(getAppAttemptId(3))));
+    verify(mDisp, times(19)).handle(
+        argThat(new IsPreemptionRequestFor(getAppAttemptId(4))));
+    verify(mDisp, times(19)).handle(
+        argThat(new IsPreemptionRequestFor(getAppAttemptId(5))));
+  }
+
+  @Test
+  public void testNodePartitionPreemptionOfAMContainer()
+      throws IOException {
+    /**
+     * <pre>
+     *       root
+     *       /  \
+     *      a    b
+     * </pre>
+     *
+     * Both a/b can access x, and guaranteed capacity of them is 3:97. Two
+     * nodes, n1 has 100 x, n2 has 100 NO_LABEL.
+     *
+     * app1/app2/app3/app4/app5 in a, both uses 20 resources(x)
+     *
+     * b has 100 pending resource of x
+     *
+     * After preemption, it should preempt 20 from app4/app5 an 19 from
+     * app1-app3. App4/app5's AM container will be preempted
+     */
+    String labelsConfig =
+        "=100,true;" + // default partition
+        "x=100,true"; // partition=x
+    String nodesConfig =
+        "n1=x;" + // n1 has partition=x
+        "n2="; // n2 is default partition
+    String queuesConfig =
+        // guaranteed,max,used,pending
+        "root(=[100 100 100 100],x=[100 100 100 100]);" + //root
+        "-a(=[50 100 0 0],x=[3 100 100 50]);" + // a
+        "-b(=[50 100 0 0],x=[97 100 0 100])"; // b
+    String appsConfig=
+        //queueName\t(priority,resource,host,expression,#repeat,reserved)
+        "a\t" // app1 in a
+        + "(1,1,n1,x,20,false);" + // uses 20 resource
+        "a\t" // app2 in a
+        + "(1,1,n1,x,20,false);" + // uses 20 resource
+        "a\t" // app3 in a
+        + "(1,1,n1,x,20,false);" + // uses 20 resource
+        "a\t" // app4 in a
+        + "(1,1,n1,x,20,false);" + // uses 20 resource
+        "a\t" // app5 in a
+        + "(1,1,n1,x,20,false);";  // uses 20 resource
+
+    buildEnv(labelsConfig, nodesConfig, queuesConfig, appsConfig);
+    policy.editSchedule();
+
+    // 4 from app1
+    verify(mDisp, times(19)).handle(
+        argThat(new IsPreemptionRequestFor(getAppAttemptId(1))));
+    // 19 from app2-app5
+    verify(mDisp, times(19)).handle(
+        argThat(new IsPreemptionRequestFor(getAppAttemptId(2))));
+    verify(mDisp, times(19)).handle(
+        argThat(new IsPreemptionRequestFor(getAppAttemptId(3))));
+    verify(mDisp, times(20)).handle(
+        argThat(new IsPreemptionRequestFor(getAppAttemptId(4))));
+    verify(mDisp, times(20)).handle(
+        argThat(new IsPreemptionRequestFor(getAppAttemptId(5))));
+  }
+
+  @Test
+  public void testNodePartitionDisablePreemptionForSingleLevelQueue()
+      throws IOException {
+    /**
+     * Queue structure is:
+     *
+     * <pre>
+     *         root
+     *       /  |  \
+     *      a   b   c
+     * </pre>
+     *
+     * Both a/b/c can access x, and guaranteed_capacity(x) of them is 40:20:40.
+     * a/b/c's max resource is 100. b is disable-preemption
+     *
+     * Two nodes, n1 has 100 x, n2 has 100 NO_LABEL.
+     *
+     * 2 apps in cluster. app1 in a (usage=50), app2 in b(usage=30), app3 in
+     * c(usage=20). All of them have 50 pending resource.
+     *
+     * After preemption, app1 will be preempt 10 containers and app2 will not be
+     * preempted
+     */
+    String labelsConfig =
+        "=100,true;" + // default partition
+        "x=100,true"; // partition=x
+    String nodesConfig =
+        "n1=x;" + // n1 has partition=x
+        "n2="; // n2 is default partition
+    String queuesConfig =
+        // guaranteed,max,used,pending
+        "root(=[100 100 100 100],x=[100 100 100 100]);" + //root
+        "-a(=[80 80 0 0],x=[40 100 50 50]);" + // a
+        "-b(=[10 100 0 0],x=[20 100 30 0]);" + // b
+        "-c(=[10 100 0 0],x=[40 100 20 50])"; //c
+    String appsConfig=
+        //queueName\t(priority,resource,host,expression,#repeat,reserved)
+        "a\t" // app1 in a
+        + "(1,1,n1,x,50,false);" + // 50x in n1
+        "b\t" // app2 in b
+        + "(1,1,n1,x,30,false);" + // 30x in n1
+        "c\t" // app3 in c
+        + "(1,1,n1,x,20,false)"; // 20x in n1
+
+    csConf.setPreemptionDisabled("root.b", true);
+    buildEnv(labelsConfig, nodesConfig, queuesConfig, appsConfig);
+    policy.editSchedule();
+
+    // 10 preempted from app1, nothing preempted from app2-app3
+    verify(mDisp, times(10)).handle(
+        argThat(new IsPreemptionRequestFor(getAppAttemptId(1))));
+    verify(mDisp, never()).handle(
+        argThat(new IsPreemptionRequestFor(getAppAttemptId(2))));
+    verify(mDisp, never()).handle(
+        argThat(new IsPreemptionRequestFor(getAppAttemptId(3))));
+  }
+
+  @Test
+  public void testNodePartitionNonAccessibleQueuesSharePartitionedResource()
+      throws IOException {
+    /**
+     * Queue structure is:
+     *
+     * <pre>
+     *           root
+     *        _________
+     *       /  |   |  \
+     *      a   b   c   d
+     * </pre>
+     *
+     * a/b can access x, their capacity is 50:50 c/d cannot access x.
+     *
+     * a uses 0, wants 30
+     * b(app1) uses 30, wants 0
+     * c(app2)&d(app3) use 35, wants 50
+     *
+     * After preemption, c/d will be preempted 15 containers, because idle
+     * resource = 100 - 30 (which is used by b) - 30 (which is asked by a) = 40
+     * will be divided by c/d, so each of c/d get 20.
+     */
+    String labelsConfig =
+        "=100,true;" + // default partition
+        "x=100,false"; // partition=x
+    String nodesConfig =
+        "n1=x;" + // n1 has partition=x
+        "n2="; // n2 is default partition
+    String queuesConfig =
+        // guaranteed,max,used,pending
+        "root(=[100 100 100 100],x=[100 100 100 100]);" + //root
+        "-a(=[25 100 0 0],x=[50 100 0 30]);" + // a
+        "-b(=[25 100 0 0],x=[50 100 30 0]);" + // b
+        "-c(=[25 100 1 0],x=[0 0 35 50]);" + //c
+        "-d(=[25 100 1 0],x=[0 0 35 50])"; //d
+    String appsConfig=
+        //queueName\t(priority,resource,host,expression,#repeat,reserved)
+        "b\t" // app1 in b
+        + "(1,1,n1,x,30,false);" + // 50x in n1
+        "c\t" // app2 in c
+        + "(1,1,n2,,1,false)" // AM container (in n2)
+        + "(1,1,n1,,30,false);" + // 30x in n1 (ignore exclusivity)
+        "d\t" // app3 in d
+        + "(1,1,n2,,1,false)" // AM container (in n2)
+        + "(1,1,n1,,30,false)"; // 30x in n1 (ignore exclusivity)
+
+    buildEnv(labelsConfig, nodesConfig, queuesConfig, appsConfig);
+    policy.editSchedule();
+
+    // 15 will be preempted app2/app3
+    verify(mDisp, times(15)).handle(
+        argThat(new IsPreemptionRequestFor(getAppAttemptId(2))));
+    verify(mDisp, times(15)).handle(
+        argThat(new IsPreemptionRequestFor(getAppAttemptId(3))));
+    verify(mDisp, never()).handle(
+        argThat(new IsPreemptionRequestFor(getAppAttemptId(1))));
+  }
+
+  @Test
+  public void testHierarchyPreemptionForMultiplePartitions()
+      throws IOException {
+    /**
+     * Queue structure is:
+     *
+     * <pre>
+     *           root
+     *           /  \
+     *          a    b
+     *        /  \  /  \
+     *       a1  a2 b1  b2
+     * </pre>
+     *
+     * Both a/b can access x/y, and in all hierarchy capacity ratio is 50:50.
+     * So for a1/a2/b1/b2, all of them can access 25x, 25y
+     *
+     * a1 uses 35x, 25y
+     * a2 uses 25x, 15y
+     * b1 uses 15x, 25y
+     * b2 uses 25x 35y
+     *
+     * So as a result, a2 will preempt from b2, and b1 will preempt from a1.
+     *
+     * After preemption, a1 will be preempted 10x and b2 will be preempted 10y.
+     */
+    String labelsConfig =
+        "=100,true;" + // default partition
+        "x=100,true;" + // partition=x
+        "y=100,true";   // partition=y
+    String nodesConfig =
+        "n1=x;" + // n1 has partition=x
+        "n2=y;" + // n2 has partition=y
+        "n3="; // n3 is default partition
+    String queuesConfig =
+        // guaranteed,max,used,pending
+        "root(=[100 100 0 0],x=[100 100 100 100],y=[100 100 100 100]);" + //root
+        "-a(=[50 100 0 0],x=[50 100 60 40],y=[50 100 40 40]);" + // a
+        "--a1(=[25 100 0 0],x=[25 100 35 20],y=[25 100 25 20]);" + // a1
+        "--a2(=[25 100 0 0],x=[25 100 25 20],y=[25 100 15 20]);" + // a2
+        "-b(=[50 100 0 0],x=[50 100 40 40],y=[50 100 60 40]);" + // b
+        "--b1(=[25 100 0 0],x=[25 100 15 20],y=[25 100 25 20]);" + // b1
+        "--b2(=[25 100 0 0],x=[25 100 25 20],y=[25 100 35 20])"; // b2
+    String appsConfig=
+        //queueName\t(priority,resource,host,expression,#repeat,reserved)
+        "a1\t" // app1 in a1
+        + "(1,1,n1,x,35,false)" // 35 of x
+        + "(1,1,n2,y,25,false);" + // 25 of y
+        "a2\t" // app2 in a2
+        + "(1,1,n1,x,25,false)" // 25 of x
+        + "(1,1,n2,y,15,false);" + // 15 of y
+        "b1\t" // app3 in b1
+        + "(1,1,n1,x,15,false)" // 15 of x
+        + "(1,1,n2,y,25,false);" + // 25 of y
+        "b2\t" // app4 in b2
+        + "(1,1,n1,x,25,false)" // 25 of x
+        + "(1,1,n2,y,35,false)"; // 35 of y
+
+    buildEnv(labelsConfig, nodesConfig, queuesConfig, appsConfig);
+    policy.editSchedule();
+
+    // 10 will be preempted from app1 (a1) /app4 (b2)
+    verify(mDisp, times(10)).handle(
+        argThat(new IsPreemptionRequestFor(getAppAttemptId(1))));
+    verify(mDisp, times(10)).handle(
+        argThat(new IsPreemptionRequestFor(getAppAttemptId(4))));
+    verify(mDisp, never()).handle(
+        argThat(new IsPreemptionRequestFor(getAppAttemptId(2))));
+    verify(mDisp, never()).handle(
+        argThat(new IsPreemptionRequestFor(getAppAttemptId(3))));
+  }
+
+  @Test
+  public void testHierarchyPreemptionForDifferenceAcessibility()
+      throws IOException {
+    /**
+     * Queue structure is:
+     *
+     * <pre>
+     *           root
+     *           /  \
+     *          a    b
+     *        /  \  /  \
+     *       a1  a2 b1  b2
+     * </pre>
+     *
+     * a can access x only and b can access y only
+     *
+     * Capacities of a1/a2, b1/b2 is 50:50
+     *
+     * a1 uses 100x and b1 uses 80y
+     *
+     * So as a result, a1 will be preempted 50 containers and b1 will be
+     * preempted 30 containers
+     */
+    String labelsConfig =
+        "=100,true;" + // default partition
+        "x=100,true;" + // partition=x
+        "y=100,true";   // partition=y
+    String nodesConfig =
+        "n1=x;" + // n1 has partition=x
+        "n2=y;" + // n2 has partition=y
+        "n3="; // n3 is default partition
+    String queuesConfig =
+        // guaranteed,max,used,pending
+        "root(=[100 100 0 0],x=[100 100 100 100],y=[100 100 100 100]);" + //root
+        "-a(=[50 100 0 0],x=[100 100 100 100]);" + // a
+        "--a1(=[25 100 0 0],x=[50 100 100 0]);" + // a1
+        "--a2(=[25 100 0 0],x=[50 100 0 100]);" + // a2
+        "-b(=[50 100 0 0],y=[100 100 80 100]);" + // b
+        "--b1(=[25 100 0 0],y=[50 100 80 0]);" + // b1
+        "--b2(=[25 100 0 0],y=[50 100 0 100])"; // b2
+    String appsConfig=
+        //queueName\t(priority,resource,host,expression,#repeat,reserved)
+        "a1\t" // app1 in a1
+        + "(1,1,n1,x,100,false);" + // 100 of x
+        "b1\t" // app2 in b1
+        + "(1,1,n2,y,80,false)"; // 80 of y
+
+    buildEnv(labelsConfig, nodesConfig, queuesConfig, appsConfig);
+    policy.editSchedule();
+
+    verify(mDisp, times(50)).handle(
+        argThat(new IsPreemptionRequestFor(getAppAttemptId(1))));
+    verify(mDisp, times(30)).handle(
+        argThat(new IsPreemptionRequestFor(getAppAttemptId(2))));
+  }
+
+
+  private ApplicationAttemptId getAppAttemptId(int id) {
+    ApplicationId appId = ApplicationId.newInstance(0L, id);
+    ApplicationAttemptId appAttemptId =
+        ApplicationAttemptId.newInstance(appId, 1);
+    return appAttemptId;
+  }
+
+  private void checkContainerNodesInApp(FiCaSchedulerApp app,
+      int expectedContainersNumber, String host) {
+    NodeId nodeId = NodeId.newInstance(host, 1);
+    int num = 0;
+    for (RMContainer c : app.getLiveContainers()) {
+      if (c.getAllocatedNode().equals(nodeId)) {
+        num++;
+      }
+    }
+    for (RMContainer c : app.getReservedContainers()) {
+      if (c.getAllocatedNode().equals(nodeId)) {
+        num++;
+      }
+    }
+    Assert.assertEquals(expectedContainersNumber, num);
+  }
+
+  private FiCaSchedulerApp getApp(String queueName, int appId) {
+    for (FiCaSchedulerApp app : ((LeafQueue) cs.getQueue(queueName))
+        .getApplications()) {
+      if (app.getApplicationId().getId() == appId) {
+        return app;
+      }
+    }
+    return null;
+  }
+
+  private void checkAbsCapacities(CSQueue queue, String partition,
+      float guaranteed, float max, float used) {
+    QueueCapacities qc = queue.getQueueCapacities();
+    Assert.assertEquals(guaranteed, qc.getAbsoluteCapacity(partition), 1e-3);
+    Assert.assertEquals(max, qc.getAbsoluteMaximumCapacity(partition), 1e-3);
+    Assert.assertEquals(used, qc.getAbsoluteUsedCapacity(partition), 1e-3);
+  }
+
+  private void checkPendingResource(CSQueue queue, String partition, int pending) {
+    ResourceUsage ru = queue.getQueueResourceUsage();
+    Assert.assertEquals(pending, ru.getPending(partition).getMemory());
+  }
+
+  private void buildEnv(String labelsConfig, String nodesConfig,
+      String queuesConfig, String appsConfig) throws IOException {
+    mockNodeLabelsManager(labelsConfig);
+    mockSchedulerNodes(nodesConfig);
+    for (NodeId nodeId : nodeIdToSchedulerNodes.keySet()) {
+      when(cs.getSchedulerNode(nodeId)).thenReturn(
+          nodeIdToSchedulerNodes.get(nodeId));
+    }
+    ParentQueue root = mockQueueHierarchy(queuesConfig);
+    when(cs.getRootQueue()).thenReturn(root);
+    when(cs.getClusterResource()).thenReturn(clusterResource);
+    mockApplications(appsConfig);
+
+    policy = new ProportionalCapacityPreemptionPolicy(conf, mDisp, cs, mClock);
+  }
+
+  private void mockContainers(String containersConfig, ApplicationAttemptId attemptId,
+      String queueName, List<RMContainer> reservedContainers,
+      List<RMContainer> liveContainers) {
+    int containerId = 1;
+    int start = containersConfig.indexOf("=") + 1;
+    int end = -1;
+
+    while (start < containersConfig.length()) {
+      while (start < containersConfig.length()
+          && containersConfig.charAt(start) != '(') {
+        start++;
+      }
+      if (start >= containersConfig.length()) {
+        throw new IllegalArgumentException(
+            "Error containers specification, line=" + containersConfig);
+      }
+      end = start + 1;
+      while (end < containersConfig.length()
+          && containersConfig.charAt(end) != ')') {
+        end++;
+      }
+      if (end >= containersConfig.length()) {
+        throw new IllegalArgumentException(
+            "Error containers specification, line=" + containersConfig);
+      }
+
+      // now we found start/end, get container values
+      String[] values = containersConfig.substring(start + 1, end).split(",");
+      if (values.length != 6) {
+        throw new IllegalArgumentException("Format to define container is:"
+            + "(priority,resource,host,expression,repeat,reserved)");
+      }
+      Priority pri = Priority.newInstance(Integer.valueOf(values[0]));
+      Resource res = Resources.createResource(Integer.valueOf(values[1]));
+      NodeId host = NodeId.newInstance(values[2], 1);
+      String exp = values[3];
+      int repeat = Integer.valueOf(values[4]);
+      boolean reserved = Boolean.valueOf(values[5]);
+
+      for (int i = 0; i < repeat; i++) {
+        Container c = mock(Container.class);
+        when(c.getResource()).thenReturn(res);
+        when(c.getPriority()).thenReturn(pri);
+        RMContainerImpl rmc = mock(RMContainerImpl.class);
+        when(rmc.getAllocatedNode()).thenReturn(host);
+        when(rmc.getNodeLabelExpression()).thenReturn(exp);
+        when(rmc.getAllocatedResource()).thenReturn(res);
+        when(rmc.getContainer()).thenReturn(c);
+        when(rmc.getApplicationAttemptId()).thenReturn(attemptId);
+        final ContainerId cId = ContainerId.newContainerId(attemptId, containerId);
+        when(rmc.getContainerId()).thenReturn(
+            cId);
+        doAnswer(new Answer<Integer>() {
+          @Override
+          public Integer answer(InvocationOnMock invocation) throws Throwable {
+            return cId.compareTo(((RMContainer) invocation.getArguments()[0])
+                .getContainerId());
+          }
+        }).when(rmc).compareTo(any(RMContainer.class));
+
+        if (containerId == 1) {
+          when(rmc.isAMContainer()).thenReturn(true);
+        }
+
+        if (reserved) {
+          reservedContainers.add(rmc);
+        } else {
+          liveContainers.add(rmc);
+        }
+
+        // If this is a non-exclusive allocation
+        String partition = null;
+        if (exp.isEmpty()
+            && !(partition = nodeIdToSchedulerNodes.get(host).getPartition())
+                .isEmpty()) {
+          LeafQueue queue = (LeafQueue) nameToCSQueues.get(queueName);
+          Map<String, TreeSet<RMContainer>> ignoreExclusivityContainers =
+              queue.getIgnoreExclusivityRMContainers();
+          if (!ignoreExclusivityContainers.containsKey(partition)) {
+            ignoreExclusivityContainers.put(partition,
+                new TreeSet<RMContainer>());
+          }
+          ignoreExclusivityContainers.get(partition).add(rmc);
+        }
+        LOG.debug("add container to app=" + attemptId + " res=" + res
+            + " node=" + host + " nodeLabelExpression=" + exp + " partition="
+            + partition);
+
+        containerId++;
+      }
+
+      start = end + 1;
+    }
+  }
+
+  /**
+   * Format is:
+   * <pre>
+   * queueName\t  // app1
+   * (priority,resource,host,expression,#repeat,reserved)
+   * (priority,resource,host,expression,#repeat,reserved);
+   * queueName\t  // app2
+   * </pre>
+   */
+  private void mockApplications(String appsConfig) {
+    int id = 1;
+    for (String a : appsConfig.split(";")) {
+      String[] strs = a.split("\t");
+      String queueName = strs[0];
+
+      // get containers
+      List<RMContainer> liveContainers = new ArrayList<RMContainer>();
+      List<RMContainer> reservedContainers = new ArrayList<RMContainer>();
+      ApplicationId appId = ApplicationId.newInstance(0L, id);
+      ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId, 1);
+
+      mockContainers(strs[1], appAttemptId, queueName, reservedContainers,
+          liveContainers);
+
+      FiCaSchedulerApp app = mock(FiCaSchedulerApp.class);
+      when(app.getLiveContainers()).thenReturn(liveContainers);
+      when(app.getReservedContainers()).thenReturn(reservedContainers);
+      when(app.getApplicationAttemptId()).thenReturn(appAttemptId);
+      when(app.getApplicationId()).thenReturn(appId);
+
+      // add to LeafQueue
+      LeafQueue queue = (LeafQueue) nameToCSQueues.get(queueName);
+      queue.getApplications().add(app);
+
+      id++;
+    }
+  }
+
+  /**
+   * Format is:
+   * host1=partition;
+   * host2=partition;
+   */
+  private void mockSchedulerNodes(String schedulerNodesConfigStr)
+      throws IOException {
+    String[] nodesConfigStrArray = schedulerNodesConfigStr.split(";");
+    for (String p : nodesConfigStrArray) {
+      NodeId nodeId = NodeId.newInstance(p.substring(0, p.indexOf("=")), 1);
+      String partition = p.substring(p.indexOf("=") + 1, p.length());
+
+      SchedulerNode sn = mock(SchedulerNode.class);
+      when(sn.getNodeID()).thenReturn(nodeId);
+      when(sn.getPartition()).thenReturn(partition);
+      nodeIdToSchedulerNodes.put(nodeId, sn);
+
+      LOG.debug("add scheduler node, id=" + nodeId + ", partition=" + partition);
+    }
+  }
+
+  /**
+   * Format is:
+   * <pre>
+   * partition0=total_resource,exclusivity;
+   * partition1=total_resource,exclusivity;
+   * ...
+   * </pre>
+   */
+  private void mockNodeLabelsManager(String nodeLabelsConfigStr) throws IOException {
+    String[] partitionConfigArr = nodeLabelsConfigStr.split(";");
+    clusterResource = Resources.createResource(0);
+    for (String p : partitionConfigArr) {
+      String partitionName = p.substring(0, p.indexOf("="));
+      int totalResource =
+          Integer.valueOf(p.substring(p.indexOf("=") + 1, p.indexOf(",")));
+      boolean exclusivity =
+          Boolean.valueOf(p.substring(p.indexOf(",") + 1, p.length()));
+      Resource res = Resources.createResource(totalResource);
+      when(nlm.getResourceByLabel(eq(partitionName), any(Resource.class)))
+          .thenReturn(res);
+      when(nlm.isExclusiveNodeLabel(eq(partitionName))).thenReturn(exclusivity);
+
+      // add to partition to resource
+      partitionToResource.put(partitionName, res);
+      LOG.debug("add partition=" + partitionName + " totalRes=" + res
+          + " exclusivity=" + exclusivity);
+      Resources.addTo(clusterResource, res);
+    }
+
+    when(nlm.getClusterNodeLabelNames()).thenReturn(
+        partitionToResource.keySet());
+  }
+
+  /**
+   * Format is:
+   * <pre>
+   * root (<partition-name-1>=[guaranteed max used pending],<partition-name-2>=..);
+   * -A(...);
+   * --A1(...);
+   * --A2(...);
+   * -B...
+   * </pre>
+   * ";" splits queues, and there should no empty lines, no extra spaces
+   */
+  @SuppressWarnings({ "unchecked", "rawtypes" })
+  private ParentQueue mockQueueHierarchy(String queueExprs) {
+    String[] queueExprArray = queueExprs.split(";");
+    ParentQueue rootQueue = null;
+    for (int idx = 0; idx < queueExprArray.length; idx++) {
+      String q = queueExprArray[idx];
+      CSQueue queue;
+
+      // Initialize queue
+      if (isParent(queueExprArray, idx)) {
+        ParentQueue parentQueue = mock(ParentQueue.class);
+        queue = parentQueue;
+        List<CSQueue> children = new ArrayList<CSQueue>();
+        when(parentQueue.getChildQueues()).thenReturn(children);
+      } else {
+        LeafQueue leafQueue = mock(LeafQueue.class);
+        final TreeSet<FiCaSchedulerApp> apps =
+            new TreeSet<>(CapacityScheduler.applicationComparator);
+        when(leafQueue.getApplications()).thenReturn(apps);
+        OrderingPolicy<FiCaSchedulerApp> so = mock(OrderingPolicy.class);
+        when(so.getPreemptionIterator()).thenAnswer(new Answer() {
+          public Object answer(InvocationOnMock invocation) {
+            return apps.descendingIterator();
+          }
+        });
+        when(leafQueue.getOrderingPolicy()).thenReturn(so);
+
+        Map<String, TreeSet<RMContainer>> ignorePartitionContainers =
+            new HashMap<>();
+        when(leafQueue.getIgnoreExclusivityRMContainers()).thenReturn(
+            ignorePartitionContainers);
+        queue = leafQueue;
+      }
+
+      setupQueue(queue, q, queueExprArray, idx);
+      if (queue.getQueueName().equals(ROOT)) {
+        rootQueue = (ParentQueue) queue;
+      }
+    }
+    return rootQueue;
+  }
+
+  private void setupQueue(CSQueue queue, String q, String[] queueExprArray,
+      int idx) {
+    LOG.debug("*** Setup queue, source=" + q);
+    String queuePath = null;
+
+    int myLevel = getLevel(q);
+    if (0 == myLevel) {
+      // It's root
+      when(queue.getQueueName()).thenReturn(ROOT);
+      queuePath = ROOT;
+    }
+
+    String queueName = getQueueName(q);
+    when(queue.getQueueName()).thenReturn(queueName);
+
+    // Setup parent queue, and add myself to parentQueue.children-list
+    ParentQueue parentQueue = getParentQueue(queueExprArray, idx, myLevel);
+    if (null != parentQueue) {
+      when(queue.getParent()).thenReturn(parentQueue);
+      parentQueue.getChildQueues().add(queue);
+
+      // Setup my path
+      queuePath = parentQueue.getQueuePath() + "." + queueName;
+    }
+    when(queue.getQueuePath()).thenReturn(queuePath);
+
+    QueueCapacities qc = new QueueCapacities(0 == myLevel);
+    ResourceUsage ru = new ResourceUsage();
+
+    when(queue.getQueueCapacities()).thenReturn(qc);
+    when(queue.getQueueResourceUsage()).thenReturn(ru);
+
+    LOG.debug("Setup queue, name=" + queue.getQueueName() + " path="
+        + queue.getQueuePath());
+    LOG.debug("Parent=" + (parentQueue == null ? "null" : parentQueue
+        .getQueueName()));
+
+    // Setup other fields like used resource, guaranteed resource, etc.
+    String capacitySettingStr = q.substring(q.indexOf("(") + 1, q.indexOf(")"));
+    for (String s : capacitySettingStr.split(",")) {
+      String partitionName = s.substring(0, s.indexOf("="));
+      String[] values = s.substring(s.indexOf("[") + 1, s.indexOf("]")).split(" ");
+      // Add a small epsilon to capacities to avoid truncate when doing
+      // Resources.multiply
+      float epsilon = 1e-6f;
+      float absGuaranteed =
+          Integer.valueOf(values[0].trim())
+              / (float) (partitionToResource.get(partitionName).getMemory())
+              + epsilon;
+      float absMax =
+          Integer.valueOf(values[1].trim())
+              / (float) (partitionToResource.get(partitionName).getMemory())
+              + epsilon;
+      float absUsed =
+          Integer.valueOf(values[2].trim())
+              / (float) (partitionToResource.get(partitionName).getMemory())
+              + epsilon;
+      Resource pending = Resources.createResource(Integer.valueOf(values[3].trim()));
+      qc.setAbsoluteCapacity(partitionName, absGuaranteed);
+      qc.setAbsoluteMaximumCapacity(partitionName, absMax);
+      qc.setAbsoluteUsedCapacity(partitionName, absUsed);
+      ru.setPending(partitionName, pending);
+      LOG.debug("Setup queue=" + queueName + " partition=" + partitionName
+          + " [abs_guaranteed=" + absGuaranteed + ",abs_max=" + absMax
+          + ",abs_used" + absUsed + ",pending_resource=" + pending + "]");
+    }
+
+    // Setup preemption disabled
+    when(queue.getPreemptionDisabled()).thenReturn(
+        csConf.getPreemptionDisabled(queuePath, false));
+
+    nameToCSQueues.put(queueName, queue);
+    when(cs.getQueue(eq(queueName))).thenReturn(queue);
+  }
+
+  /**
+   * Level of a queue is how many "-" at beginning, root's level is 0
+   */
+  private int getLevel(String q) {
+    int level = 0; // level = how many "-" at beginning
+    while (level < q.length() && q.charAt(level) == '-') {
+      level++;
+    }
+    return level;
+  }
+
+  private String getQueueName(String q) {
+    int idx = 0;
+    // find first != '-' char
+    while (idx < q.length() && q.charAt(idx) == '-') {
+      idx++;
+    }
+    if (idx == q.length()) {
+      throw new IllegalArgumentException("illegal input:" + q);
+    }
+    // name = after '-' and before '('
+    String name = q.substring(idx, q.indexOf('('));
+    if (name.isEmpty()) {
+      throw new IllegalArgumentException("queue name shouldn't be empty:" + q);
+    }
+    if (name.contains(".")) {
+      throw new IllegalArgumentException("queue name shouldn't contain '.':"
+          + name);
+    }
+    return name;
+  }
+
+  private ParentQueue getParentQueue(String[] queueExprArray, int idx, int myLevel) {
+    idx--;
+    while (idx >= 0) {
+      int level = getLevel(queueExprArray[idx]);
+      if (level < myLevel) {
+        String parentQueuName = getQueueName(queueExprArray[idx]);
+        return (ParentQueue) nameToCSQueues.get(parentQueuName);
+      }
+      idx--;
+    }
+
+    return null;
+  }
+
+  /**
+   * Get if a queue is ParentQueue
+   */
+  private boolean isParent(String[] queues, int idx) {
+    int myLevel = getLevel(queues[idx]);
+    idx++;
+    while (idx < queues.length && getLevel(queues[idx]) == myLevel) {
+      idx++;
+    }
+    if (idx >= queues.length || getLevel(queues[idx]) < myLevel) {
+      // It's a LeafQueue
+      return false;
+    } else {
+      return true;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d497f6ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java
index 2608dcb..31661da 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java
@@ -138,7 +138,7 @@ public class TestChildQueueOrder {
         } else {
           FiCaSchedulerApp app1 = getMockApplication(0, "");
           ((LeafQueue)queue).allocateResource(clusterResource, app1, 
-              allocatedResource, null);
+              allocatedResource, null, null);
         }
 
         // Next call - nothing

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d497f6ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
index 34248a4..1c8622f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
@@ -815,9 +815,9 @@ public class TestLeafQueue {
     qb.finishApplication(app_0.getApplicationId(), user_0);
     qb.finishApplication(app_2.getApplicationId(), user_1);
     qb.releaseResource(clusterResource, app_0, app_0.getResource(u0Priority),
-        null);
+        null, null);
     qb.releaseResource(clusterResource, app_2, app_2.getResource(u1Priority),
-        null);
+        null, null);
 
     qb.setUserLimit(50);
     qb.setUserLimitFactor(1);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d497f6ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestNodeLabelContainerAllocation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestNodeLabelContainerAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestNodeLabelContainerAllocation.java
index 46aa7ec..48d6602 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestNodeLabelContainerAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestNodeLabelContainerAllocation.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
@@ -46,6 +47,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppRepor
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
 import org.junit.Assert;
 import org.junit.Before;
@@ -1015,6 +1017,20 @@ public class TestNodeLabelContainerAllocation {
     // app1 gets all resource in partition=x
     Assert.assertEquals(10, schedulerNode1.getNumContainers());
 
+    // check non-exclusive containers of LeafQueue is correctly updated
+    LeafQueue leafQueue = (LeafQueue) cs.getQueue("a");
+    Assert.assertFalse(leafQueue.getIgnoreExclusivityRMContainers().containsKey(
+        "y"));
+    Assert.assertEquals(10,
+        leafQueue.getIgnoreExclusivityRMContainers().get("x").size());
+
+    // completes all containers of app1, ignoreExclusivityRMContainers should be
+    // updated as well.
+    cs.handle(new AppAttemptRemovedSchedulerEvent(
+        am1.getApplicationAttemptId(), RMAppAttemptState.FINISHED, false));
+    Assert.assertFalse(leafQueue.getIgnoreExclusivityRMContainers().containsKey(
+        "x"));
+
     rm1.close();
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d497f6ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
index bdbd168..4deaaae 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
@@ -150,7 +150,7 @@ public class TestParentQueue {
         } else {
           FiCaSchedulerApp app1 = getMockApplication(0, "");
           ((LeafQueue)queue).allocateResource(clusterResource, app1, 
-              allocatedResource, null);
+              allocatedResource, null, null);
         }
         
         // Next call - nothing


[14/50] [abbrv] hadoop git commit: HADOOP-11857. Fix CommandFormat#commandFormat java doc annotation. Contributed by J.Andreina.

Posted by ji...@apache.org.
HADOOP-11857. Fix CommandFormat#commandFormat java doc annotation. Contributed by J.Andreina.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/1a2459bd
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1a2459bd
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1a2459bd

Branch: refs/heads/HDFS-7240
Commit: 1a2459bd4be54e64eec0eebffd941989476c2a5b
Parents: 8847777
Author: Jakob Homan <jg...@gmail.com>
Authored: Sun Apr 26 18:35:01 2015 -0700
Committer: Jakob Homan <jg...@gmail.com>
Committed: Sun Apr 26 18:35:01 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt                 | 5 ++++-
 .../src/main/java/org/apache/hadoop/fs/shell/CommandFormat.java | 2 +-
 2 files changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1a2459bd/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 597496a..390dbaf 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -204,7 +204,10 @@ Trunk (Unreleased)
     HADOOP-11850. Typos in hadoop-common java docs. (Surendra Singh Lilhore
     via jghoman)
 
-    HADOOP-11852. Disable symlinks in trunk.
+    HADOOP-11852. Disable symlinks in trunk. (Andrew Wang)
+
+    HADOOP-11857. Fix CommandFormat#commandFormat java doc annotation.
+    (J.Andreina via jghoman)
 
   BUG FIXES
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1a2459bd/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandFormat.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandFormat.java
index 0f9aa38..371168d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandFormat.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandFormat.java
@@ -43,7 +43,7 @@ public class CommandFormat {
    * @see #CommandFormat(int, int, String...)
    */
   @Deprecated
-  public CommandFormat(String n, int min, int max, String ... possibleOpt) {
+  public CommandFormat(String name, int min, int max, String ... possibleOpt) {
     this(min, max, possibleOpt);
   }
   


[42/50] [abbrv] hadoop git commit: HDFS-7770. Need document for storage type label of data node storage locations under dfs.data.dir. Contributed by Xiaoyu Yao.

Posted by ji...@apache.org.
HDFS-7770. Need document for storage type label of data node storage locations under dfs.data.dir. Contributed by Xiaoyu Yao.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/de9404f0
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/de9404f0
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/de9404f0

Branch: refs/heads/HDFS-7240
Commit: de9404f02f36bf9a1100c67f41db907d494bb9ed
Parents: f384a06
Author: Akira Ajisaka <aa...@apache.org>
Authored: Thu Apr 30 20:12:32 2015 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Thu Apr 30 20:12:32 2015 +0900

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                  | 3 +++
 .../hadoop-hdfs/src/main/resources/hdfs-default.xml          | 7 +++++--
 .../hadoop-hdfs/src/site/markdown/ArchivalStorage.md         | 8 ++++++++
 3 files changed, 16 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/de9404f0/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index c538b78..e092dc8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -603,6 +603,9 @@ Release 2.7.1 - UNRELEASED
     HDFS-8204. Mover/Balancer should not schedule two replicas to the same
     datanode.  (Walter Su via szetszwo)
 
+    HDFS-7770. Need document for storage type label of data node storage
+    locations under dfs.data.dir. (Xiaoyu Yao via aajisaka)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de9404f0/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 092d5aa..f03c0fb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -430,8 +430,11 @@
   <description>Determines where on the local filesystem an DFS data node
   should store its blocks.  If this is a comma-delimited
   list of directories, then data will be stored in all named
-  directories, typically on different devices.
-  Directories that do not exist are ignored.
+  directories, typically on different devices. The directories should be tagged
+  with corresponding storage types ([SSD]/[DISK]/[ARCHIVE]/[RAM_DISK]) for HDFS
+  storage policies. The default storage type will be DISK if the directory does
+  not have a storage type tagged explicitly. Directories that do not exist will
+  be created if local filesystem permission allows.
   </description>
 </property>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de9404f0/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
index 2038401..d924f80 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
@@ -95,6 +95,14 @@ The effective storage policy can be retrieved by the "[`dfsadmin -getStoragePoli
 ### Configuration
 
 * **dfs.storage.policy.enabled** - for enabling/disabling the storage policy feature. The default value is `true`.
+* **dfs.datanode.data.dir** - on each data node, the comma-separated storage locations should be tagged with their storage types. This allows storage policies to place the blocks on different storage types according to policy. For example:
+
+    1.  A datanode storage location /grid/dn/disk0 on DISK should be configured with `[DISK]file:///grid/dn/disk0`
+    2.  A datanode storage location /grid/dn/ssd0 on SSD can should configured with `[SSD]file:///grid/dn/ssd0`
+    3.  A datanode storage location /grid/dn/archive0 on ARCHIVE should be configured with `[ARCHIVE]file:///grid/dn/archive0`
+    4.  A datanode storage location /grid/dn/ram0 on RAM_DISK should be configured with `[RAM_DISK]file:///grid/dn/ram0`
+
+    The default storage type of a datanode storage location will be DISK if it does not have a storage type tagged explicitly.
 
 Mover - A New Data Migration Tool
 ---------------------------------


[37/50] [abbrv] hadoop git commit: YARN-3533. Test: Fix launchAM in MockRM to wait for attempt to be scheduled. Contributed by Anubhav Dhoot

Posted by ji...@apache.org.
YARN-3533. Test: Fix launchAM in MockRM to wait for attempt to be scheduled. Contributed by Anubhav Dhoot


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/4c1af156
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/4c1af156
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/4c1af156

Branch: refs/heads/HDFS-7240
Commit: 4c1af156aef4f3bb1d9823d5980c59b12007dc77
Parents: 2e21548
Author: Jian He <ji...@apache.org>
Authored: Wed Apr 29 14:50:01 2015 -0700
Committer: Jian He <ji...@apache.org>
Committed: Wed Apr 29 14:50:01 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                              | 3 +++
 .../apache/hadoop/yarn/server/resourcemanager/MockRM.java    | 8 +++++++-
 2 files changed, 10 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4c1af156/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 6b8bde9..f583e6a 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -271,6 +271,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3517. RM web ui for dumping scheduler logs should be for admins only
     (Varun Vasudev via tgraves)
 
+    YARN-3533. Test: Fix launchAM in MockRM to wait for attempt to be scheduled.
+    (Anubhav Dhoot via jianhe)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4c1af156/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
index f2b1d86..63d6557 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
@@ -164,6 +164,8 @@ public class MockRM extends ResourceManager {
       nm.nodeHeartbeat(true);
       Thread.sleep(200);
     }
+    Assert.assertNotNull("Failed in waiting for " + containerId + " " +
+        "allocation.", getResourceScheduler().getRMContainer(containerId));
   }
 
   public void waitForContainerToComplete(RMAppAttempt attempt,
@@ -662,7 +664,7 @@ public class MockRM extends ResourceManager {
     am.waitForState(RMAppAttemptState.FINISHED);
     rm.waitForState(rmApp.getApplicationId(), RMAppState.FINISHED);
   }
-  
+
   @SuppressWarnings("rawtypes")
   private static void waitForSchedulerAppAttemptAdded(
       ApplicationAttemptId attemptId, MockRM rm) throws InterruptedException {
@@ -677,6 +679,9 @@ public class MockRM extends ResourceManager {
       }
       tick++;
     }
+    Assert.assertNotNull("Timed out waiting for SchedulerApplicationAttempt=" +
+      attemptId + " to be added.", ((AbstractYarnScheduler)
+        rm.getResourceScheduler()).getApplicationAttempt(attemptId));
   }
 
   public static MockAM launchAM(RMApp app, MockRM rm, MockNM nm)
@@ -684,6 +689,7 @@ public class MockRM extends ResourceManager {
     rm.waitForState(app.getApplicationId(), RMAppState.ACCEPTED);
     RMAppAttempt attempt = app.getCurrentAppAttempt();
     waitForSchedulerAppAttemptAdded(attempt.getAppAttemptId(), rm);
+    rm.waitForState(attempt.getAppAttemptId(), RMAppAttemptState.SCHEDULED);
     System.out.println("Launch AM " + attempt.getAppAttemptId());
     nm.nodeHeartbeat(true);
     MockAM am = rm.sendAMLaunched(attempt.getAppAttemptId());


[36/50] [abbrv] hadoop git commit: YARN-3517. RM web ui for dumping scheduler logs should be for admins only (Varun Vasudev via tgraves)

Posted by ji...@apache.org.
YARN-3517. RM web ui for dumping scheduler logs should be for admins only (Varun Vasudev via tgraves)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/2e215484
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2e215484
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2e215484

Branch: refs/heads/HDFS-7240
Commit: 2e215484bd05cd5e3b7a81d3558c6879a05dd2d2
Parents: 3dd6395
Author: tgraves <tg...@apache.org>
Authored: Wed Apr 29 21:25:42 2015 +0000
Committer: tgraves <tg...@apache.org>
Committed: Wed Apr 29 21:25:42 2015 +0000

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../server/security/ApplicationACLsManager.java | 11 +++
 .../webapp/CapacitySchedulerPage.java           | 51 +++++++++----
 .../resourcemanager/webapp/RMWebServices.java   | 13 +++-
 .../webapp/TestRMWebServices.java               | 77 ++++++++++++++++++++
 5 files changed, 139 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e215484/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index b5581d6..6b8bde9 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -268,6 +268,9 @@ Release 2.8.0 - UNRELEASED
     YARN-2740. Fix NodeLabelsManager to properly handle node label modifications 
     when distributed node label configuration enabled. (Naganarasimha G R via wangda)
 
+    YARN-3517. RM web ui for dumping scheduler logs should be for admins only
+    (Varun Vasudev via tgraves)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e215484/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/security/ApplicationACLsManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/security/ApplicationACLsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/security/ApplicationACLsManager.java
index 4daaa68..97b4163 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/security/ApplicationACLsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/security/ApplicationACLsManager.java
@@ -138,4 +138,15 @@ public class ApplicationACLsManager {
     }
     return false;
   }
+
+  /**
+   * Check if the given user in an admin.
+   *
+   * @param calledUGI
+   *          UserGroupInformation for the user
+   * @return true if the user is an admin, false otherwise
+   */
+  public final boolean isAdmin(final UserGroupInformation calledUGI) {
+    return this.adminAclsManager.isAdmin(calledUGI);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e215484/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
index 2eeda66..fa22a0d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
@@ -24,6 +24,7 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerHealth;
@@ -33,6 +34,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.UserInfo
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedulerInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedulerLeafQueueInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedulerQueueInfo;
+import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.util.Times;
 import org.apache.hadoop.yarn.webapp.ResponseInfo;
 import org.apache.hadoop.yarn.webapp.SubView;
@@ -190,28 +192,46 @@ class CapacitySchedulerPage extends RmView {
   static class QueuesBlock extends HtmlBlock {
     final CapacityScheduler cs;
     final CSQInfo csqinfo;
+    private final ResourceManager rm;
 
     @Inject QueuesBlock(ResourceManager rm, CSQInfo info) {
       cs = (CapacityScheduler) rm.getResourceScheduler();
       csqinfo = info;
+      this.rm = rm;
     }
 
     @Override
     public void render(Block html) {
       html._(MetricsOverviewTable.class);
-      // Dump CapacityScheduler debug logs
-      html.div()
+
+      UserGroupInformation callerUGI = this.getCallerUGI();
+      boolean isAdmin = false;
+      ApplicationACLsManager aclsManager = rm.getApplicationACLsManager();
+      if (aclsManager.areACLsEnabled()) {
+        if (callerUGI != null && aclsManager.isAdmin(callerUGI)) {
+          isAdmin = true;
+        }
+      } else {
+        isAdmin = true;
+      }
+
+      // only show button to dump CapacityScheduler debug logs to admins
+      if (isAdmin) {
+        html.div()
           .button()
-          .$onclick("confirmAction()").b("Dump scheduler logs")._()
-          .select().$id("time")
-            .option().$value("60")._("1 min")._()
-            .option().$value("300")._("5 min")._()
-            .option().$value("600")._("10 min")._()
+          .$style(
+              "border-style: solid; border-color: #000000; border-width: 1px;"
+                  + " cursor: hand; cursor: pointer; border-radius: 4px")
+          .$onclick("confirmAction()").b("Dump scheduler logs")._().select()
+          .$id("time").option().$value("60")._("1 min")._().option()
+          .$value("300")._("5 min")._().option().$value("600")._("10 min")._()
           ._()._();
 
-      StringBuilder script = new StringBuilder();
-      script.append("function confirmAction() {")
-          .append(" b = confirm(\"Are you sure you wish to generate scheduler logs?\");")
+        StringBuilder script = new StringBuilder();
+        script
+          .append("function confirmAction() {")
+          .append(" b = confirm(\"Are you sure you wish to generate"
+              + " scheduler logs?\");")
           .append(" if (b == true) {")
           .append(" var timePeriod = $(\"#time\").val();")
           .append(" $.ajax({")
@@ -225,13 +245,14 @@ class CapacitySchedulerPage extends RmView {
           .append(" alert(\"Scheduler log is being generated.\");")
           .append(" }, 1000);")
           .append(" }).fail(function(data){")
-          .append(" alert(\"Scheduler log generation failed. Please check the ResourceManager log for more informtion.\");")
-          .append(" console.log(data);")
-          .append(" });")
-          .append(" }")
+          .append(
+              " alert(\"Scheduler log generation failed. Please check the"
+                  + " ResourceManager log for more informtion.\");")
+          .append(" console.log(data);").append(" });").append(" }")
           .append("}");
 
-      html.script().$type("text/javascript")._(script.toString())._();
+        html.script().$type("text/javascript")._(script.toString())._();
+      }
 
       UL<DIV<DIV<Hamlet>>> ul = html.
         div("#cs-wrapper.ui-widget").

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e215484/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
index 9aea62d..4ce2b54 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
@@ -142,10 +142,12 @@ import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ResourceInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedulerInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedulerTypeInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.StatisticsItemInfo;
+import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.AdHocLogDumper;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.webapp.BadRequestException;
+import org.apache.hadoop.yarn.webapp.ForbiddenException;
 import org.apache.hadoop.yarn.webapp.NotFoundException;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 
@@ -263,8 +265,17 @@ public class RMWebServices {
   @POST
   @Path("/scheduler/logs")
   @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
-  public String dumpSchedulerLogs(@FormParam("time") String time) throws IOException {
+  public String dumpSchedulerLogs(@FormParam("time") String time,
+      @Context HttpServletRequest hsr) throws IOException {
     init();
+    UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true);
+    ApplicationACLsManager aclsManager = rm.getApplicationACLsManager();
+    if (aclsManager.areACLsEnabled()) {
+      if (callerUGI == null || !aclsManager.isAdmin(callerUGI)) {
+        String msg = "Only admins can carry out this operation.";
+        throw new ForbiddenException(msg);
+      }
+    }
     ResourceScheduler rs = rm.getResourceScheduler();
     int period = Integer.parseInt(time);
     if (period <= 0) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e215484/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java
index e4614f8..cd1d771 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java
@@ -26,7 +26,9 @@ import static org.mockito.Matchers.isA;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
+import java.io.File;
 import java.io.StringReader;
+import java.security.Principal;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.Set;
@@ -37,6 +39,7 @@ import javax.ws.rs.core.MediaType;
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 
+import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.Service.STATE;
 import org.apache.hadoop.util.VersionInfo;
@@ -54,9 +57,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppsInfo;
+import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.util.YarnVersionInfo;
+import org.apache.hadoop.yarn.webapp.ForbiddenException;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
 import org.apache.hadoop.yarn.webapp.JerseyTestBase;
 import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
@@ -643,4 +650,74 @@ public class TestRMWebServices extends JerseyTestBase {
         null, null, null, null, null, null, null, emptySet, emptySet);
     assertTrue(appsInfo.getApps().isEmpty());
   }
+
+  @Test
+  public void testDumpingSchedulerLogs() throws Exception {
+
+    ResourceManager mockRM = mock(ResourceManager.class);
+    Configuration conf = new YarnConfiguration();
+    HttpServletRequest mockHsr = mock(HttpServletRequest.class);
+    ApplicationACLsManager aclsManager = new ApplicationACLsManager(conf);
+    when(mockRM.getApplicationACLsManager()).thenReturn(aclsManager);
+    RMWebServices webSvc =
+        new RMWebServices(mockRM, conf, mock(HttpServletResponse.class));
+
+    // nothing should happen
+    webSvc.dumpSchedulerLogs("1", mockHsr);
+    Thread.sleep(1000);
+    checkSchedulerLogFileAndCleanup();
+
+    conf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
+    conf.setStrings(YarnConfiguration.YARN_ADMIN_ACL, "admin");
+    aclsManager = new ApplicationACLsManager(conf);
+    when(mockRM.getApplicationACLsManager()).thenReturn(aclsManager);
+    webSvc = new RMWebServices(mockRM, conf, mock(HttpServletResponse.class));
+    boolean exceptionThrown = false;
+    try {
+      webSvc.dumpSchedulerLogs("1", mockHsr);
+      fail("Dumping logs should fail");
+    } catch (ForbiddenException ae) {
+      exceptionThrown = true;
+    }
+    assertTrue("ForbiddenException expected", exceptionThrown);
+    exceptionThrown = false;
+    when(mockHsr.getUserPrincipal()).thenReturn(new Principal() {
+      @Override
+      public String getName() {
+        return "testuser";
+      }
+    });
+    try {
+      webSvc.dumpSchedulerLogs("1", mockHsr);
+      fail("Dumping logs should fail");
+    } catch (ForbiddenException ae) {
+      exceptionThrown = true;
+    }
+    assertTrue("ForbiddenException expected", exceptionThrown);
+
+    when(mockHsr.getUserPrincipal()).thenReturn(new Principal() {
+      @Override
+      public String getName() {
+        return "admin";
+      }
+    });
+    webSvc.dumpSchedulerLogs("1", mockHsr);
+    Thread.sleep(1000);
+    checkSchedulerLogFileAndCleanup();
+  }
+
+  private void checkSchedulerLogFileAndCleanup() {
+    String targetFile;
+    ResourceScheduler scheduler = rm.getResourceScheduler();
+    if (scheduler instanceof FairScheduler) {
+      targetFile = "yarn-fair-scheduler-debug.log";
+    } else if (scheduler instanceof CapacityScheduler) {
+      targetFile = "yarn-capacity-scheduler-debug.log";
+    } else {
+      targetFile = "yarn-scheduler-debug.log";
+    }
+    File logFile = new File(System.getProperty("yarn.log.dir"), targetFile);
+    assertTrue("scheduler log file doesn't exist", logFile.exists());
+    FileUtils.deleteQuietly(logFile);
+  }
 }


[15/50] [abbrv] hadoop git commit: MAPREDUCE-6057. Remove obsolete entries from mapred-default.xml (Ray Chiang via aw)

Posted by ji...@apache.org.
MAPREDUCE-6057. Remove obsolete entries from mapred-default.xml (Ray Chiang via aw)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/618ba707
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/618ba707
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/618ba707

Branch: refs/heads/HDFS-7240
Commit: 618ba707f0f2ddc353414dbd0eee0ab9e83b8013
Parents: 1a2459b
Author: Allen Wittenauer <aw...@apache.org>
Authored: Sun Apr 26 20:31:40 2015 -0700
Committer: Allen Wittenauer <aw...@apache.org>
Committed: Sun Apr 26 20:31:40 2015 -0700

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |  3 ++
 .../java/org/apache/hadoop/mapred/MapTask.java  |  5 ++--
 .../apache/hadoop/mapreduce/MRJobConfig.java    |  3 ++
 .../src/main/resources/mapred-default.xml       | 31 --------------------
 .../resources/job_1329348432655_0001_conf.xml   |  4 ---
 .../src/main/data/2jobs2min-rumen-jh.json       |  6 ----
 6 files changed, 9 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/618ba707/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 397f94a..4166e6a 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -105,6 +105,9 @@ Trunk (Unreleased)
     MAPREDUCE-6260. Convert site documentation to markdown (Masatake Iwasaki
     via aw)
 
+    MAPREDUCE-6057. Remove obsolete entries from mapred-default.xml
+    (Ray Chiang via aw)
+
   BUG FIXES
 
     MAPREDUCE-6191. Improve clearing stale state of Java serialization

http://git-wip-us.apache.org/repos/asf/hadoop/blob/618ba707/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java
index c4957b7..a523291 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java
@@ -978,8 +978,9 @@ public class MapTask extends Task {
         throw new IOException(
             "Invalid \"" + JobContext.IO_SORT_MB + "\": " + sortmb);
       }
-      sorter = ReflectionUtils.newInstance(job.getClass("map.sort.class",
-            QuickSort.class, IndexedSorter.class), job);
+      sorter = ReflectionUtils.newInstance(job.getClass(
+                   MRJobConfig.MAP_SORT_CLASS, QuickSort.class,
+                   IndexedSorter.class), job);
       // buffers and accounting
       int maxMemUsage = sortmb << 20;
       maxMemUsage -= maxMemUsage % METASIZE;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/618ba707/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
index e64b9b6..0c2a2e9 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
@@ -28,6 +28,9 @@ import org.apache.hadoop.yarn.util.Apps;
 @InterfaceStability.Evolving
 public interface MRJobConfig {
 
+  // Used by MapTask
+  public static final String MAP_SORT_CLASS = "map.sort.class";
+
   // Put all of the attribute names in here so that Job and JobContext are
   // consistent.
   public static final String INPUT_FORMAT_CLASS_ATTR = "mapreduce.job.inputformat.class";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/618ba707/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
index 820c1ac..ff4511f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
@@ -58,13 +58,6 @@
 </property>
 
 <property>
-  <name>mapreduce.local.clientfactory.class.name</name>
-  <value>org.apache.hadoop.mapred.LocalClientFactory</value>
-  <description>This the client factory that is responsible for 
-  creating local job runner client</description>
-</property>
-
-<property>
   <name>mapreduce.job.maps</name>
   <value>2</value>
   <description>The default number of map tasks per job.
@@ -884,30 +877,6 @@
   </property>
   
   <property>
-    <name>mapreduce.map.skip.proc.count.autoincr</name>
-    <value>true</value>
-    <description> The flag which if set to true, 
-    SkipBadRecords.COUNTER_MAP_PROCESSED_RECORDS is incremented 
-    by MapRunner after invoking the map function. This value must be set to 
-    false for applications which process the records asynchronously 
-    or buffer the input records. For example streaming. 
-    In such cases applications should increment this counter on their own.
-    </description>
-  </property>
-  
-  <property>
-    <name>mapreduce.reduce.skip.proc.count.autoincr</name>
-    <value>true</value>
-    <description> The flag which if set to true, 
-    SkipBadRecords.COUNTER_REDUCE_PROCESSED_GROUPS is incremented 
-    by framework after invoking the reduce function. This value must be set to 
-    false for applications which process the records asynchronously 
-    or buffer the input records. For example streaming. 
-    In such cases applications should increment this counter on their own.
-    </description>
-  </property>
-  
-  <property>
     <name>mapreduce.job.skip.outdir</name>
     <value></value>
     <description> If no value is specified here, the skipped records are 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/618ba707/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/resources/job_1329348432655_0001_conf.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/resources/job_1329348432655_0001_conf.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/resources/job_1329348432655_0001_conf.xml
index 6d1c175..c469f78 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/resources/job_1329348432655_0001_conf.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/resources/job_1329348432655_0001_conf.xml
@@ -93,7 +93,6 @@
 <property><!--Loaded from job.xml--><name>mapreduce.reduce.input.buffer.percent</name><value>0.0</value></property>
 <property><!--Loaded from job.xml--><name>mapreduce.map.output.compress.codec</name><value>org.apache.hadoop.io.compress.DefaultCodec</value></property>
 <property><!--Loaded from job.xml--><name>yarn.resourcemanager.delegation-token.keepalive-time-ms</name><value>300000</value></property>
-<property><!--Loaded from job.xml--><name>mapreduce.map.skip.proc.count.autoincr</name><value>true</value></property>
 <property><!--Loaded from job.xml--><name>dfs.datanode.directoryscan.threads</name><value>1</value></property>
 <property><!--Loaded from job.xml--><name>mapreduce.jobtracker.address</name><value>local</value></property>
 <property><!--Loaded from job.xml--><name>mapreduce.cluster.local.dir</name><value>${hadoop.tmp.dir}/mapred/local</value></property>
@@ -197,7 +196,6 @@
 <property><!--Loaded from job.xml--><name>dfs.block.access.key.update.interval</name><value>600</value></property>
 <property><!--Loaded from job.xml--><name>mapreduce.jobhistory.move.interval-ms</name><value>30000</value></property>
 <property><!--Loaded from job.xml--><name>dfs.datanode.dns.interface</name><value>default</value></property>
-<property><!--Loaded from job.xml--><name>mapreduce.reduce.skip.proc.count.autoincr</name><value>true</value></property>
 <property><!--Loaded from job.xml--><name>dfs.namenode.backup.http-address</name><value>0.0.0.0:50105</value></property>
 <property><!--Loaded from job.xml--><name>yarn.nodemanager.container-monitor.interval-ms</name><value>3000</value></property>
 <property><!--Loaded from job.xml--><name>mapred.reducer.new-api</name><value>true</value></property>
@@ -290,7 +288,6 @@
 <property><!--Loaded from job.xml--><name>yarn.app.mapreduce.am.command-opts</name><value>-Xmx500m</value></property>
 <property><!--Loaded from job.xml--><name>mapreduce.admin.user.env</name><value>LD_LIBRARY_PATH=$HADOOP_COMMON_HOME/lib/native</value></property>
 <property><!--Loaded from job.xml--><name>dfs.namenode.checkpoint.edits.dir</name><value>${dfs.namenode.checkpoint.dir}</value></property>
-<property><!--Loaded from job.xml--><name>mapreduce.local.clientfactory.class.name</name><value>org.apache.hadoop.mapred.LocalClientFactory</value></property>
 <property><!--Loaded from job.xml--><name>hadoop.common.configuration.version</name><value>0.23.0</value></property>
 <property><!--Loaded from job.xml--><name>mapreduce.tasktracker.dns.interface</name><value>default</value></property>
 <property><!--Loaded from job.xml--><name>io.serializations</name><value>org.apache.hadoop.io.serializer.WritableSerialization,org.apache.hadoop.io.serializer.avro.AvroSpecificSerialization,org.apache.hadoop.io.serializer.avro.AvroReflectSerialization</value></property>
@@ -380,7 +377,6 @@
 <property><!--Loaded from job.xml--><name>fs.AbstractFileSystem.viewfs.impl</name><value>org.apache.hadoop.fs.viewfs.ViewFs</value></property>
 <property><!--Loaded from job.xml--><name>yarn.resourcemanager.resource-tracker.client.thread-count</name><value>50</value></property>
 <property><!--Loaded from job.xml--><name>mapreduce.tasktracker.dns.nameserver</name><value>default</value></property>
-<property><!--Loaded from job.xml--><name>mapreduce.clientfactory.class.name</name><value>org.apache.hadoop.mapred.YarnClientFactory</value></property>
 <property><!--Loaded from job.xml--><name>mapreduce.map.output.compress</name><value>false</value></property>
 <property><!--Loaded from job.xml--><name>mapreduce.job.counters.limit</name><value>120</value></property>
 <property><!--Loaded from job.xml--><name>dfs.datanode.ipc.address</name><value>0.0.0.0:50020</value></property>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/618ba707/hadoop-tools/hadoop-sls/src/main/data/2jobs2min-rumen-jh.json
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/data/2jobs2min-rumen-jh.json b/hadoop-tools/hadoop-sls/src/main/data/2jobs2min-rumen-jh.json
index f54c26f..4be3e08 100644
--- a/hadoop-tools/hadoop-sls/src/main/data/2jobs2min-rumen-jh.json
+++ b/hadoop-tools/hadoop-sls/src/main/data/2jobs2min-rumen-jh.json
@@ -4645,7 +4645,6 @@
     "dfs.ha.log-roll.period" : "120",
     "mapreduce.reduce.input.buffer.percent" : "0.0",
     "mapreduce.map.output.compress.codec" : "org.apache.hadoop.io.compress.SnappyCodec",
-    "mapreduce.map.skip.proc.count.autoincr" : "true",
     "dfs.client.failover.sleep.base.millis" : "500",
     "dfs.datanode.directoryscan.threads" : "1",
     "mapreduce.jobtracker.address" : "neededForHive:999999",
@@ -4765,7 +4764,6 @@
     "dfs.namenode.backup.address" : "0.0.0.0:50100",
     "hadoop.util.hash.type" : "murmur",
     "dfs.block.access.key.update.interval" : "600",
-    "mapreduce.reduce.skip.proc.count.autoincr" : "true",
     "dfs.datanode.dns.interface" : "default",
     "dfs.datanode.use.datanode.hostname" : "false",
     "mapreduce.job.output.key.class" : "org.apache.hadoop.io.Text",
@@ -4875,7 +4873,6 @@
     "mapreduce.reduce.shuffle.read.timeout" : "180000",
     "mapreduce.admin.user.env" : "LD_LIBRARY_PATH=$HADOOP_COMMON_HOME/lib/native",
     "yarn.app.mapreduce.am.command-opts" : " -Xmx1238932873",
-    "mapreduce.local.clientfactory.class.name" : "org.apache.hadoop.mapred.LocalClientFactory",
     "dfs.namenode.checkpoint.edits.dir" : "${dfs.namenode.checkpoint.dir}",
     "fs.permissions.umask-mode" : "022",
     "dfs.client.domain.socket.data.traffic" : "false",
@@ -9754,7 +9751,6 @@
     "dfs.ha.log-roll.period" : "120",
     "mapreduce.reduce.input.buffer.percent" : "0.0",
     "mapreduce.map.output.compress.codec" : "org.apache.hadoop.io.compress.SnappyCodec",
-    "mapreduce.map.skip.proc.count.autoincr" : "true",
     "dfs.client.failover.sleep.base.millis" : "500",
     "dfs.datanode.directoryscan.threads" : "1",
     "mapreduce.jobtracker.address" : "neededForHive:999999",
@@ -9874,7 +9870,6 @@
     "dfs.namenode.backup.address" : "0.0.0.0:50100",
     "hadoop.util.hash.type" : "murmur",
     "dfs.block.access.key.update.interval" : "600",
-    "mapreduce.reduce.skip.proc.count.autoincr" : "true",
     "dfs.datanode.dns.interface" : "default",
     "dfs.datanode.use.datanode.hostname" : "false",
     "mapreduce.job.output.key.class" : "org.apache.hadoop.io.Text",
@@ -9984,7 +9979,6 @@
     "mapreduce.reduce.shuffle.read.timeout" : "180000",
     "mapreduce.admin.user.env" : "LD_LIBRARY_PATH=$HADOOP_COMMON_HOME/lib/native",
     "yarn.app.mapreduce.am.command-opts" : " -Xmx1238932873",
-    "mapreduce.local.clientfactory.class.name" : "org.apache.hadoop.mapred.LocalClientFactory",
     "dfs.namenode.checkpoint.edits.dir" : "${dfs.namenode.checkpoint.dir}",
     "fs.permissions.umask-mode" : "022",
     "dfs.client.domain.socket.data.traffic" : "false",


[27/50] [abbrv] hadoop git commit: HADOOP-11881. test-patch.sh javac result is wildly wrong (Kengo Seki via aw)

Posted by ji...@apache.org.
HADOOP-11881. test-patch.sh javac result is wildly wrong (Kengo Seki via aw)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/eccf709a
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/eccf709a
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/eccf709a

Branch: refs/heads/HDFS-7240
Commit: eccf709a619b05aaa92b27693a9c302d349acf22
Parents: 99fe03e
Author: Allen Wittenauer <aw...@apache.org>
Authored: Tue Apr 28 10:32:32 2015 -0700
Committer: Allen Wittenauer <aw...@apache.org>
Committed: Tue Apr 28 10:32:32 2015 -0700

----------------------------------------------------------------------
 dev-support/test-patch.sh                       | 2 +-
 hadoop-common-project/hadoop-common/CHANGES.txt | 3 +++
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/eccf709a/dev-support/test-patch.sh
----------------------------------------------------------------------
diff --git a/dev-support/test-patch.sh b/dev-support/test-patch.sh
index e331deb..ae21837 100755
--- a/dev-support/test-patch.sh
+++ b/dev-support/test-patch.sh
@@ -1552,7 +1552,7 @@ function check_javac
         > "${PATCH_DIR}/diffJavacWarnings.txt"
 
         add_jira_table -1 javac "The applied patch generated "\
-        "$((patchJavacWarnings-${PATCH_BRANCH}JavacWarnings))" \
+        "$((patchJavacWarnings-branchJavacWarnings))" \
         " additional warning messages."
 
         add_jira_footer javac "@@BASE@@/diffJavacWarnings.txt"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eccf709a/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 1783e8d..39d14c9 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -576,6 +576,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11870. [JDK8] AuthenticationFilter, CertificateUtil,
     SignerSecretProviders, KeyAuthorizationKeyProvider Javadoc issues (rkanter)
 
+    HADOOP-11881. test-patch.sh javac result is wildly wrong (Kengo Seki via
+    aw)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES


[06/50] [abbrv] hadoop git commit: MAPREDUCE-6333. TestEvents, TestAMWebServicesTasks, TestAppController are broken due to MAPREDUCE-6297. (Siqi Li via gera)

Posted by ji...@apache.org.
MAPREDUCE-6333. TestEvents,TestAMWebServicesTasks,TestAppController are broken due to MAPREDUCE-6297. (Siqi Li via gera)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/78c6b462
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/78c6b462
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/78c6b462

Branch: refs/heads/HDFS-7240
Commit: 78c6b462412bbadad4a1a13ed4c597927b0cf188
Parents: 2f82ae0
Author: Gera Shegalov <ge...@apache.org>
Authored: Fri Apr 24 09:21:44 2015 -0700
Committer: Gera Shegalov <ge...@apache.org>
Committed: Fri Apr 24 17:31:10 2015 -0700

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |  3 ++
 .../hadoop/mapreduce/jobhistory/TestEvents.java | 29 ++++++++++----------
 .../v2/app/webapp/TestAMWebServicesTasks.java   | 27 ++++++++++++------
 .../v2/app/webapp/TestAppController.java        |  9 +++---
 4 files changed, 41 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/78c6b462/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 5b26910..397f94a 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -343,6 +343,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6330. Fix typo in Task Attempt API's URL in documentations.
     (Ryu Kobayashi via ozawa)
 
+    MAPREDUCE-6333. TestEvents,TestAMWebServicesTasks,TestAppController are
+    broken due to MAPREDUCE-6297. (Siqi Li via gera)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78c6b462/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java
index 00be4b8..bb9b56b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java
@@ -39,6 +39,7 @@ import org.junit.Test;
 
 public class TestEvents {
 
+  private static final String taskId = "task_1_2_r_3";
   /**
    * test a getters of TaskAttemptFinishedEvent and TaskAttemptFinished
    * 
@@ -131,7 +132,7 @@ public class TestEvents {
 
     e = reader.getNextEvent();
     assertTrue(e.getEventType().equals(EventType.REDUCE_ATTEMPT_KILLED));
-    assertEquals("task_1_2_r03_4",
+    assertEquals(taskId,
         ((TaskAttemptUnsuccessfulCompletion) e.getDatum()).taskid.toString());
 
     e = reader.getNextEvent();
@@ -141,42 +142,42 @@ public class TestEvents {
 
     e = reader.getNextEvent();
     assertTrue(e.getEventType().equals(EventType.REDUCE_ATTEMPT_STARTED));
-    assertEquals("task_1_2_r03_4",
+    assertEquals(taskId,
         ((TaskAttemptStarted) e.getDatum()).taskid.toString());
 
     e = reader.getNextEvent();
     assertTrue(e.getEventType().equals(EventType.REDUCE_ATTEMPT_FINISHED));
-    assertEquals("task_1_2_r03_4",
+    assertEquals(taskId,
         ((TaskAttemptFinished) e.getDatum()).taskid.toString());
 
     e = reader.getNextEvent();
     assertTrue(e.getEventType().equals(EventType.REDUCE_ATTEMPT_KILLED));
-    assertEquals("task_1_2_r03_4",
+    assertEquals(taskId,
         ((TaskAttemptUnsuccessfulCompletion) e.getDatum()).taskid.toString());
 
     e = reader.getNextEvent();
     assertTrue(e.getEventType().equals(EventType.REDUCE_ATTEMPT_KILLED));
-    assertEquals("task_1_2_r03_4",
+    assertEquals(taskId,
         ((TaskAttemptUnsuccessfulCompletion) e.getDatum()).taskid.toString());
 
     e = reader.getNextEvent();
     assertTrue(e.getEventType().equals(EventType.REDUCE_ATTEMPT_STARTED));
-    assertEquals("task_1_2_r03_4",
+    assertEquals(taskId,
         ((TaskAttemptStarted) e.getDatum()).taskid.toString());
 
     e = reader.getNextEvent();
     assertTrue(e.getEventType().equals(EventType.REDUCE_ATTEMPT_FINISHED));
-    assertEquals("task_1_2_r03_4",
+    assertEquals(taskId,
         ((TaskAttemptFinished) e.getDatum()).taskid.toString());
 
     e = reader.getNextEvent();
     assertTrue(e.getEventType().equals(EventType.REDUCE_ATTEMPT_KILLED));
-    assertEquals("task_1_2_r03_4",
+    assertEquals(taskId,
         ((TaskAttemptUnsuccessfulCompletion) e.getDatum()).taskid.toString());
 
     e = reader.getNextEvent();
     assertTrue(e.getEventType().equals(EventType.REDUCE_ATTEMPT_KILLED));
-    assertEquals("task_1_2_r03_4",
+    assertEquals(taskId,
         ((TaskAttemptUnsuccessfulCompletion) e.getDatum()).taskid.toString());
 
     reader.close();
@@ -234,7 +235,7 @@ public class TestEvents {
     datum.hostname = "hostname";
     datum.rackname = "rackname";
     datum.physMemKbytes = Arrays.asList(1000, 2000, 3000);
-    datum.taskid = "task_1_2_r03_4";
+    datum.taskid = taskId;
     datum.port = 1000;
     datum.taskType = "REDUCE";
     datum.status = "STATUS";
@@ -260,7 +261,7 @@ public class TestEvents {
     datum.hostname = "hostname";
     datum.rackname = "rackName";
     datum.state = "state";
-    datum.taskid = "task_1_2_r03_4";
+    datum.taskid = taskId;
     datum.taskStatus = "taskStatus";
     datum.taskType = "REDUCE";
     result.setDatum(datum);
@@ -278,7 +279,7 @@ public class TestEvents {
     datum.locality = "locality";
     datum.shufflePort = 10001;
     datum.startTime = 1;
-    datum.taskid = "task_1_2_r03_4";
+    datum.taskid = taskId;
     datum.taskType = "taskType";
     datum.trackerName = "trackerName";
     result.setDatum(datum);
@@ -308,7 +309,7 @@ public class TestEvents {
     datum.hostname = "hostname";
     datum.rackname = "rackname";
     datum.state = "state";
-    datum.taskid = "task_1_2_r03_4";
+    datum.taskid = taskId;
     datum.taskStatus = "taskStatus";
     datum.taskType = "REDUCE";
     result.setDatum(datum);
@@ -325,7 +326,7 @@ public class TestEvents {
     datum.locality = "locality";
     datum.shufflePort = 10001;
     datum.startTime = 1;
-    datum.taskid = "task_1_2_r03_4";
+    datum.taskid = taskId;
     datum.taskType = "taskType";
     datum.trackerName = "trackerName";
     result.setDatum(datum);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78c6b462/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesTasks.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesTasks.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesTasks.java
index 8bf1bb7..15ef5ac 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesTasks.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesTasks.java
@@ -31,6 +31,7 @@ import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskReport;
@@ -355,9 +356,11 @@ public class TestAMWebServicesTasks extends JerseyTest {
         String message = exception.getString("message");
         String type = exception.getString("exception");
         String classname = exception.getString("javaClassName");
-        WebServicesTestUtils.checkStringMatch("exception message",
+        WebServicesTestUtils.checkStringEqual("exception message",
             "java.lang.Exception: TaskId string : "
-                + "bogustaskid is not properly formed", message);
+                + "bogustaskid is not properly formed"
+                + "\nReason: java.util.regex.Matcher[pattern=" +
+                TaskID.TASK_ID_REGEX + " region=0,11 lastmatch=]", message);
         WebServicesTestUtils.checkStringMatch("exception type",
             "NotFoundException", type);
         WebServicesTestUtils.checkStringMatch("exception classname",
@@ -419,9 +422,11 @@ public class TestAMWebServicesTasks extends JerseyTest {
         String message = exception.getString("message");
         String type = exception.getString("exception");
         String classname = exception.getString("javaClassName");
-        WebServicesTestUtils.checkStringMatch("exception message",
-            "java.lang.Exception: Bad TaskType identifier. TaskId string : "
-                + "task_0_0000_d_000000 is not properly formed.", message);
+        WebServicesTestUtils.checkStringEqual("exception message",
+            "java.lang.Exception: TaskId string : "
+                + "task_0_0000_d_000000 is not properly formed"
+                + "\nReason: java.util.regex.Matcher[pattern=" +
+                TaskID.TASK_ID_REGEX + " region=0,20 lastmatch=]", message);
         WebServicesTestUtils.checkStringMatch("exception type",
             "NotFoundException", type);
         WebServicesTestUtils.checkStringMatch("exception classname",
@@ -451,9 +456,11 @@ public class TestAMWebServicesTasks extends JerseyTest {
         String message = exception.getString("message");
         String type = exception.getString("exception");
         String classname = exception.getString("javaClassName");
-        WebServicesTestUtils.checkStringMatch("exception message",
+        WebServicesTestUtils.checkStringEqual("exception message",
             "java.lang.Exception: TaskId string : "
-                + "task_0_m_000000 is not properly formed", message);
+                + "task_0_m_000000 is not properly formed"
+                + "\nReason: java.util.regex.Matcher[pattern=" +
+                TaskID.TASK_ID_REGEX + " region=0,15 lastmatch=]", message);
         WebServicesTestUtils.checkStringMatch("exception type",
             "NotFoundException", type);
         WebServicesTestUtils.checkStringMatch("exception classname",
@@ -483,9 +490,11 @@ public class TestAMWebServicesTasks extends JerseyTest {
         String message = exception.getString("message");
         String type = exception.getString("exception");
         String classname = exception.getString("javaClassName");
-        WebServicesTestUtils.checkStringMatch("exception message",
+        WebServicesTestUtils.checkStringEqual("exception message",
             "java.lang.Exception: TaskId string : "
-                + "task_0_0000_m is not properly formed", message);
+                + "task_0_0000_m is not properly formed"
+                + "\nReason: java.util.regex.Matcher[pattern=" +
+                TaskID.TASK_ID_REGEX + " region=0,13 lastmatch=]", message);
         WebServicesTestUtils.checkStringMatch("exception type",
             "NotFoundException", type);
         WebServicesTestUtils.checkStringMatch("exception classname",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78c6b462/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAppController.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAppController.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAppController.java
index c1bc9ef..92786e3 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAppController.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAppController.java
@@ -45,6 +45,7 @@ public class TestAppController {
   private AppControllerForTest appController;
   private RequestContext ctx;
   private Job job;
+  private static final String taskId = "task_01_01_m_01";
 
   @Before
   public void setUp() throws IOException {
@@ -70,7 +71,7 @@ public class TestAppController {
 
     appController = new AppControllerForTest(app, configuration, ctx);
     appController.getProperty().put(AMParams.JOB_ID, "job_01_01");
-    appController.getProperty().put(AMParams.TASK_ID, "task_01_01_m01_01");
+    appController.getProperty().put(AMParams.TASK_ID, taskId);
 
   }
 
@@ -205,7 +206,7 @@ public class TestAppController {
         "Access denied: User user does not have permission to view job job_01_01missing task ID",
         appController.getData());
 
-    appController.getProperty().put(AMParams.TASK_ID, "task_01_01_m01_01");
+    appController.getProperty().put(AMParams.TASK_ID, taskId);
     appController.taskCounters();
     assertEquals(CountersPage.class, appController.getClazz());
   }
@@ -247,7 +248,7 @@ public class TestAppController {
   public void testTask() {
  
     appController.task();
-    assertEquals("Attempts for task_01_01_m01_01" ,
+    assertEquals("Attempts for " + taskId ,
         appController.getProperty().get("title"));
 
     assertEquals(TaskPage.class, appController.getClazz());
@@ -290,7 +291,7 @@ public class TestAppController {
         "Access denied: User user does not have permission to view job job_01_01",
         appController.getData());
 
-    appController.getProperty().put(AMParams.TASK_ID, "task_01_01_m01_01");
+    appController.getProperty().put(AMParams.TASK_ID, taskId);
     appController.attempts();
     assertEquals("Bad request: missing task-type.", appController.getProperty()
         .get("title"));


[40/50] [abbrv] hadoop git commit: HDFS-5574. Remove buffer copy in BlockReader.skip. Contributed by Binglin Chang.

Posted by ji...@apache.org.
HDFS-5574. Remove buffer copy in BlockReader.skip. Contributed by Binglin Chang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e89fc53a
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e89fc53a
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e89fc53a

Branch: refs/heads/HDFS-7240
Commit: e89fc53a1d264fde407dd2c36defab5241cd0b52
Parents: f5b3847
Author: Akira Ajisaka <aa...@apache.org>
Authored: Thu Apr 30 19:09:57 2015 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Thu Apr 30 19:11:22 2015 +0900

----------------------------------------------------------------------
 .../org/apache/hadoop/fs/FSInputChecker.java    |  25 +++-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../apache/hadoop/hdfs/RemoteBlockReader.java   |  18 +--
 .../apache/hadoop/hdfs/RemoteBlockReader2.java  |  37 +++---
 .../apache/hadoop/hdfs/TestBlockReaderBase.java |  94 +++++++++++++++
 .../apache/hadoop/hdfs/TestDFSInputStream.java  | 114 +++++++++++++++++++
 .../hadoop/hdfs/TestRemoteBlockReader.java      |  27 +++++
 .../hadoop/hdfs/TestRemoteBlockReader2.java     |  25 ++++
 8 files changed, 309 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e89fc53a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSInputChecker.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSInputChecker.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSInputChecker.java
index 889ccc1..9b66c95 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSInputChecker.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSInputChecker.java
@@ -214,7 +214,30 @@ abstract public class FSInputChecker extends FSInputStream {
     count = readChecksumChunk(buf, 0, maxChunkSize);
     if (count < 0) count = 0;
   }
-  
+
+  /**
+   * Like read(byte[], int, int), but does not provide a dest buffer,
+   * so the read data is discarded.
+   * @param      len maximum number of bytes to read.
+   * @return     the number of bytes read.
+   * @throws     IOException  if an I/O error occurs.
+   */
+  final protected synchronized int readAndDiscard(int len) throws IOException {
+    int total = 0;
+    while (total < len) {
+      if (pos >= count) {
+        count = readChecksumChunk(buf, 0, maxChunkSize);
+        if (count <= 0) {
+          break;
+        }
+      }
+      int rd = Math.min(count - pos, len - total);
+      pos += rd;
+      total += rd;
+    }
+    return total;
+  }
+
   /*
    * Read characters into a portion of an array, reading from the underlying
    * stream at most once if necessary.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e89fc53a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index fbeb45d..c538b78 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -483,6 +483,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8283. DataStreamer cleanup and some minor improvement. (szetszwo via
     jing9)
 
+    HDFS-5574. Remove buffer copy in BlockReader.skip.
+    (Binglin Chang via aajisaka)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e89fc53a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
index ce96ac9..d70f419 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
@@ -97,7 +97,6 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
   private boolean eos = false;
   private boolean sentStatusCode = false;
   
-  byte[] skipBuf = null;
   ByteBuffer checksumBytes = null;
   /** Amount of unread data in the current received packet */
   int dataLeft = 0;
@@ -126,10 +125,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
     if (lastChunkLen < 0 && startOffset > firstChunkOffset && len > 0) {
       // Skip these bytes. But don't call this.skip()!
       int toSkip = (int)(startOffset - firstChunkOffset);
-      if ( skipBuf == null ) {
-        skipBuf = new byte[bytesPerChecksum];
-      }
-      if ( super.read(skipBuf, 0, toSkip) != toSkip ) {
+      if ( super.readAndDiscard(toSkip) != toSkip ) {
         // should never happen
         throw new IOException("Could not skip required number of bytes");
       }
@@ -152,15 +148,11 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
   public synchronized long skip(long n) throws IOException {
     /* How can we make sure we don't throw a ChecksumException, at least
      * in majority of the cases?. This one throws. */  
-    if ( skipBuf == null ) {
-      skipBuf = new byte[bytesPerChecksum]; 
-    }
-
     long nSkipped = 0;
-    while ( nSkipped < n ) {
-      int toSkip = (int)Math.min(n-nSkipped, skipBuf.length);
-      int ret = read(skipBuf, 0, toSkip);
-      if ( ret <= 0 ) {
+    while (nSkipped < n) {
+      int toSkip = (int)Math.min(n-nSkipped, Integer.MAX_VALUE);
+      int ret = readAndDiscard(toSkip);
+      if (ret <= 0) {
         return nSkipped;
       }
       nSkipped += ret;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e89fc53a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
index 9245a84..c368d65 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
@@ -122,12 +122,7 @@ public class RemoteBlockReader2  implements BlockReader {
   private final boolean verifyChecksum;
 
   private boolean sentStatusCode = false;
-  
-  byte[] skipBuf = null;
-  ByteBuffer checksumBytes = null;
-  /** Amount of unread data in the current received packet */
-  int dataLeft = 0;
-  
+
   @VisibleForTesting
   public Peer getPeer() {
     return peer;
@@ -172,7 +167,7 @@ public class RemoteBlockReader2  implements BlockReader {
 
 
   @Override
-  public int read(ByteBuffer buf) throws IOException {
+  public synchronized int read(ByteBuffer buf) throws IOException {
     if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
       TraceScope scope = Trace.startSpan(
           "RemoteBlockReader2#readNextPacket(" + blockId + ")", Sampler.NEVER);
@@ -257,21 +252,23 @@ public class RemoteBlockReader2  implements BlockReader {
   @Override
   public synchronized long skip(long n) throws IOException {
     /* How can we make sure we don't throw a ChecksumException, at least
-     * in majority of the cases?. This one throws. */  
-    if ( skipBuf == null ) {
-      skipBuf = new byte[bytesPerChecksum]; 
-    }
-
-    long nSkipped = 0;
-    while ( nSkipped < n ) {
-      int toSkip = (int)Math.min(n-nSkipped, skipBuf.length);
-      int ret = read(skipBuf, 0, toSkip);
-      if ( ret <= 0 ) {
-        return nSkipped;
+     * in majority of the cases?. This one throws. */
+    long skipped = 0;
+    while (skipped < n) {
+      long needToSkip = n - skipped;
+      if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
+        readNextPacket();
       }
-      nSkipped += ret;
+      if (curDataSlice.remaining() == 0) {
+        // we're at EOF now
+        break;
+      }
+
+      int skip = (int)Math.min(curDataSlice.remaining(), needToSkip);
+      curDataSlice.position(curDataSlice.position() + skip);
+      skipped += skip;
     }
-    return nSkipped;
+    return skipped;
   }
 
   private void readTrailingEmptyPacket() throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e89fc53a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderBase.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderBase.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderBase.java
new file mode 100644
index 0000000..3d916a7
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderBase.java
@@ -0,0 +1,94 @@
+/**
+ * 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.hdfs;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.util.Random;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+abstract public class TestBlockReaderBase {
+  private BlockReaderTestUtil util;
+  private byte[] blockData;
+  private BlockReader reader;
+
+  /**
+   * if override this, make sure return array length is less than
+   * block size.
+   */
+  byte [] getBlockData() {
+    int length = 1 << 22;
+    byte[] data = new byte[length];
+    for (int i = 0; i < length; i++) {
+      data[i] = (byte) (i % 133);
+    }
+    return data;
+  }
+
+  private BlockReader getBlockReader(LocatedBlock block) throws Exception {
+    return util.getBlockReader(block, 0, blockData.length);
+  }
+
+  abstract HdfsConfiguration createConf();
+
+  @Before
+  public void setup() throws Exception {
+    util = new BlockReaderTestUtil(1, createConf());
+    blockData = getBlockData();
+    DistributedFileSystem fs = util.getCluster().getFileSystem();
+    Path testfile = new Path("/testfile");
+    FSDataOutputStream fout = fs.create(testfile);
+    fout.write(blockData);
+    fout.close();
+    LocatedBlock blk = util.getFileBlocks(testfile, blockData.length).get(0);
+    reader = getBlockReader(blk);
+  }
+
+  @After
+  public void shutdown() throws Exception {
+    util.shutdown();
+  }
+
+  @Test(timeout=60000)
+  public void testSkip() throws IOException {
+    Random random = new Random();
+    byte [] buf = new byte[1];
+    for (int pos = 0; pos < blockData.length;) {
+      long skip = random.nextInt(100) + 1;
+      long skipped = reader.skip(skip);
+      if (pos + skip >= blockData.length) {
+        assertEquals(blockData.length, pos + skipped);
+        break;
+      } else {
+        assertEquals(skip, skipped);
+        pos += skipped;
+        assertEquals(1, reader.read(buf, 0, 1));
+
+        assertEquals(blockData[pos], buf[0]);
+        pos += 1;
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e89fc53a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInputStream.java
new file mode 100644
index 0000000..b9ec2ce
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInputStream.java
@@ -0,0 +1,114 @@
+/**
+ * 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.hdfs;
+
+import static org.junit.Assert.assertEquals;
+import static org.hamcrest.CoreMatchers.equalTo;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Random;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.net.unix.DomainSocket;
+import org.apache.hadoop.net.unix.TemporarySocketDirectory;
+import org.junit.Assume;
+import org.junit.Test;
+
+public class TestDFSInputStream {
+  private void testSkipInner(MiniDFSCluster cluster) throws IOException {
+    DistributedFileSystem fs = cluster.getFileSystem();
+    DFSClient client = fs.dfs;
+    Path file = new Path("/testfile");
+    int fileLength = 1 << 22;
+    byte[] fileContent = new byte[fileLength];
+    for (int i = 0; i < fileLength; i++) {
+      fileContent[i] = (byte) (i % 133);
+    }
+    FSDataOutputStream fout = fs.create(file);
+    fout.write(fileContent);
+    fout.close();
+    Random random = new Random();
+    for (int i = 3; i < 18; i++) {
+      DFSInputStream fin = client.open("/testfile");
+      for (long pos = 0; pos < fileLength;) {
+        long skip = random.nextInt(1 << i) + 1;
+        long skipped = fin.skip(skip);
+        if (pos + skip >= fileLength) {
+          assertEquals(fileLength, pos + skipped);
+          break;
+        } else {
+          assertEquals(skip, skipped);
+          pos += skipped;
+          int data = fin.read();
+          assertEquals(pos % 133, data);
+          pos += 1;
+        }
+      }
+      fin.close();
+    }
+  }
+
+  @Test(timeout=60000)
+  public void testSkipWithRemoteBlockReader() throws IOException {
+    Configuration conf = new Configuration();
+    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADER, true);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
+    try {
+      testSkipInner(cluster);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  @Test(timeout=60000)
+  public void testSkipWithRemoteBlockReader2() throws IOException {
+    Configuration conf = new Configuration();
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
+    try {
+      testSkipInner(cluster);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  @Test(timeout=60000)
+  public void testSkipWithLocalBlockReader() throws IOException {
+    Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
+    TemporarySocketDirectory sockDir = new TemporarySocketDirectory();
+    DomainSocket.disableBindPathValidation();
+    Configuration conf = new Configuration();
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, true);
+    conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
+        new File(sockDir.getDir(),
+          "TestShortCircuitLocalRead._PORT.sock").getAbsolutePath());
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
+    try {
+      DFSInputStream.tcpReadsDisabledForTesting = true;
+      testSkipInner(cluster);
+    } finally {
+      DFSInputStream.tcpReadsDisabledForTesting = false;
+      cluster.shutdown();
+      sockDir.close();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e89fc53a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRemoteBlockReader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRemoteBlockReader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRemoteBlockReader.java
new file mode 100644
index 0000000..8ab110d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRemoteBlockReader.java
@@ -0,0 +1,27 @@
+/**
+ * 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.hdfs;
+
+public class TestRemoteBlockReader extends TestBlockReaderBase {
+
+  HdfsConfiguration createConf() {
+    HdfsConfiguration conf = new HdfsConfiguration();
+    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADER, true);
+    return conf;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e89fc53a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRemoteBlockReader2.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRemoteBlockReader2.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRemoteBlockReader2.java
new file mode 100644
index 0000000..c23b4b7
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRemoteBlockReader2.java
@@ -0,0 +1,25 @@
+/**
+ * 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.hdfs;
+
+public class TestRemoteBlockReader2 extends TestBlockReaderBase {
+  HdfsConfiguration createConf() {
+    HdfsConfiguration conf = new HdfsConfiguration();
+    return conf;
+  }
+}


[18/50] [abbrv] hadoop git commit: YARN-3530. ATS throws exception on trying to filter results without otherinfo. Contributed by zhijie shen

Posted by ji...@apache.org.
YARN-3530. ATS throws exception on trying to filter results without
otherinfo. Contributed by zhijie shen


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/7f07c4d8
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7f07c4d8
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7f07c4d8

Branch: refs/heads/HDFS-7240
Commit: 7f07c4d81023e3bf4bf8980e64cc9420ec31cf55
Parents: 9a3dda3
Author: Xuan <xg...@apache.org>
Authored: Mon Apr 27 10:36:42 2015 -0700
Committer: Xuan <xg...@apache.org>
Committed: Mon Apr 27 10:36:42 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../server/timeline/LeveldbTimelineStore.java   | 34 ++++++-
 .../server/timeline/TimelineStoreTestUtils.java | 99 ++++++++++++++------
 3 files changed, 104 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f07c4d8/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 87db291..fdc3f4a 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -271,6 +271,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3464. Race condition in LocalizerRunner kills localizer before 
     localizing all resources. (Zhihai Xu via kasha)
 
+    YARN-3530. ATS throws exception on trying to filter results without otherinfo.
+    (zhijie shen via xgong)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f07c4d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/LeveldbTimelineStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/LeveldbTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/LeveldbTimelineStore.java
index d521f70..8cfa0c7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/LeveldbTimelineStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/LeveldbTimelineStore.java
@@ -357,6 +357,9 @@ public class LeveldbTimelineStore extends AbstractService
       iterator = new LeveldbIterator(db);
       iterator.seek(prefix);
 
+      if (fields == null) {
+        fields = EnumSet.allOf(Field.class);
+      }
       return getEntity(entityId, entityType, revStartTime, fields, iterator,
           prefix, prefix.length);
     } catch(DBException e) {
@@ -373,10 +376,6 @@ public class LeveldbTimelineStore extends AbstractService
   private static TimelineEntity getEntity(String entityId, String entityType,
       Long startTime, EnumSet<Field> fields, LeveldbIterator iterator,
       byte[] prefix, int prefixlen) throws IOException {
-    if (fields == null) {
-      fields = EnumSet.allOf(Field.class);
-    }
-
     TimelineEntity entity = new TimelineEntity();
     boolean events = false;
     boolean lastEvent = false;
@@ -590,6 +589,25 @@ public class LeveldbTimelineStore extends AbstractService
       String entityType, Long limit, Long starttime, Long endtime,
       String fromId, Long fromTs, Collection<NameValuePair> secondaryFilters,
       EnumSet<Field> fields, CheckAcl checkAcl) throws IOException {
+    // Even if other info and primary filter fields are not included, we
+    // still need to load them to match secondary filters when they are
+    // non-empty
+    if (fields == null) {
+      fields = EnumSet.allOf(Field.class);
+    }
+    boolean addPrimaryFilters = false;
+    boolean addOtherInfo = false;
+    if (secondaryFilters != null && secondaryFilters.size() > 0) {
+      if (!fields.contains(Field.PRIMARY_FILTERS)) {
+        fields.add(Field.PRIMARY_FILTERS);
+        addPrimaryFilters = true;
+      }
+      if (!fields.contains(Field.OTHER_INFO)) {
+        fields.add(Field.OTHER_INFO);
+        addOtherInfo = true;
+      }
+    }
+
     LeveldbIterator iterator = null;
     try {
       KeyBuilder kb = KeyBuilder.newInstance().add(base).add(entityType);
@@ -690,6 +708,14 @@ public class LeveldbTimelineStore extends AbstractService
             entity.setDomainId(DEFAULT_DOMAIN_ID);
           }
           if (checkAcl == null || checkAcl.check(entity)) {
+            // Remove primary filter and other info if they are added for
+            // matching secondary filters
+            if (addPrimaryFilters) {
+              entity.setPrimaryFilters(null);
+            }
+            if (addOtherInfo) {
+              entity.setOtherInfo(null);
+            }
             entities.addEntity(entity);
           }
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f07c4d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineStoreTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineStoreTestUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineStoreTestUtils.java
index da71f46..6ac5a35 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineStoreTestUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineStoreTestUtils.java
@@ -28,6 +28,7 @@ import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -487,6 +488,13 @@ public class TimelineStoreTestUtils {
         primaryFilter, secondaryFilters, null, null).getEntities();
   }
 
+  protected List<TimelineEntity> getEntitiesWithFilters(String entityType,
+      NameValuePair primaryFilter, Collection<NameValuePair> secondaryFilters,
+      EnumSet<Field> fields) throws IOException {
+    return store.getEntities(entityType, null, null, null, null, null,
+        primaryFilter, secondaryFilters, fields, null).getEntities();
+  }
+
   protected List<TimelineEntity> getEntities(String entityType, Long limit,
       Long windowStart, Long windowEnd, NameValuePair primaryFilter,
       EnumSet<Field> fields) throws IOException {
@@ -751,38 +759,73 @@ public class TimelineStoreTestUtils {
   }
 
   public void testGetEntitiesWithSecondaryFilters() throws IOException {
-    // test using secondary filter
-    List<TimelineEntity> entities = getEntitiesWithFilters("type_1", null,
-        goodTestingFilters);
-    assertEquals(3, entities.size());
-    verifyEntityInfo(entityId1, entityType1, events1, EMPTY_REL_ENTITIES,
-        primaryFilters, otherInfo, entities.get(0), domainId1);
-    verifyEntityInfo(entityId1b, entityType1, events1, EMPTY_REL_ENTITIES,
-        primaryFilters, otherInfo, entities.get(1), domainId1);
-    verifyEntityInfo(entityId6, entityType1, EMPTY_EVENTS, EMPTY_REL_ENTITIES,
-        primaryFilters, otherInfo, entities.get(2), domainId2);
-
-    entities = getEntitiesWithFilters("type_1", userFilter, goodTestingFilters);
-    assertEquals(3, entities.size());
-    verifyEntityInfo(entityId1, entityType1, events1, EMPTY_REL_ENTITIES,
-        primaryFilters, otherInfo, entities.get(0), domainId1);
-    verifyEntityInfo(entityId1b, entityType1, events1, EMPTY_REL_ENTITIES,
-        primaryFilters, otherInfo, entities.get(1), domainId1);
-    verifyEntityInfo(entityId6, entityType1, EMPTY_EVENTS, EMPTY_REL_ENTITIES,
-        primaryFilters, otherInfo, entities.get(2), domainId2);
+    for (int i = 0; i < 4; ++i) {
+      // Verify the secondary filter works both other info is included or not.
+      EnumSet<Field> fields = null;
+      if (i == 1) {
+        fields = EnumSet.noneOf(Field.class);
+      } else if (i == 2) {
+        fields = EnumSet.of(Field.PRIMARY_FILTERS);
+      } else if (i == 3) {
+        fields = EnumSet.of(Field.OTHER_INFO);
+      }
+      // test using secondary filter
+      List<TimelineEntity> entities = getEntitiesWithFilters("type_1", null,
+          goodTestingFilters, fields);
+      assertEquals(3, entities.size());
+      verifyEntityInfo(entityId1, entityType1,
+          (i == 0 ? events1 : null),
+          (i == 0 ? EMPTY_REL_ENTITIES : null),
+          (i == 0 || i == 2 ? primaryFilters : null),
+          (i == 0 || i == 3 ? otherInfo : null), entities.get(0), domainId1);
+      verifyEntityInfo(entityId1b, entityType1,
+          (i == 0 ? events1 : null),
+          (i == 0 ? EMPTY_REL_ENTITIES : null),
+          (i == 0 || i == 2 ? primaryFilters : null),
+          (i == 0 || i == 3 ? otherInfo : null), entities.get(1), domainId1);
+      verifyEntityInfo(entityId6, entityType1,
+          (i == 0 ? EMPTY_EVENTS : null),
+          (i == 0 ? EMPTY_REL_ENTITIES : null),
+          (i == 0 || i == 2 ? primaryFilters : null),
+          (i == 0 || i == 3 ? otherInfo : null), entities.get(2), domainId2);
+
+      entities =
+          getEntitiesWithFilters("type_1", userFilter, goodTestingFilters, fields);
+      assertEquals(3, entities.size());
+      if (i == 0) {
+        verifyEntityInfo(entityId1, entityType1,
+            (i == 0 ? events1 : null),
+            (i == 0 ? EMPTY_REL_ENTITIES : null),
+            (i == 0 || i == 2 ? primaryFilters : null),
+            (i == 0 || i == 3 ? otherInfo : null), entities.get(0), domainId1);
+        verifyEntityInfo(entityId1b, entityType1,
+            (i == 0 ? events1 : null),
+            (i == 0 ? EMPTY_REL_ENTITIES : null),
+            (i == 0 || i == 2 ? primaryFilters : null),
+            (i == 0 || i == 3 ? otherInfo : null), entities.get(1), domainId1);
+        verifyEntityInfo(entityId6, entityType1,
+            (i == 0 ? EMPTY_EVENTS : null),
+            (i == 0 ? EMPTY_REL_ENTITIES : null),
+            (i == 0 || i == 2 ? primaryFilters : null),
+            (i == 0 || i == 3 ? otherInfo : null), entities.get(2), domainId2);
+      }
 
-    entities = getEntitiesWithFilters("type_1", null,
-        Collections.singleton(new NameValuePair("user", "none")));
-    assertEquals(0, entities.size());
+      entities = getEntitiesWithFilters("type_1", null,
+          Collections.singleton(new NameValuePair("user", "none")), fields);
+      assertEquals(0, entities.size());
 
-    entities = getEntitiesWithFilters("type_1", null, badTestingFilters);
-    assertEquals(0, entities.size());
+      entities =
+          getEntitiesWithFilters("type_1", null, badTestingFilters, fields);
+      assertEquals(0, entities.size());
 
-    entities = getEntitiesWithFilters("type_1", userFilter, badTestingFilters);
-    assertEquals(0, entities.size());
+      entities =
+          getEntitiesWithFilters("type_1", userFilter, badTestingFilters, fields);
+      assertEquals(0, entities.size());
 
-    entities = getEntitiesWithFilters("type_5", null, badTestingFilters);
-    assertEquals(0, entities.size());
+      entities =
+          getEntitiesWithFilters("type_5", null, badTestingFilters, fields);
+      assertEquals(0, entities.size());
+    }
   }
 
   public void testGetEvents() throws IOException {


[24/50] [abbrv] hadoop git commit: YARN-2740. Fix NodeLabelsManager to properly handle node label modifications when distributed node label configuration enabled. (Naganarasimha G R via wangda)

Posted by ji...@apache.org.
YARN-2740. Fix NodeLabelsManager to properly handle node label modifications when distributed node label configuration enabled. (Naganarasimha G R via wangda)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/db1b674b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/db1b674b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/db1b674b

Branch: refs/heads/HDFS-7240
Commit: db1b674b50ddecf2774f4092d677c412722bdcb1
Parents: 9fc32c5
Author: Wangda Tan <wa...@apache.org>
Authored: Mon Apr 27 16:24:18 2015 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Mon Apr 27 16:24:38 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../hadoop/yarn/conf/YarnConfiguration.java     |   6 +
 .../nodelabels/CommonNodeLabelsManager.java     |  20 +++-
 .../nodelabels/FileSystemNodeLabelsStore.java   |  16 ++-
 .../hadoop/yarn/nodelabels/NodeLabelsStore.java |  13 +-
 .../DummyCommonNodeLabelsManager.java           |   3 +-
 .../nodelabels/TestCommonNodeLabelsManager.java |  25 ++++
 .../TestFileSystemNodeLabelsStore.java          |  34 ++++++
 .../server/resourcemanager/AdminService.java    |  49 +++++---
 .../resourcemanager/ResourceTrackerService.java |  15 +--
 .../resourcemanager/webapp/RMWebServices.java   | 119 ++++++++++---------
 .../resourcemanager/TestRMAdminService.java     |  68 +++++++++++
 .../nodelabels/NullRMNodeLabelsManager.java     |   3 +-
 .../webapp/TestRMWebServices.java               |   2 +
 .../webapp/TestRMWebServicesNodeLabels.java     | 107 ++++++++++++++---
 15 files changed, 379 insertions(+), 104 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/db1b674b/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 1ac7a13..9039460 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -265,6 +265,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3530. ATS throws exception on trying to filter results without otherinfo.
     (zhijie shen via xgong)
 
+    YARN-2740. Fix NodeLabelsManager to properly handle node label modifications 
+    when distributed node label configuration enabled. (Naganarasimha G R via wangda)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/db1b674b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index a7f485d..eb568b9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -1779,6 +1779,12 @@ public class YarnConfiguration extends Configuration {
   public static final String DEFAULT_NODELABEL_CONFIGURATION_TYPE =
       CENTALIZED_NODELABEL_CONFIGURATION_TYPE;
 
+  @Private
+  public static boolean isDistributedNodeLabelConfiguration(Configuration conf) {
+    return DISTRIBUTED_NODELABEL_CONFIGURATION_TYPE.equals(conf.get(
+        NODELABEL_CONFIGURATION_TYPE, DEFAULT_NODELABEL_CONFIGURATION_TYPE));
+  }
+
   public YarnConfiguration() {
     super();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/db1b674b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
index 7493169..f2ff0f6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
@@ -97,6 +97,8 @@ public class CommonNodeLabelsManager extends AbstractService {
   protected NodeLabelsStore store;
   private boolean nodeLabelsEnabled = false;
 
+  private boolean isDistributedNodeLabelConfiguration = false;
+
   /**
    * A <code>Host</code> can have multiple <code>Node</code>s 
    */
@@ -213,6 +215,10 @@ public class CommonNodeLabelsManager extends AbstractService {
     nodeLabelsEnabled =
         conf.getBoolean(YarnConfiguration.NODE_LABELS_ENABLED,
             YarnConfiguration.DEFAULT_NODE_LABELS_ENABLED);
+
+    isDistributedNodeLabelConfiguration  =
+        YarnConfiguration.isDistributedNodeLabelConfiguration(conf);
+
     if (nodeLabelsEnabled) {
       initNodeLabelStore(conf);
     }
@@ -223,7 +229,7 @@ public class CommonNodeLabelsManager extends AbstractService {
   protected void initNodeLabelStore(Configuration conf) throws Exception {
     this.store = new FileSystemNodeLabelsStore(this);
     this.store.init(conf);
-    this.store.recover();
+    this.store.recover(isDistributedNodeLabelConfiguration);
   }
 
   // for UT purpose
@@ -613,7 +619,10 @@ public class CommonNodeLabelsManager extends AbstractService {
       }
     }
     
-    if (null != dispatcher) {
+    if (null != dispatcher && !isDistributedNodeLabelConfiguration) {
+      // In case of DistributedNodeLabelConfiguration, no need to save the the
+      // NodeLabels Mapping to the back-end store, as on RM restart/failover
+      // NodeLabels are collected from NM through Register/Heartbeat again
       dispatcher.getEventHandler().handle(
           new UpdateNodeToLabelsMappingsEvent(newNMToLabels));
     }
@@ -799,8 +808,10 @@ public class CommonNodeLabelsManager extends AbstractService {
       readLock.lock();
       List<NodeLabel> nodeLabels = new ArrayList<>();
       for (RMNodeLabel label : labelCollections.values()) {
-        nodeLabels.add(NodeLabel.newInstance(label.getLabelName(),
-            label.getIsExclusive()));
+        if (!label.getLabelName().equals(NO_LABEL)) {
+          nodeLabels.add(NodeLabel.newInstance(label.getLabelName(),
+              label.getIsExclusive()));
+        }
       }
       return nodeLabels;
     } finally {
@@ -824,7 +835,6 @@ public class CommonNodeLabelsManager extends AbstractService {
       readLock.unlock();
     }
   }
-  
 
   private void checkAndThrowLabelName(String label) throws IOException {
     if (label == null || label.isEmpty() || label.length() > MAX_LABEL_LENGTH) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/db1b674b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/FileSystemNodeLabelsStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/FileSystemNodeLabelsStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/FileSystemNodeLabelsStore.java
index ea185f2..f26e204 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/FileSystemNodeLabelsStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/FileSystemNodeLabelsStore.java
@@ -154,8 +154,12 @@ public class FileSystemNodeLabelsStore extends NodeLabelsStore {
     ensureCloseEditlogFile();
   }
 
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.yarn.nodelabels.NodeLabelsStore#recover(boolean)
+   */
   @Override
-  public void recover() throws YarnException, IOException {
+  public void recover(boolean ignoreNodeToLabelsMappings) throws YarnException,
+      IOException {
     /*
      * Steps of recover
      * 1) Read from last mirror (from mirror or mirror.old)
@@ -222,7 +226,15 @@ public class FileSystemNodeLabelsStore extends NodeLabelsStore {
                 new ReplaceLabelsOnNodeRequestPBImpl(
                     ReplaceLabelsOnNodeRequestProto.parseDelimitedFrom(is))
                     .getNodeToLabels();
-            mgr.replaceLabelsOnNode(map);
+            if (!ignoreNodeToLabelsMappings) {
+              /*
+               * In case of Distributed NodeLabels setup,
+               * ignoreNodeToLabelsMappings will be set to true and recover will
+               * be invoked. As RM will collect the node labels from NM through
+               * registration/HB
+               */
+              mgr.replaceLabelsOnNode(map);
+            }
             break;
           }
           }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/db1b674b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelsStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelsStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelsStore.java
index 47b7370..46b94fd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelsStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelsStore.java
@@ -56,9 +56,18 @@ public abstract class NodeLabelsStore implements Closeable {
       throws IOException;
 
   /**
-   * Recover labels and node to labels mappings from store
+   * Recover labels and node to labels mappings from store, but if
+   * ignoreNodeToLabelsMappings is true then node to labels mappings should not
+   * be recovered. In case of Distributed NodeLabels setup
+   * ignoreNodeToLabelsMappings will be set to true and recover will be invoked
+   * as RM will collect the node labels from NM through registration/HB
+   *
+   * @param ignoreNodeToLabelsMappings
+   * @throws IOException
+   * @throws YarnException
    */
-  public abstract void recover() throws IOException, YarnException;
+  public abstract void recover(boolean ignoreNodeToLabelsMappings)
+      throws IOException, YarnException;
   
   public void init(Configuration conf) throws Exception {}
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/db1b674b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/DummyCommonNodeLabelsManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/DummyCommonNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/DummyCommonNodeLabelsManager.java
index 48d6dc8..fce663a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/DummyCommonNodeLabelsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/DummyCommonNodeLabelsManager.java
@@ -39,7 +39,8 @@ public class DummyCommonNodeLabelsManager extends CommonNodeLabelsManager {
     this.store = new NodeLabelsStore(this) {
 
       @Override
-      public void recover() throws IOException {
+      public void recover(boolean ignoreNodeToLabelsMappings)
+          throws IOException {
       }
 
       @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/db1b674b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestCommonNodeLabelsManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestCommonNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestCommonNodeLabelsManager.java
index beb2cf8..09838b4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestCommonNodeLabelsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestCommonNodeLabelsManager.java
@@ -554,4 +554,29 @@ public class TestCommonNodeLabelsManager extends NodeLabelTestBase {
       Assert.assertTrue(expectedAddedLabelNames.contains(label.getName()));
     }
   }
+
+  @Test(timeout = 5000)
+  public void testReplaceLabelsOnNodeInDistributedMode() throws Exception {
+    //create new DummyCommonNodeLabelsManager than the one got from @before
+    mgr.stop();
+    mgr = new DummyCommonNodeLabelsManager();
+    Configuration conf = new YarnConfiguration();
+    conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true);
+    conf.set(YarnConfiguration.NODELABEL_CONFIGURATION_TYPE,
+        YarnConfiguration.DISTRIBUTED_NODELABEL_CONFIGURATION_TYPE);
+
+    mgr.init(conf);
+    mgr.start();
+
+    mgr.addToCluserNodeLabelsWithDefaultExclusivity(toSet("p1", "p2", "p3"));
+    mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("n1"), toSet("p1")));
+    Set<String> labelsByNode = mgr.getLabelsByNode(toNodeId("n1"));
+
+    Assert.assertNull(
+        "Labels are not expected to be written to the NodeLabelStore",
+        mgr.lastNodeToLabels);
+    Assert.assertNotNull("Updated labels should be available from the Mgr",
+        labelsByNode);
+    Assert.assertTrue(labelsByNode.contains("p1"));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/db1b674b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestFileSystemNodeLabelsStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestFileSystemNodeLabelsStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestFileSystemNodeLabelsStore.java
index f070c20..fb60cd6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestFileSystemNodeLabelsStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestFileSystemNodeLabelsStore.java
@@ -146,6 +146,40 @@ public class TestFileSystemNodeLabelsStore extends NodeLabelTestBase {
 
   @SuppressWarnings({ "unchecked", "rawtypes" })
   @Test(timeout = 10000)
+  public void testRecoverWithDistributedNodeLabels() throws Exception {
+    mgr.addToCluserNodeLabelsWithDefaultExclusivity(toSet("p1", "p2", "p3"));
+    mgr.addToCluserNodeLabelsWithDefaultExclusivity(toSet("p4"));
+    mgr.addToCluserNodeLabelsWithDefaultExclusivity(toSet("p5", "p6"));
+    mgr.replaceLabelsOnNode((Map) ImmutableMap.of(toNodeId("n1"), toSet("p1"),
+        toNodeId("n2"), toSet("p2")));
+    mgr.replaceLabelsOnNode((Map) ImmutableMap.of(toNodeId("n3"), toSet("p3"),
+        toNodeId("n4"), toSet("p4"), toNodeId("n5"), toSet("p5"),
+        toNodeId("n6"), toSet("p6"), toNodeId("n7"), toSet("p6")));
+
+    mgr.removeFromClusterNodeLabels(toSet("p1"));
+    mgr.removeFromClusterNodeLabels(Arrays.asList("p3", "p5"));
+    mgr.stop();
+
+    mgr = new MockNodeLabelManager();
+    Configuration cf = new Configuration(conf);
+    cf.set(YarnConfiguration.NODELABEL_CONFIGURATION_TYPE,
+        YarnConfiguration.DISTRIBUTED_NODELABEL_CONFIGURATION_TYPE);
+    mgr.init(cf);
+
+    // check variables
+    Assert.assertEquals(3, mgr.getClusterNodeLabels().size());
+    Assert.assertTrue(mgr.getClusterNodeLabelNames().containsAll(
+        Arrays.asList("p2", "p4", "p6")));
+
+    Assert.assertTrue("During recovery in distributed node-labels setup, "
+        + "node to labels mapping should not be recovered ", mgr
+        .getNodeLabels().size() == 0);
+
+    mgr.stop();
+  }
+
+  @SuppressWarnings({ "unchecked", "rawtypes" })
+  @Test(timeout = 10000)
   public void testEditlogRecover() throws Exception {
     mgr.addToCluserNodeLabelsWithDefaultExclusivity(toSet("p1", "p2", "p3"));
     mgr.addToCluserNodeLabelsWithDefaultExclusivity(toSet("p4"));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/db1b674b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
index c921326..0ad90c0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
@@ -112,6 +112,9 @@ public class AdminService extends CompositeService implements
   private final RecordFactory recordFactory = 
     RecordFactoryProvider.getRecordFactory(null);
 
+  @VisibleForTesting
+  boolean isDistributedNodeLabelConfiguration = false;
+
   public AdminService(ResourceManager rm, RMContext rmContext) {
     super(AdminService.class.getName());
     this.rm = rm;
@@ -141,6 +144,10 @@ public class AdminService extends CompositeService implements
         YarnConfiguration.DEFAULT_YARN_ADMIN_ACL)), UserGroupInformation
         .getCurrentUser());
     rmId = conf.get(YarnConfiguration.RM_HA_ID);
+
+    isDistributedNodeLabelConfiguration =
+        YarnConfiguration.isDistributedNodeLabelConfiguration(conf);
+
     super.serviceInit(conf);
   }
 
@@ -637,32 +644,35 @@ public class AdminService extends CompositeService implements
   @Override
   public RemoveFromClusterNodeLabelsResponse removeFromClusterNodeLabels(
       RemoveFromClusterNodeLabelsRequest request) throws YarnException, IOException {
-    String argName = "removeFromClusterNodeLabels";
+    String operation = "removeFromClusterNodeLabels";
     final String msg = "remove labels.";
-    UserGroupInformation user = checkAcls(argName);
 
-    checkRMStatus(user.getShortUserName(), argName, msg);
+    UserGroupInformation user = checkAcls(operation);
+
+    checkRMStatus(user.getShortUserName(), operation, msg);
 
     RemoveFromClusterNodeLabelsResponse response =
         recordFactory.newRecordInstance(RemoveFromClusterNodeLabelsResponse.class);
     try {
       rmContext.getNodeLabelManager().removeFromClusterNodeLabels(request.getNodeLabels());
       RMAuditLogger
-          .logSuccess(user.getShortUserName(), argName, "AdminService");
+          .logSuccess(user.getShortUserName(), operation, "AdminService");
       return response;
     } catch (IOException ioe) {
-      throw logAndWrapException(ioe, user.getShortUserName(), argName, msg);
+      throw logAndWrapException(ioe, user.getShortUserName(), operation, msg);
     }
   }
 
   @Override
   public ReplaceLabelsOnNodeResponse replaceLabelsOnNode(
       ReplaceLabelsOnNodeRequest request) throws YarnException, IOException {
-    String argName = "replaceLabelsOnNode";
+    String operation = "replaceLabelsOnNode";
     final String msg = "set node to labels.";
-    UserGroupInformation user = checkAcls(argName);
 
-    checkRMStatus(user.getShortUserName(), argName, msg);
+    checkAndThrowIfDistributedNodeLabelConfEnabled(operation);
+    UserGroupInformation user = checkAcls(operation);
+
+    checkRMStatus(user.getShortUserName(), operation, msg);
 
     ReplaceLabelsOnNodeResponse response =
         recordFactory.newRecordInstance(ReplaceLabelsOnNodeResponse.class);
@@ -670,30 +680,41 @@ public class AdminService extends CompositeService implements
       rmContext.getNodeLabelManager().replaceLabelsOnNode(
           request.getNodeToLabels());
       RMAuditLogger
-          .logSuccess(user.getShortUserName(), argName, "AdminService");
+          .logSuccess(user.getShortUserName(), operation, "AdminService");
       return response;
     } catch (IOException ioe) {
-      throw logAndWrapException(ioe, user.getShortUserName(), argName, msg);
+      throw logAndWrapException(ioe, user.getShortUserName(), operation, msg);
     }
   }
 
-  private void checkRMStatus(String user, String argName, String msg)
+  private void checkRMStatus(String user, String operation, String msg)
       throws StandbyException {
     if (!isRMActive()) {
-      RMAuditLogger.logFailure(user, argName, "", 
+      RMAuditLogger.logFailure(user, operation, "",
           "AdminService", "ResourceManager is not active. Can not " + msg);
       throwStandbyException();
     }
   }
 
   private YarnException logAndWrapException(Exception exception, String user,
-      String argName, String msg) throws YarnException {
+      String operation, String msg) throws YarnException {
     LOG.warn("Exception " + msg, exception);
-    RMAuditLogger.logFailure(user, argName, "", 
+    RMAuditLogger.logFailure(user, operation, "",
         "AdminService", "Exception " + msg);
     return RPCUtil.getRemoteException(exception);
   }
 
+  private void checkAndThrowIfDistributedNodeLabelConfEnabled(String operation)
+      throws YarnException {
+    if (isDistributedNodeLabelConfiguration) {
+      String msg =
+          String.format("Error when invoke method=%s because of "
+              + "distributed node label configuration enabled.", operation);
+      LOG.error(msg);
+      throw RPCUtil.getRemoteException(new IOException(msg));
+    }
+  }
+
   @Override
   public CheckForDecommissioningNodesResponse checkForDecommissioningNodes(
       CheckForDecommissioningNodesRequest checkForDecommissioningNodesRequest)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/db1b674b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
index 5e2dc7e..16b6a89 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
@@ -104,7 +104,7 @@ public class ResourceTrackerService extends AbstractService implements
   private int minAllocMb;
   private int minAllocVcores;
 
-  private boolean isDistributesNodeLabelsConf;
+  private boolean isDistributedNodeLabelsConf;
 
   static {
     resync.setNodeAction(NodeAction.RESYNC);
@@ -155,13 +155,8 @@ public class ResourceTrackerService extends AbstractService implements
         YarnConfiguration.RM_NODEMANAGER_MINIMUM_VERSION,
         YarnConfiguration.DEFAULT_RM_NODEMANAGER_MINIMUM_VERSION);
 
-    String nodeLabelConfigurationType =
-        conf.get(YarnConfiguration.NODELABEL_CONFIGURATION_TYPE,
-            YarnConfiguration.DEFAULT_NODELABEL_CONFIGURATION_TYPE);
-
-    isDistributesNodeLabelsConf =
-        YarnConfiguration.DISTRIBUTED_NODELABEL_CONFIGURATION_TYPE
-            .equals(nodeLabelConfigurationType);
+    isDistributedNodeLabelsConf =
+        YarnConfiguration.isDistributedNodeLabelConfiguration(conf);
 
     super.serviceInit(conf);
   }
@@ -352,7 +347,7 @@ public class ResourceTrackerService extends AbstractService implements
 
     // Update node's labels to RM's NodeLabelManager.
     Set<String> nodeLabels = request.getNodeLabels();
-    if (isDistributesNodeLabelsConf && nodeLabels != null) {
+    if (isDistributedNodeLabelsConf && nodeLabels != null) {
       try {
         updateNodeLabelsFromNMReport(nodeLabels, nodeId);
         response.setAreNodeLabelsAcceptedByRM(true);
@@ -470,7 +465,7 @@ public class ResourceTrackerService extends AbstractService implements
     this.rmContext.getDispatcher().getEventHandler().handle(nodeStatusEvent);
 
     // 5. Update node's labels to RM's NodeLabelManager.
-    if (isDistributesNodeLabelsConf && request.getNodeLabels() != null) {
+    if (isDistributedNodeLabelsConf && request.getNodeLabels() != null) {
       try {
         updateNodeLabelsFromNMReport(request.getNodeLabels(), nodeId);
         nodeHeartBeatResponse.setAreNodeLabelsAcceptedByRM(true);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/db1b674b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
index 6cd6d56..9aea62d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
@@ -149,6 +149,7 @@ import org.apache.hadoop.yarn.webapp.BadRequestException;
 import org.apache.hadoop.yarn.webapp.NotFoundException;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.inject.Inject;
 import com.google.inject.Singleton;
 
@@ -165,6 +166,9 @@ public class RMWebServices {
   private final Configuration conf;
   private @Context HttpServletResponse response;
 
+  @VisibleForTesting
+  boolean isDistributedNodeLabelConfiguration = false;
+
   public final static String DELEGATION_TOKEN_HEADER =
       "Hadoop-YARN-RM-Delegation-Token";
 
@@ -172,6 +176,19 @@ public class RMWebServices {
   public RMWebServices(final ResourceManager rm, Configuration conf) {
     this.rm = rm;
     this.conf = conf;
+    isDistributedNodeLabelConfiguration =
+        YarnConfiguration.isDistributedNodeLabelConfiguration(conf);
+  }
+
+  private void checkAndThrowIfDistributedNodeLabelConfEnabled(String operation)
+      throws IOException {
+    if (isDistributedNodeLabelConfiguration) {
+      String msg =
+          String.format("Error when invoke method=%s because of "
+              + "distributed node label configuration enabled.", operation);
+      LOG.error(msg);
+      throw new IOException(msg);
+    }
   }
 
   RMWebServices(ResourceManager rm, Configuration conf,
@@ -816,38 +833,64 @@ public class RMWebServices {
   @POST
   @Path("/replace-node-to-labels")
   @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
-  public Response replaceLabelsOnNodes(
-    final NodeToLabelsInfo newNodeToLabels,
-    @Context HttpServletRequest hsr) 
-    throws IOException {
+  public Response replaceLabelsOnNodes(final NodeToLabelsInfo newNodeToLabels,
+      @Context HttpServletRequest hsr) throws IOException {
+    Map<NodeId, Set<String>> nodeIdToLabels =
+        new HashMap<NodeId, Set<String>>();
+
+    for (Map.Entry<String, NodeLabelsInfo> nitle : newNodeToLabels
+        .getNodeToLabels().entrySet()) {
+      nodeIdToLabels.put(
+          ConverterUtils.toNodeIdWithDefaultPort(nitle.getKey()),
+          new HashSet<String>(nitle.getValue().getNodeLabels()));
+    }
+
+    return replaceLabelsOnNode(nodeIdToLabels, hsr, "/replace-node-to-labels");
+  }
+
+  @POST
+  @Path("/nodes/{nodeId}/replace-labels")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public Response replaceLabelsOnNode(NodeLabelsInfo newNodeLabelsInfo,
+      @Context HttpServletRequest hsr, @PathParam("nodeId") String nodeId)
+      throws Exception {
+    NodeId nid = ConverterUtils.toNodeIdWithDefaultPort(nodeId);
+    Map<NodeId, Set<String>> newLabelsForNode =
+        new HashMap<NodeId, Set<String>>();
+    newLabelsForNode.put(nid,
+        new HashSet<String>(newNodeLabelsInfo.getNodeLabels()));
+
+    return replaceLabelsOnNode(newLabelsForNode, hsr, "/nodes/nodeid/replace-labels");
+  }
+
+  private Response replaceLabelsOnNode(
+      Map<NodeId, Set<String>> newLabelsForNode, HttpServletRequest hsr,
+      String operation) throws IOException {
     init();
-    
+
+    checkAndThrowIfDistributedNodeLabelConfEnabled("replaceLabelsOnNode");
+
     UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true);
     if (callerUGI == null) {
-      String msg = "Unable to obtain user name, user not authenticated for"
-        + " post to .../replace-node-to-labels";
+      String msg =
+          "Unable to obtain user name, user not authenticated for"
+              + " post to ..." + operation;
       throw new AuthorizationException(msg);
     }
+
     if (!rm.getRMContext().getNodeLabelManager().checkAccess(callerUGI)) {
-      String msg = "User " + callerUGI.getShortUserName() + " not authorized"
-        + " for post to .../replace-node-to-labels ";
+      String msg =
+          "User " + callerUGI.getShortUserName() + " not authorized"
+              + " for post to ..." + operation;
       throw new AuthorizationException(msg);
     }
-    
-    Map<NodeId, Set<String>> nodeIdToLabels = 
-      new HashMap<NodeId, Set<String>>();
 
-    for (Map.Entry<String, NodeLabelsInfo> nitle : 
-      newNodeToLabels.getNodeToLabels().entrySet()) {
-     nodeIdToLabels.put(ConverterUtils.toNodeIdWithDefaultPort(nitle.getKey()),
-       new HashSet<String>(nitle.getValue().getNodeLabels()));
-    }
-    
-    rm.getRMContext().getNodeLabelManager().replaceLabelsOnNode(nodeIdToLabels);
+    rm.getRMContext().getNodeLabelManager()
+        .replaceLabelsOnNode(newLabelsForNode);
 
     return Response.status(Status.OK).build();
   }
-  
+
   @GET
   @Path("/get-node-labels")
   @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
@@ -897,7 +940,7 @@ public class RMWebServices {
       @Context HttpServletRequest hsr)
       throws Exception {
     init();
-    
+
     UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true);
     if (callerUGI == null) {
       String msg = "Unable to obtain user name, user not authenticated for"
@@ -931,40 +974,6 @@ public class RMWebServices {
       rm.getRMContext().getNodeLabelManager().getLabelsOnNode(nid));
 
   }
-  
-  @POST
-  @Path("/nodes/{nodeId}/replace-labels")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
-  public Response replaceLabelsOnNode(NodeLabelsInfo newNodeLabelsInfo,
-      @Context HttpServletRequest hsr, @PathParam("nodeId") String nodeId)
-      throws Exception {
-    init();
-    
-    UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true);
-    if (callerUGI == null) {
-      String msg = "Unable to obtain user name, user not authenticated for"
-        + " post to .../nodes/nodeid/replace-labels";
-      throw new AuthorizationException(msg);
-    }
-
-    if (!rm.getRMContext().getNodeLabelManager().checkAccess(callerUGI)) {
-      String msg = "User " + callerUGI.getShortUserName() + " not authorized"
-        + " for post to .../nodes/nodeid/replace-labels";
-      throw new AuthorizationException(msg);
-    }
-    
-    NodeId nid = ConverterUtils.toNodeIdWithDefaultPort(nodeId);
-    
-    Map<NodeId, Set<String>> newLabelsForNode = new HashMap<NodeId,
-      Set<String>>();
-    
-    newLabelsForNode.put(nid, new HashSet<String>(newNodeLabelsInfo.getNodeLabels()));
-    
-    rm.getRMContext().getNodeLabelManager().replaceLabelsOnNode(newLabelsForNode);
-    
-    return Response.status(Status.OK).build();
-
-  }
 
   protected Response killApp(RMApp app, UserGroupInformation callerUGI,
       HttpServletRequest hsr) throws IOException, InterruptedException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/db1b674b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
index da04c9e..fe0b8a8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
 import java.io.DataOutputStream;
@@ -44,6 +45,7 @@ import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
 import org.apache.hadoop.yarn.api.records.DecommissionType;
+import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.conf.HAUtil;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -53,6 +55,9 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshQueuesRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshServiceAclsRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshSuperUserGroupsConfigurationRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshUserToGroupsMappingsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RemoveFromClusterNodeLabelsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.ReplaceLabelsOnNodeRequest;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.junit.After;
@@ -60,6 +65,8 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
 
 public class TestRMAdminService {
 
@@ -754,6 +761,67 @@ public class TestRMAdminService {
     }
   }
 
+  @Test
+  public void testModifyLabelsOnNodesWithDistributedConfigurationDisabled()
+      throws IOException, YarnException {
+    // create RM and set it's ACTIVE
+    MockRM rm = new MockRM();
+    ((RMContextImpl) rm.getRMContext())
+        .setHAServiceState(HAServiceState.ACTIVE);
+    RMNodeLabelsManager labelMgr = rm.rmContext.getNodeLabelManager();
+
+    // by default, distributed configuration for node label is disabled, this
+    // should pass
+    labelMgr.addToCluserNodeLabelsWithDefaultExclusivity(ImmutableSet.of("x", "y"));
+    rm.adminService.replaceLabelsOnNode(ReplaceLabelsOnNodeRequest
+        .newInstance(ImmutableMap.of(NodeId.newInstance("host", 0),
+            (Set<String>) ImmutableSet.of("x"))));
+    rm.close();
+  }
+
+  @Test(expected = YarnException.class)
+  public void testModifyLabelsOnNodesWithDistributedConfigurationEnabled()
+      throws IOException, YarnException {
+    // create RM and set it's ACTIVE, and set distributed node label
+    // configuration to true
+    MockRM rm = new MockRM();
+    rm.adminService.isDistributedNodeLabelConfiguration = true;
+
+    ((RMContextImpl) rm.getRMContext())
+        .setHAServiceState(HAServiceState.ACTIVE);
+    RMNodeLabelsManager labelMgr = rm.rmContext.getNodeLabelManager();
+
+    // by default, distributed configuration for node label is disabled, this
+    // should pass
+    labelMgr.addToCluserNodeLabelsWithDefaultExclusivity(ImmutableSet.of("x", "y"));
+    rm.adminService.replaceLabelsOnNode(ReplaceLabelsOnNodeRequest
+        .newInstance(ImmutableMap.of(NodeId.newInstance("host", 0),
+            (Set<String>) ImmutableSet.of("x"))));
+    rm.close();
+  }
+
+  @Test
+  public void testRemoveClusterNodeLabelsWithDistributedConfigurationEnabled()
+      throws IOException, YarnException {
+    // create RM and set it's ACTIVE
+    MockRM rm = new MockRM();
+    ((RMContextImpl) rm.getRMContext())
+        .setHAServiceState(HAServiceState.ACTIVE);
+    RMNodeLabelsManager labelMgr = rm.rmContext.getNodeLabelManager();
+    rm.adminService.isDistributedNodeLabelConfiguration = true;
+
+    // by default, distributed configuration for node label is disabled, this
+    // should pass
+    labelMgr.addToCluserNodeLabelsWithDefaultExclusivity(ImmutableSet.of("x", "y"));
+    rm.adminService
+        .removeFromClusterNodeLabels(RemoveFromClusterNodeLabelsRequest
+            .newInstance((Set<String>) ImmutableSet.of("x")));
+
+    Set<String> clusterNodeLabels = labelMgr.getClusterNodeLabelNames();
+    assertEquals(1,clusterNodeLabels.size());
+    rm.close();
+  }
+
   private String writeConfigurationXML(Configuration conf, String confXMLName)
       throws IOException {
     DataOutputStream output = null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/db1b674b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NullRMNodeLabelsManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NullRMNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NullRMNodeLabelsManager.java
index 9548029..2e21d26 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NullRMNodeLabelsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NullRMNodeLabelsManager.java
@@ -40,7 +40,8 @@ public class NullRMNodeLabelsManager extends RMNodeLabelsManager {
     this.store = new NodeLabelsStore(this) {
 
       @Override
-      public void recover() throws IOException {
+      public void recover(boolean ignoreNodeToLabelsMappings)
+          throws IOException {
         // do nothing
       }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/db1b674b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java
index 298246c..e4614f8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.ClusterMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
@@ -623,6 +624,7 @@ public class TestRMWebServices extends JerseyTestBase {
         null, null, null, null, null);
     when(mockRM.getRMContext()).thenReturn(rmContext);
     when(mockRM.getClientRMService()).thenReturn(mockClientSvc);
+    rmContext.setNodeLabelManager(mock(RMNodeLabelsManager.class));
 
     RMWebServices webSvc = new RMWebServices(mockRM, new Configuration(),
         mock(HttpServletResponse.class));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/db1b674b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodeLabels.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodeLabels.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodeLabels.java
index 40c54a3..2d5518d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodeLabels.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodeLabels.java
@@ -19,10 +19,10 @@
 package org.apache.hadoop.yarn.server.resourcemanager.webapp;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
-import java.io.StringReader;
 import java.io.StringWriter;
 
 import javax.ws.rs.core.MediaType;
@@ -51,7 +51,6 @@ import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.WebResource;
 import com.sun.jersey.api.json.JSONJAXBContext;
 import com.sun.jersey.api.json.JSONMarshaller;
-import com.sun.jersey.api.json.JSONUnmarshaller;
 import com.sun.jersey.core.util.MultivaluedMapImpl;
 import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
 import com.sun.jersey.test.framework.WebAppDescriptor;
@@ -66,13 +65,13 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
 
   private String userName;
   private String notUserName;
+  private RMWebServices rmWebService;
 
   private Injector injector = Guice.createInjector(new ServletModule() {
+
     @Override
     protected void configureServlets() {
       bind(JAXBContextResolver.class);
-      bind(RMWebServices.class);
-      bind(GenericExceptionHandler.class);
       try {
         userName = UserGroupInformation.getCurrentUser().getShortUserName();
       } catch (IOException ioe) {
@@ -83,6 +82,9 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
       conf = new YarnConfiguration();
       conf.set(YarnConfiguration.YARN_ADMIN_ACL, userName);
       rm = new MockRM(conf);
+      rmWebService = new RMWebServices(rm,conf);
+      bind(RMWebServices.class).toInstance(rmWebService);
+      bind(GenericExceptionHandler.class);
       bind(ResourceManager.class).toInstance(rm);
       filter("/*").through(
           TestRMWebServicesAppsModification.TestRMCustomAuthFilter.class);
@@ -113,7 +115,6 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
     ClientResponse response;
     JSONObject json;
     JSONArray jarr;
-    String responseString;
 
     // Add a label
     response =
@@ -386,6 +387,93 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     String res = response.getEntity(String.class);
     assertTrue(res.equals("null"));
+
+    // Following test cases are to test replace when distributed node label
+    // configuration is on
+    // Reset for testing : add cluster labels
+    response =
+        r.path("ws")
+            .path("v1")
+            .path("cluster")
+            .path("add-node-labels")
+            .queryParam("user.name", userName)
+            .accept(MediaType.APPLICATION_JSON)
+            .entity("{\"nodeLabels\":[\"x\",\"y\"]}",
+                MediaType.APPLICATION_JSON).post(ClientResponse.class);
+    // Reset for testing : Add labels to a node
+    response =
+        r.path("ws").path("v1").path("cluster").path("nodes").path("nid:0")
+            .path("replace-labels").queryParam("user.name", userName)
+            .accept(MediaType.APPLICATION_JSON)
+            .entity("{\"nodeLabels\": [\"y\"]}", MediaType.APPLICATION_JSON)
+            .post(ClientResponse.class);
+    LOG.info("posted node nodelabel");
+
+    //setting rmWebService for Distributed NodeLabel Configuration
+    rmWebService.isDistributedNodeLabelConfiguration = true;
+
+    // Case1 : Replace labels using node-to-labels
+    ntli = new NodeToLabelsInfo();
+    nli = new NodeLabelsInfo();
+    nli.getNodeLabels().add("x");
+    ntli.getNodeToLabels().put("nid:0", nli);
+    response =
+        r.path("ws")
+            .path("v1")
+            .path("cluster")
+            .path("replace-node-to-labels")
+            .queryParam("user.name", userName)
+            .accept(MediaType.APPLICATION_JSON)
+            .entity(toJson(ntli, NodeToLabelsInfo.class),
+                MediaType.APPLICATION_JSON).post(ClientResponse.class);
+
+    // Verify, using node-to-labels that previous operation has failed
+    response =
+        r.path("ws").path("v1").path("cluster").path("get-node-to-labels")
+            .queryParam("user.name", userName)
+            .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
+    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    ntli = response.getEntity(NodeToLabelsInfo.class);
+    nli = ntli.getNodeToLabels().get("nid:0");
+    assertEquals(1, nli.getNodeLabels().size());
+    assertFalse(nli.getNodeLabels().contains("x"));
+
+    // Case2 : failure to Replace labels using replace-labels
+    response =
+        r.path("ws").path("v1").path("cluster").path("nodes").path("nid:0")
+            .path("replace-labels").queryParam("user.name", userName)
+            .accept(MediaType.APPLICATION_JSON)
+            .entity("{\"nodeLabels\": [\"x\"]}", MediaType.APPLICATION_JSON)
+            .post(ClientResponse.class);
+    LOG.info("posted node nodelabel");
+
+    // Verify, using node-to-labels that previous operation has failed
+    response =
+        r.path("ws").path("v1").path("cluster").path("get-node-to-labels")
+            .queryParam("user.name", userName)
+            .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
+    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    ntli = response.getEntity(NodeToLabelsInfo.class);
+    nli = ntli.getNodeToLabels().get("nid:0");
+    assertEquals(1, nli.getNodeLabels().size());
+    assertFalse(nli.getNodeLabels().contains("x"));
+
+    //  Case3 : Remove cluster label should be successfull
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("remove-node-labels")
+            .queryParam("user.name", userName)
+            .accept(MediaType.APPLICATION_JSON)
+            .entity("{\"nodeLabels\":\"x\"}", MediaType.APPLICATION_JSON)
+            .post(ClientResponse.class);
+    // Verify
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("get-node-labels").queryParam("user.name", userName)
+            .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
+    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    json = response.getEntity(JSONObject.class);
+    assertEquals("y", json.getString("nodeLabels"));
   }
 
   @SuppressWarnings("rawtypes")
@@ -396,13 +484,4 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
     jm.marshallToJSON(nsli, sw);
     return sw.toString();
   }
-
-  @SuppressWarnings({ "rawtypes", "unchecked" })
-  private Object fromJson(String json, Class klass) throws Exception {
-    StringReader sr = new StringReader(json);
-    JSONJAXBContext ctx = new JSONJAXBContext(klass);
-    JSONUnmarshaller jm = ctx.createJSONUnmarshaller();
-    return jm.unmarshalFromJSON(sr, klass);
-  }
-
 }