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 vi...@apache.org on 2014/11/09 17:54:03 UTC

[01/30] hadoop git commit: MAPREDUCE-5960. JobSubmitter's check whether job.jar is local is incorrect with no authority in job jar path. Contributed by Gera Shegalov

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-EC 4ce3a132e -> 3a1b3f82d


MAPREDUCE-5960. JobSubmitter's check whether job.jar is local is incorrect with no authority in job jar path. Contributed by Gera Shegalov


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

Branch: refs/heads/HDFS-EC
Commit: 10f9f5101c44be7c675a44ded4aad212627ecdee
Parents: 80d7d18
Author: Jason Lowe <jl...@apache.org>
Authored: Thu Nov 6 15:10:40 2014 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Thu Nov 6 15:10:40 2014 +0000

----------------------------------------------------------------------
 .../java/org/apache/hadoop/fs/FileContext.java  |  3 ++
 hadoop-mapreduce-project/CHANGES.txt            |  3 ++
 .../v2/app/job/impl/TaskAttemptImpl.java        |  8 ++--
 .../apache/hadoop/mapreduce/JobSubmitter.java   |  9 ++---
 .../org/apache/hadoop/mapred/YARNRunner.java    |  5 ++-
 .../apache/hadoop/mapreduce/v2/TestMRJobs.java  | 41 ++++++++++++++++----
 .../apache/hadoop/mapreduce/v2/TestUberAM.java  | 40 +++----------------
 7 files changed, 56 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/10f9f510/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
index 2323650..85f8136 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
@@ -430,6 +430,9 @@ public class FileContext {
       final Configuration aConf) throws UnsupportedFileSystemException {
     UserGroupInformation currentUser = null;
     AbstractFileSystem defaultAfs = null;
+    if (defaultFsUri.getScheme() == null) {
+      return getFileContext(aConf);
+    }
     try {
       currentUser = UserGroupInformation.getCurrentUser();
       defaultAfs = getAbstractFileSystem(currentUser, defaultFsUri, aConf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10f9f510/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index bbe96c2..fd42f82 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -459,6 +459,9 @@ Release 2.6.0 - UNRELEASED
     MAPREDUCE-6048. Fixed TestJavaSerialization failure. (Varun Vasudev via
     jianhe)
 
+    MAPREDUCE-5960. JobSubmitter's check whether job.jar is local is incorrect
+    with no authority in job jar path. (Gera Shegalov via jlowe)
+
 Release 2.5.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10f9f510/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
index 288e18e..dfc6a3f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
@@ -657,9 +657,11 @@ public abstract class TaskAttemptImpl implements
       // //////////// Set up JobJar to be localized properly on the remote NM.
       String jobJar = conf.get(MRJobConfig.JAR);
       if (jobJar != null) {
-        Path remoteJobJar = (new Path(jobJar)).makeQualified(remoteFS
-            .getUri(), remoteFS.getWorkingDirectory());
-        LocalResource rc = createLocalResource(remoteFS, remoteJobJar,
+        final Path jobJarPath = new Path(jobJar);
+        final FileSystem jobJarFs = FileSystem.get(jobJarPath.toUri(), conf);
+        Path remoteJobJar = jobJarPath.makeQualified(jobJarFs.getUri(),
+            jobJarFs.getWorkingDirectory());
+        LocalResource rc = createLocalResource(jobJarFs, remoteJobJar,
             LocalResourceType.PATTERN, LocalResourceVisibility.APPLICATION);
         String pattern = conf.getPattern(JobContext.JAR_UNPACK_PATTERN, 
             JobConf.UNPACK_JAR_PATTERN_DEFAULT).pattern();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10f9f510/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java
index b76a734..ba496ee 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java
@@ -250,11 +250,10 @@ class JobSubmitter {
       }
       Path jobJarPath = new Path(jobJar);
       URI jobJarURI = jobJarPath.toUri();
-      // If the job jar is already in fs, we don't need to copy it from local fs
-      if (jobJarURI.getScheme() == null || jobJarURI.getAuthority() == null
-              || !(jobJarURI.getScheme().equals(jtFs.getUri().getScheme()) 
-                  && jobJarURI.getAuthority().equals(
-                                            jtFs.getUri().getAuthority()))) {
+      // If the job jar is already in a global fs,
+      // we don't need to copy it from local fs
+      if (     jobJarURI.getScheme() == null
+            || jobJarURI.getScheme().equals("file")) {
         copyJar(jobJarPath, JobSubmissionFiles.getJobJar(submitJobDir), 
             replication);
         job.setJar(JobSubmissionFiles.getJobJar(submitJobDir).toString());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10f9f510/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
index a1c4c32..7b2cf53 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
@@ -357,8 +357,9 @@ public class YARNRunner implements ClientProtocol {
             jobConfPath, LocalResourceType.FILE));
     if (jobConf.get(MRJobConfig.JAR) != null) {
       Path jobJarPath = new Path(jobConf.get(MRJobConfig.JAR));
-      LocalResource rc = createApplicationResource(defaultFileContext,
-          jobJarPath, 
+      LocalResource rc = createApplicationResource(
+          FileContext.getFileContext(jobJarPath.toUri(), jobConf),
+          jobJarPath,
           LocalResourceType.PATTERN);
       String pattern = conf.getPattern(JobContext.JAR_UNPACK_PATTERN, 
           JobConf.UNPACK_JAR_PATTERN_DEFAULT).pattern();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10f9f510/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java
index 0044925..2b45049 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java
@@ -40,6 +40,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.FailingMapper;
 import org.apache.hadoop.RandomTextWriterJob;
 import org.apache.hadoop.RandomTextWriterJob.RandomInputFormat;
+import org.apache.hadoop.fs.viewfs.ConfigUtil;
 import org.apache.hadoop.mapreduce.SleepJob;
 import org.apache.hadoop.mapreduce.SleepJob.SleepMapper;
 import org.apache.hadoop.conf.Configuration;
@@ -89,6 +90,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.ApplicationClassLoader;
+import org.apache.hadoop.util.ClassUtil;
 import org.apache.hadoop.util.JarFinder;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -97,6 +99,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.log4j.Level;
+import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -111,6 +114,9 @@ public class TestMRJobs {
   private static final String TEST_IO_SORT_MB = "11";
   private static final String TEST_GROUP_MAX = "200";
 
+  private static final int DEFAULT_REDUCES = 2;
+  protected int numSleepReducers = DEFAULT_REDUCES;
+
   protected static MiniMRYarnCluster mrCluster;
   protected static MiniDFSCluster dfsCluster;
 
@@ -175,10 +181,23 @@ public class TestMRJobs {
     }
   }
 
+  @After
+  public void resetInit() {
+    numSleepReducers = DEFAULT_REDUCES;
+  }
+
+  @Test (timeout = 300000)
+  public void testSleepJob() throws Exception {
+    testSleepJobInternal(false);
+  }
+
   @Test (timeout = 300000)
-  public void testSleepJob() throws IOException, InterruptedException,
-      ClassNotFoundException { 
-    LOG.info("\n\n\nStarting testSleepJob().");
+  public void testSleepJobWithRemoteJar() throws Exception {
+    testSleepJobInternal(true);
+  }
+
+  private void testSleepJobInternal(boolean useRemoteJar) throws Exception {
+    LOG.info("\n\n\nStarting testSleepJob: useRemoteJar=" + useRemoteJar);
 
     if (!(new File(MiniMRYarnCluster.APPJAR)).exists()) {
       LOG.info("MRAppJar " + MiniMRYarnCluster.APPJAR
@@ -192,14 +211,20 @@ public class TestMRJobs {
     
     SleepJob sleepJob = new SleepJob();
     sleepJob.setConf(sleepConf);
-
-    int numReduces = sleepConf.getInt("TestMRJobs.testSleepJob.reduces", 2); // or sleepConf.getConfig().getInt(MRJobConfig.NUM_REDUCES, 2);
    
     // job with 3 maps (10s) and numReduces reduces (5s), 1 "record" each:
-    Job job = sleepJob.createJob(3, numReduces, 10000, 1, 5000, 1);
+    Job job = sleepJob.createJob(3, numSleepReducers, 10000, 1, 5000, 1);
 
     job.addFileToClassPath(APP_JAR); // The AppMaster jar itself.
-    job.setJarByClass(SleepJob.class);
+    if (useRemoteJar) {
+      final Path localJar = new Path(
+          ClassUtil.findContainingJar(SleepJob.class));
+      ConfigUtil.addLink(job.getConfiguration(), "/jobjars",
+          localFs.makeQualified(localJar.getParent()).toUri());
+      job.setJar("viewfs:///jobjars/" + localJar.getName());
+    } else {
+      job.setJarByClass(SleepJob.class);
+    }
     job.setMaxMapAttempts(1); // speed up failures
     job.submit();
     String trackingUrl = job.getTrackingURL();
@@ -381,7 +406,7 @@ public class TestMRJobs {
         .getValue());
     Assert.assertEquals(3, counters.findCounter(JobCounter.TOTAL_LAUNCHED_MAPS)
         .getValue());
-    Assert.assertEquals(2,
+    Assert.assertEquals(numSleepReducers,
         counters.findCounter(JobCounter.TOTAL_LAUNCHED_REDUCES).getValue());
     Assert
         .assertTrue(counters.findCounter(JobCounter.SLOTS_MILLIS_MAPS) != null

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10f9f510/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestUberAM.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestUberAM.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestUberAM.java
index e89a919..e198f99 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestUberAM.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestUberAM.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.mapreduce.v2;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.Arrays;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -40,8 +39,7 @@ import org.junit.Test;
 public class TestUberAM extends TestMRJobs {
 
   private static final Log LOG = LogFactory.getLog(TestUberAM.class);
-  private int numSleepReducers;
-  
+
   @BeforeClass
   public static void setup() throws IOException {
     TestMRJobs.setup();
@@ -54,21 +52,15 @@ public class TestUberAM extends TestMRJobs {
   @Override
   @Test
   public void testSleepJob()
-  throws IOException, InterruptedException, ClassNotFoundException {
+  throws Exception {
     numSleepReducers = 1;
-    if (mrCluster != null) {
-    	mrCluster.getConfig().setInt("TestMRJobs.testSleepJob.reduces", numSleepReducers);
-    }
     super.testSleepJob();
   }
   
   @Test
   public void testSleepJobWithMultipleReducers()
-  throws IOException, InterruptedException, ClassNotFoundException {
+  throws Exception {
     numSleepReducers = 3;
-    if (mrCluster != null) {
-      mrCluster.getConfig().setInt("TestMRJobs.testSleepJob.reduces", numSleepReducers);
-    }
     super.testSleepJob();
   }
   
@@ -76,20 +68,7 @@ public class TestUberAM extends TestMRJobs {
   protected void verifySleepJobCounters(Job job) throws InterruptedException,
       IOException {
     Counters counters = job.getCounters();
-
-    Assert.assertEquals(3, counters.findCounter(JobCounter.OTHER_LOCAL_MAPS)
-        .getValue());
-    Assert.assertEquals(3, counters.findCounter(JobCounter.TOTAL_LAUNCHED_MAPS)
-        .getValue());
-    Assert.assertEquals(numSleepReducers,
-        counters.findCounter(JobCounter.TOTAL_LAUNCHED_REDUCES).getValue());
-    Assert
-        .assertTrue(counters.findCounter(JobCounter.SLOTS_MILLIS_MAPS) != null
-            && counters.findCounter(JobCounter.SLOTS_MILLIS_MAPS).getValue() != 0);
-    Assert
-        .assertTrue(counters.findCounter(JobCounter.SLOTS_MILLIS_MAPS) != null
-            && counters.findCounter(JobCounter.SLOTS_MILLIS_MAPS).getValue() != 0);
-
+    super.verifySleepJobCounters(job);
     Assert.assertEquals(3,
         counters.findCounter(JobCounter.NUM_UBER_SUBMAPS).getValue());
     Assert.assertEquals(numSleepReducers,
@@ -168,16 +147,7 @@ public class TestUberAM extends TestMRJobs {
   protected void verifyFailingMapperCounters(Job job)
       throws InterruptedException, IOException {
     Counters counters = job.getCounters();
-    Assert.assertEquals(2, counters.findCounter(JobCounter.OTHER_LOCAL_MAPS)
-        .getValue());
-    Assert.assertEquals(2, counters.findCounter(JobCounter.TOTAL_LAUNCHED_MAPS)
-        .getValue());
-    Assert.assertEquals(2, counters.findCounter(JobCounter.NUM_FAILED_MAPS)
-        .getValue());
-    Assert
-        .assertTrue(counters.findCounter(JobCounter.SLOTS_MILLIS_MAPS) != null
-            && counters.findCounter(JobCounter.SLOTS_MILLIS_MAPS).getValue() != 0);
-
+    super.verifyFailingMapperCounters(job);
     Assert.assertEquals(2,
         counters.findCounter(JobCounter.TOTAL_LAUNCHED_UBERTASKS).getValue());
     Assert.assertEquals(2, counters.findCounter(JobCounter.NUM_UBER_SUBMAPS)


[07/30] hadoop git commit: YARN-2744. Fixed CapacityScheduler to validate node-labels correctly against queues. Contributed by Wangda Tan.

Posted by vi...@apache.org.
YARN-2744. Fixed CapacityScheduler to validate node-labels correctly against queues. 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/a3839a9f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a3839a9f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a3839a9f

Branch: refs/heads/HDFS-EC
Commit: a3839a9fbfb8eec396b9bf85472d25e0ffc3aab2
Parents: 16b3482
Author: Vinod Kumar Vavilapalli <vi...@apache.org>
Authored: Thu Nov 6 17:28:12 2014 -0800
Committer: Vinod Kumar Vavilapalli <vi...@apache.org>
Committed: Thu Nov 6 17:28:12 2014 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../scheduler/capacity/AbstractCSQueue.java     |   4 +-
 .../scheduler/capacity/TestQueueParsing.java    | 144 ++++++++++++++++++-
 3 files changed, 144 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3839a9f/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 0c5fc4c..f6b39e3 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -882,6 +882,9 @@ Release 2.6.0 - UNRELEASED
     YARN-2812. TestApplicationHistoryServer is likely to fail on less powerful machine.
     (Zhijie Shen via xgong)
 
+    YARN-2744. Fixed CapacityScheduler to validate node-labels correctly against
+    queues. (Wangda Tan via vinodkv)
+
 Release 2.5.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3839a9f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.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/AbstractCSQueue.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/AbstractCSQueue.java
index 7159e4d..c612846 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/AbstractCSQueue.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/AbstractCSQueue.java
@@ -105,9 +105,9 @@ public abstract class AbstractCSQueue implements CSQueue {
     // inherit from parent if labels not set
     if (this.accessibleLabels == null && parent != null) {
       this.accessibleLabels = parent.getAccessibleNodeLabels();
-      SchedulerUtils.checkIfLabelInClusterNodeLabels(labelManager,
-          this.accessibleLabels);
     }
+    SchedulerUtils.checkIfLabelInClusterNodeLabels(labelManager,
+        this.accessibleLabels);
     
     // inherit from parent if labels not set
     if (this.defaultLabelExpression == null && parent != null

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3839a9f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueParsing.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/TestQueueParsing.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/TestQueueParsing.java
index 20a7e53..42db030 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/TestQueueParsing.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/TestQueueParsing.java
@@ -26,9 +26,11 @@ import java.io.IOException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.service.ServiceOperations;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.MemoryRMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
@@ -80,7 +82,7 @@ public class TestQueueParsing {
     Assert.assertEquals(0.7 * 0.5 * 0.45, c12.getAbsoluteCapacity(), DELTA);
     Assert.assertEquals(0.7 * 0.55 * 0.7, 
         c12.getAbsoluteMaximumCapacity(), DELTA);
-    capacityScheduler.stop();
+    ServiceOperations.stopQuietly(capacityScheduler);
   }
   
   private void setupQueueConfiguration(CapacitySchedulerConfiguration conf) {
@@ -164,7 +166,7 @@ public class TestQueueParsing {
     capacityScheduler.init(conf);
     capacityScheduler.start();
     capacityScheduler.reinitialize(conf, null);
-    capacityScheduler.stop();
+    ServiceOperations.stopQuietly(capacityScheduler);
   }
   
   public void testMaxCapacity() throws Exception {
@@ -339,6 +341,27 @@ public class TestQueueParsing {
     conf.setCapacityByLabel(A2, "red", 50);
   }
   
+  private void setupQueueConfigurationWithSingleLevel(
+      CapacitySchedulerConfiguration conf) {
+    // Define top-level queues
+    conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"});
+
+    // Set A configuration
+    final String A = CapacitySchedulerConfiguration.ROOT + ".a";
+    conf.setCapacity(A, 10);
+    conf.setMaximumCapacity(A, 15);
+    conf.setAccessibleNodeLabels(A, ImmutableSet.of("red", "blue"));
+    conf.setCapacityByLabel(A, "red", 90);
+    conf.setCapacityByLabel(A, "blue", 90);
+    
+    // Set B configuraiton
+    final String B = CapacitySchedulerConfiguration.ROOT + ".b";
+    conf.setCapacity(B, 90);
+    conf.setAccessibleNodeLabels(B, ImmutableSet.of("red", "blue"));
+    conf.setCapacityByLabel(B, "red", 10);
+    conf.setCapacityByLabel(B, "blue", 10);
+  }
+  
   @Test
   public void testQueueParsingReinitializeWithLabels() throws IOException {
     CapacitySchedulerConfiguration csConf =
@@ -362,7 +385,7 @@ public class TestQueueParsing {
     conf = new YarnConfiguration(csConf);
     capacityScheduler.reinitialize(conf, rmContext);
     checkQueueLabels(capacityScheduler);
-    capacityScheduler.stop();
+    ServiceOperations.stopQuietly(capacityScheduler);
   }
   
   private void checkQueueLabels(CapacityScheduler capacityScheduler) {
@@ -429,7 +452,7 @@ public class TestQueueParsing {
     capacityScheduler.init(csConf);
     capacityScheduler.start();
     checkQueueLabels(capacityScheduler);
-    capacityScheduler.stop();
+    ServiceOperations.stopQuietly(capacityScheduler);
   }
   
   @Test
@@ -451,6 +474,117 @@ public class TestQueueParsing {
     capacityScheduler.init(csConf);
     capacityScheduler.start();
     checkQueueLabelsInheritConfig(capacityScheduler);
-    capacityScheduler.stop();
+    ServiceOperations.stopQuietly(capacityScheduler);
+  }
+  
+  @Test(expected = Exception.class)
+  public void testQueueParsingWhenLabelsNotExistedInNodeLabelManager()
+      throws IOException {
+    YarnConfiguration conf = new YarnConfiguration();
+    CapacitySchedulerConfiguration csConf =
+        new CapacitySchedulerConfiguration(conf);
+    setupQueueConfigurationWithLabels(csConf);
+
+    CapacityScheduler capacityScheduler = new CapacityScheduler();
+    RMContextImpl rmContext =
+        new RMContextImpl(null, null, null, null, null, null,
+            new RMContainerTokenSecretManager(csConf),
+            new NMTokenSecretManagerInRM(csConf),
+            new ClientToAMTokenSecretManagerInRM(), null);
+    
+    RMNodeLabelsManager nodeLabelsManager = new MemoryRMNodeLabelsManager();
+    nodeLabelsManager.init(conf);
+    nodeLabelsManager.start();
+    
+    rmContext.setNodeLabelManager(nodeLabelsManager);
+    capacityScheduler.setConf(csConf);
+    capacityScheduler.setRMContext(rmContext);
+    capacityScheduler.init(csConf);
+    capacityScheduler.start();
+    ServiceOperations.stopQuietly(capacityScheduler);
+    ServiceOperations.stopQuietly(nodeLabelsManager);
+  }
+  
+  @Test(expected = Exception.class)
+  public void testQueueParsingWhenLabelsInheritedNotExistedInNodeLabelManager()
+      throws IOException {
+    YarnConfiguration conf = new YarnConfiguration();
+    CapacitySchedulerConfiguration csConf =
+        new CapacitySchedulerConfiguration(conf);
+    setupQueueConfigurationWithLabelsInherit(csConf);
+
+    CapacityScheduler capacityScheduler = new CapacityScheduler();
+    RMContextImpl rmContext =
+        new RMContextImpl(null, null, null, null, null, null,
+            new RMContainerTokenSecretManager(csConf),
+            new NMTokenSecretManagerInRM(csConf),
+            new ClientToAMTokenSecretManagerInRM(), null);
+    
+    RMNodeLabelsManager nodeLabelsManager = new MemoryRMNodeLabelsManager();
+    nodeLabelsManager.init(conf);
+    nodeLabelsManager.start();
+    
+    rmContext.setNodeLabelManager(nodeLabelsManager);
+    capacityScheduler.setConf(csConf);
+    capacityScheduler.setRMContext(rmContext);
+    capacityScheduler.init(csConf);
+    capacityScheduler.start();
+    ServiceOperations.stopQuietly(capacityScheduler);
+    ServiceOperations.stopQuietly(nodeLabelsManager);
+  }
+  
+  @Test(expected = Exception.class)
+  public void testSingleLevelQueueParsingWhenLabelsNotExistedInNodeLabelManager()
+      throws IOException {
+    YarnConfiguration conf = new YarnConfiguration();
+    CapacitySchedulerConfiguration csConf =
+        new CapacitySchedulerConfiguration(conf);
+    setupQueueConfigurationWithSingleLevel(csConf);
+
+    CapacityScheduler capacityScheduler = new CapacityScheduler();
+    RMContextImpl rmContext =
+        new RMContextImpl(null, null, null, null, null, null,
+            new RMContainerTokenSecretManager(csConf),
+            new NMTokenSecretManagerInRM(csConf),
+            new ClientToAMTokenSecretManagerInRM(), null);
+    
+    RMNodeLabelsManager nodeLabelsManager = new MemoryRMNodeLabelsManager();
+    nodeLabelsManager.init(conf);
+    nodeLabelsManager.start();
+    
+    rmContext.setNodeLabelManager(nodeLabelsManager);
+    capacityScheduler.setConf(csConf);
+    capacityScheduler.setRMContext(rmContext);
+    capacityScheduler.init(csConf);
+    capacityScheduler.start();
+    ServiceOperations.stopQuietly(capacityScheduler);
+    ServiceOperations.stopQuietly(nodeLabelsManager);
+  }
+  
+  @Test(expected = Exception.class)
+  public void testQueueParsingWhenLabelsNotExist() throws IOException {
+    YarnConfiguration conf = new YarnConfiguration();
+    CapacitySchedulerConfiguration csConf =
+        new CapacitySchedulerConfiguration(conf);
+    setupQueueConfigurationWithLabels(csConf);
+
+    CapacityScheduler capacityScheduler = new CapacityScheduler();
+    RMContextImpl rmContext =
+        new RMContextImpl(null, null, null, null, null, null,
+            new RMContainerTokenSecretManager(csConf),
+            new NMTokenSecretManagerInRM(csConf),
+            new ClientToAMTokenSecretManagerInRM(), null);
+    
+    RMNodeLabelsManager nodeLabelsManager = new MemoryRMNodeLabelsManager();
+    nodeLabelsManager.init(conf);
+    nodeLabelsManager.start();
+    
+    rmContext.setNodeLabelManager(nodeLabelsManager);
+    capacityScheduler.setConf(csConf);
+    capacityScheduler.setRMContext(rmContext);
+    capacityScheduler.init(csConf);
+    capacityScheduler.start();
+    ServiceOperations.stopQuietly(capacityScheduler);
+    ServiceOperations.stopQuietly(nodeLabelsManager);
   }
 }


[24/30] hadoop git commit: YARN-2819. NPE in ATS Timeline Domains when upgrading from 2.4 to 2.6. Contributed by Zhijie Shen

Posted by vi...@apache.org.
YARN-2819. NPE in ATS Timeline Domains when upgrading from 2.4 to 2.6. 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/4a114dd6
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/4a114dd6
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/4a114dd6

Branch: refs/heads/HDFS-EC
Commit: 4a114dd67aae83e5bb2d65470166de954acf36a2
Parents: 1e215e8
Author: Xuan <xg...@apache.org>
Authored: Fri Nov 7 16:11:30 2014 -0800
Committer: Xuan <xg...@apache.org>
Committed: Fri Nov 7 16:11:30 2014 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../server/timeline/LeveldbTimelineStore.java   |  46 ++++--
 .../server/timeline/TimelineDataManager.java    |  20 ++-
 .../timeline/TestLeveldbTimelineStore.java      |  81 +++++++++-
 .../timeline/TestTimelineDataManager.java       | 152 +++++++++++++++++++
 .../server/timeline/TimelineStoreTestUtils.java |  12 ++
 6 files changed, 293 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a114dd6/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index df7e3ea..d4c8827 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -911,6 +911,9 @@ Release 2.6.0 - UNRELEASED
 
     YARN-2825. Container leak on NM (Jian He via jlowe)
 
+    YARN-2819. NPE in ATS Timeline Domains when upgrading from 2.4 to 2.6.
+    (Zhijie Shen via xgong)
+
 Release 2.5.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a114dd6/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 e1f790d..c4ea996 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
@@ -792,7 +792,8 @@ public class LeveldbTimelineStore extends AbstractService
    * Put a single entity.  If there is an error, add a TimelinePutError to the
    * given response.
    */
-  private void put(TimelineEntity entity, TimelinePutResponse response) {
+  private void put(TimelineEntity entity, TimelinePutResponse response,
+      boolean allowEmptyDomainId) {
     LockMap.CountingReentrantLock<EntityIdentifier> lock =
         writeLocks.getLock(new EntityIdentifier(entity.getEntityId(),
             entity.getEntityType()));
@@ -867,10 +868,18 @@ public class LeveldbTimelineStore extends AbstractService
                   new EntityIdentifier(relatedEntityId, relatedEntityType));
               continue;
             } else {
+              // This is the existing entity
               byte[] domainIdBytes = db.get(createDomainIdKey(
                   relatedEntityId, relatedEntityType, relatedEntityStartTime));
-              // This is the existing entity
-              String domainId = new String(domainIdBytes);
+              // The timeline data created by the server before 2.6 won't have
+              // the domain field. We assume this timeline data is in the
+              // default timeline domain.
+              String domainId = null;
+              if (domainIdBytes == null) {
+                domainId = TimelineDataManager.DEFAULT_DOMAIN_ID;
+              } else {
+                domainId = new String(domainIdBytes);
+              }
               if (!domainId.equals(entity.getDomainId())) {
                 // in this case the entity will be put, but the relation will be
                 // ignored
@@ -923,12 +932,14 @@ public class LeveldbTimelineStore extends AbstractService
           entity.getEntityType(), revStartTime);
       if (entity.getDomainId() == null ||
           entity.getDomainId().length() == 0) {
-        TimelinePutError error = new TimelinePutError();
-        error.setEntityId(entity.getEntityId());
-        error.setEntityType(entity.getEntityType());
-        error.setErrorCode(TimelinePutError.NO_DOMAIN);
-        response.addError(error);
-        return;
+        if (!allowEmptyDomainId) {
+          TimelinePutError error = new TimelinePutError();
+          error.setEntityId(entity.getEntityId());
+          error.setEntityType(entity.getEntityType());
+          error.setErrorCode(TimelinePutError.NO_DOMAIN);
+          response.addError(error);
+          return;
+        }
       } else {
         writeBatch.put(key, entity.getDomainId().getBytes());
         writePrimaryFilterEntries(writeBatch, primaryFilters, key,
@@ -1011,7 +1022,22 @@ public class LeveldbTimelineStore extends AbstractService
       deleteLock.readLock().lock();
       TimelinePutResponse response = new TimelinePutResponse();
       for (TimelineEntity entity : entities.getEntities()) {
-        put(entity, response);
+        put(entity, response, false);
+      }
+      return response;
+    } finally {
+      deleteLock.readLock().unlock();
+    }
+  }
+
+  @Private
+  @VisibleForTesting
+  public TimelinePutResponse putWithNoDomainId(TimelineEntities entities) {
+    try {
+      deleteLock.readLock().lock();
+      TimelinePutResponse response = new TimelinePutResponse();
+      for (TimelineEntity entity : entities.getEntities()) {
+        put(entity, response, true);
       }
       return response;
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a114dd6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.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/TimelineDataManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java
index 7ef0a67..888c283 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java
@@ -124,6 +124,7 @@ public class TimelineDataManager extends AbstractService {
           entities.getEntities().iterator();
       while (entitiesItr.hasNext()) {
         TimelineEntity entity = entitiesItr.next();
+        addDefaultDomainIdIfAbsent(entity);
         try {
           // check ACLs
           if (!timelineACLsManager.checkAccess(
@@ -161,6 +162,7 @@ public class TimelineDataManager extends AbstractService {
     entity =
         store.getEntity(entityId, entityType, fields);
     if (entity != null) {
+      addDefaultDomainIdIfAbsent(entity);
       // check ACLs
       if (!timelineACLsManager.checkAccess(
           callerUGI, ApplicationAccessType.VIEW_APP, entity)) {
@@ -203,6 +205,7 @@ public class TimelineDataManager extends AbstractService {
               eventsOfOneEntity.getEntityId(),
               eventsOfOneEntity.getEntityType(),
               EnumSet.of(Field.PRIMARY_FILTERS));
+          addDefaultDomainIdIfAbsent(entity);
           // check ACLs
           if (!timelineACLsManager.checkAccess(
               callerUGI, ApplicationAccessType.VIEW_APP, entity)) {
@@ -254,10 +257,12 @@ public class TimelineDataManager extends AbstractService {
         existingEntity =
             store.getEntity(entityID.getId(), entityID.getType(),
                 EnumSet.of(Field.PRIMARY_FILTERS));
-        if (existingEntity != null &&
-            !existingEntity.getDomainId().equals(entity.getDomainId())) {
-          throw new YarnException("The domain of the timeline entity "
-            + entityID + " is not allowed to be changed.");
+        if (existingEntity != null) {
+          addDefaultDomainIdIfAbsent(existingEntity);
+          if (!existingEntity.getDomainId().equals(entity.getDomainId())) {
+            throw new YarnException("The domain of the timeline entity "
+              + entityID + " is not allowed to be changed.");
+          }
         }
         if (!timelineACLsManager.checkAccess(
             callerUGI, ApplicationAccessType.MODIFY_APP, entity)) {
@@ -355,4 +360,11 @@ public class TimelineDataManager extends AbstractService {
     }
   }
 
+  private static void addDefaultDomainIdIfAbsent(TimelineEntity entity) {
+    // be compatible with the timeline data created before 2.6
+    if (entity.getDomainId() == null) {
+      entity.setDomainId(DEFAULT_DOMAIN_ID);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a114dd6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLeveldbTimelineStore.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/TestLeveldbTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLeveldbTimelineStore.java
index f315930..5ebc96b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLeveldbTimelineStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLeveldbTimelineStore.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.service.ServiceStateException;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
+import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse.TimelinePutError;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.records.Version;
 import org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore;
@@ -160,12 +161,13 @@ public class TestLeveldbTimelineStore extends TimelineStoreTestUtils {
   @Test
   public void testGetEntityTypes() throws IOException {
     List<String> entityTypes = ((LeveldbTimelineStore)store).getEntityTypes();
-    assertEquals(5, entityTypes.size());
-    assertEquals(entityType1, entityTypes.get(0));
-    assertEquals(entityType2, entityTypes.get(1));
-    assertEquals(entityType4, entityTypes.get(2));
-    assertEquals(entityType5, entityTypes.get(3));
-    assertEquals(entityType7, entityTypes.get(4));
+    assertEquals(6, entityTypes.size());
+    assertEquals("OLD_ENTITY_TYPE_1", entityTypes.get(0));
+    assertEquals(entityType1, entityTypes.get(1));
+    assertEquals(entityType2, entityTypes.get(2));
+    assertEquals(entityType4, entityTypes.get(3));
+    assertEquals(entityType5, entityTypes.get(4));
+    assertEquals(entityType7, entityTypes.get(5));
   }
 
   @Test
@@ -196,7 +198,7 @@ public class TestLeveldbTimelineStore extends TimelineStoreTestUtils {
     ((LeveldbTimelineStore)store).discardOldEntities(-123l);
     assertEquals(2, getEntities("type_1").size());
     assertEquals(0, getEntities("type_2").size());
-    assertEquals(4, ((LeveldbTimelineStore)store).getEntityTypes().size());
+    assertEquals(5, ((LeveldbTimelineStore)store).getEntityTypes().size());
 
     ((LeveldbTimelineStore)store).discardOldEntities(123l);
     assertEquals(0, getEntities("type_1").size());
@@ -327,4 +329,69 @@ public class TestLeveldbTimelineStore extends TimelineStoreTestUtils {
     super.testGetDomains();
   }
 
+  @Test
+  public void testRelatingToNonExistingEntity() throws IOException {
+    TimelineEntity entityToStore = new TimelineEntity();
+    entityToStore.setEntityType("TEST_ENTITY_TYPE_1");
+    entityToStore.setEntityId("TEST_ENTITY_ID_1");
+    entityToStore.setDomainId(TimelineDataManager.DEFAULT_DOMAIN_ID);
+    entityToStore.addRelatedEntity("TEST_ENTITY_TYPE_2", "TEST_ENTITY_ID_2");
+    TimelineEntities entities = new TimelineEntities();
+    entities.addEntity(entityToStore);
+    store.put(entities);
+    TimelineEntity entityToGet =
+        store.getEntity("TEST_ENTITY_ID_2", "TEST_ENTITY_TYPE_2", null);
+    Assert.assertNotNull(entityToGet);
+    Assert.assertEquals("DEFAULT", entityToGet.getDomainId());
+    Assert.assertEquals("TEST_ENTITY_TYPE_1",
+        entityToGet.getRelatedEntities().keySet().iterator().next());
+    Assert.assertEquals("TEST_ENTITY_ID_1",
+        entityToGet.getRelatedEntities().values().iterator().next()
+            .iterator().next());
+  }
+
+  @Test
+  public void testRelatingToOldEntityWithoutDomainId() throws IOException {
+    // New entity is put in the default domain
+    TimelineEntity entityToStore = new TimelineEntity();
+    entityToStore.setEntityType("NEW_ENTITY_TYPE_1");
+    entityToStore.setEntityId("NEW_ENTITY_ID_1");
+    entityToStore.setDomainId(TimelineDataManager.DEFAULT_DOMAIN_ID);
+    entityToStore.addRelatedEntity("OLD_ENTITY_TYPE_1", "OLD_ENTITY_ID_1");
+    TimelineEntities entities = new TimelineEntities();
+    entities.addEntity(entityToStore);
+    store.put(entities);
+
+    TimelineEntity entityToGet =
+        store.getEntity("OLD_ENTITY_ID_1", "OLD_ENTITY_TYPE_1", null);
+    Assert.assertNotNull(entityToGet);
+    Assert.assertNull(entityToGet.getDomainId());
+    Assert.assertEquals("NEW_ENTITY_TYPE_1",
+        entityToGet.getRelatedEntities().keySet().iterator().next());
+    Assert.assertEquals("NEW_ENTITY_ID_1",
+        entityToGet.getRelatedEntities().values().iterator().next()
+            .iterator().next());
+
+    // New entity is not put in the default domain
+    entityToStore = new TimelineEntity();
+    entityToStore.setEntityType("NEW_ENTITY_TYPE_2");
+    entityToStore.setEntityId("NEW_ENTITY_ID_2");
+    entityToStore.setDomainId("NON_DEFAULT");
+    entityToStore.addRelatedEntity("OLD_ENTITY_TYPE_1", "OLD_ENTITY_ID_1");
+    entities = new TimelineEntities();
+    entities.addEntity(entityToStore);
+    TimelinePutResponse response = store.put(entities);
+    Assert.assertEquals(1, response.getErrors().size());
+    Assert.assertEquals(TimelinePutError.FORBIDDEN_RELATION,
+        response.getErrors().get(0).getErrorCode());
+    entityToGet =
+        store.getEntity("OLD_ENTITY_ID_1", "OLD_ENTITY_TYPE_1", null);
+    Assert.assertNotNull(entityToGet);
+    Assert.assertNull(entityToGet.getDomainId());
+    // Still have one related entity
+    Assert.assertEquals(1, entityToGet.getRelatedEntities().keySet().size());
+    Assert.assertEquals(1, entityToGet.getRelatedEntities().values()
+        .iterator().next().size());
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a114dd6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestTimelineDataManager.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/TestTimelineDataManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestTimelineDataManager.java
new file mode 100644
index 0000000..f749567
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestTimelineDataManager.java
@@ -0,0 +1,152 @@
+/**
+ * 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.timeline;
+
+import java.io.File;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.timeline.security.TimelineACLsManager;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+
+public class TestTimelineDataManager extends TimelineStoreTestUtils {
+
+  private FileContext fsContext;
+  private File fsPath;
+  private TimelineDataManager dataManaer;
+
+  @Before
+  public void setup() throws Exception {
+    fsPath = new File("target", this.getClass().getSimpleName() +
+        "-tmpDir").getAbsoluteFile();
+    fsContext = FileContext.getLocalFSFileContext();
+    fsContext.delete(new Path(fsPath.getAbsolutePath()), true);
+    Configuration conf = new YarnConfiguration();
+    conf.set(YarnConfiguration.TIMELINE_SERVICE_LEVELDB_PATH,
+        fsPath.getAbsolutePath());
+    conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_TTL_ENABLE, false);
+    store = new LeveldbTimelineStore();
+    store.init(conf);
+    store.start();
+    loadTestEntityData();
+    loadVerificationEntityData();
+    loadTestDomainData();
+
+    TimelineACLsManager aclsManager = new TimelineACLsManager(conf);
+    dataManaer = new TimelineDataManager(store, aclsManager);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (store != null) {
+      store.stop();
+    }
+    if (fsContext != null) {
+      fsContext.delete(new Path(fsPath.getAbsolutePath()), true);
+    }
+  }
+
+  @Test
+  public void testGetOldEntityWithOutDomainId() throws Exception {
+    TimelineEntity entity = dataManaer.getEntity(
+        "OLD_ENTITY_TYPE_1", "OLD_ENTITY_ID_1", null,
+        UserGroupInformation.getCurrentUser());
+    Assert.assertNotNull(entity);
+    Assert.assertEquals("OLD_ENTITY_ID_1", entity.getEntityId());
+    Assert.assertEquals("OLD_ENTITY_TYPE_1", entity.getEntityType());
+    Assert.assertEquals(
+        TimelineDataManager.DEFAULT_DOMAIN_ID, entity.getDomainId());
+  }
+
+  @Test
+  public void testGetOldEntitiesWithOutDomainId() throws Exception {
+    TimelineEntities entities = dataManaer.getEntities(
+        "OLD_ENTITY_TYPE_1", null, null, null, null, null, null, null, null,
+        UserGroupInformation.getCurrentUser());
+    Assert.assertEquals(2, entities.getEntities().size());
+    Assert.assertEquals("OLD_ENTITY_ID_2",
+        entities.getEntities().get(0).getEntityId());
+    Assert.assertEquals("OLD_ENTITY_TYPE_1",
+        entities.getEntities().get(0).getEntityType());
+    Assert.assertEquals(TimelineDataManager.DEFAULT_DOMAIN_ID,
+        entities.getEntities().get(0).getDomainId());
+    Assert.assertEquals("OLD_ENTITY_ID_1",
+        entities.getEntities().get(1).getEntityId());
+    Assert.assertEquals("OLD_ENTITY_TYPE_1",
+        entities.getEntities().get(1).getEntityType());
+    Assert.assertEquals(TimelineDataManager.DEFAULT_DOMAIN_ID,
+        entities.getEntities().get(1).getDomainId());
+  }
+
+  @Test
+  public void testUpdatingOldEntityWithoutDomainId() throws Exception {
+    // Set the domain to the default domain when updating
+    TimelineEntity entity = new TimelineEntity();
+    entity.setEntityType("OLD_ENTITY_TYPE_1");
+    entity.setEntityId("OLD_ENTITY_ID_1");
+    entity.setDomainId(TimelineDataManager.DEFAULT_DOMAIN_ID);
+    entity.addOtherInfo("NEW_OTHER_INFO_KEY", "NEW_OTHER_INFO_VALUE");
+    TimelineEntities entities = new TimelineEntities();
+    entities.addEntity(entity);
+    TimelinePutResponse response = dataManaer.postEntities(
+        entities, UserGroupInformation.getCurrentUser());
+    Assert.assertEquals(0, response.getErrors().size());
+    entity = store.getEntity("OLD_ENTITY_ID_1", "OLD_ENTITY_TYPE_1", null);
+    Assert.assertNotNull(entity);
+    // Even in leveldb, the domain is updated to the default domain Id
+    Assert.assertEquals(
+        TimelineDataManager.DEFAULT_DOMAIN_ID, entity.getDomainId());
+    Assert.assertEquals(1, entity.getOtherInfo().size());
+    Assert.assertEquals("NEW_OTHER_INFO_KEY",
+        entity.getOtherInfo().keySet().iterator().next());
+    Assert.assertEquals("NEW_OTHER_INFO_VALUE",
+        entity.getOtherInfo().values().iterator().next());
+    
+    // Set the domain to the non-default domain when updating
+    entity = new TimelineEntity();
+    entity.setEntityType("OLD_ENTITY_TYPE_1");
+    entity.setEntityId("OLD_ENTITY_ID_2");
+    entity.setDomainId("NON_DEFAULT");
+    entity.addOtherInfo("NEW_OTHER_INFO_KEY", "NEW_OTHER_INFO_VALUE");
+    entities = new TimelineEntities();
+    entities.addEntity(entity);
+    response = dataManaer.postEntities(
+        entities, UserGroupInformation.getCurrentUser());
+    Assert.assertEquals(1, response.getErrors().size());
+    Assert.assertEquals(TimelinePutResponse.TimelinePutError.ACCESS_DENIED,
+        response.getErrors().get(0).getErrorCode());
+    entity = store.getEntity("OLD_ENTITY_ID_2", "OLD_ENTITY_TYPE_1", null);
+    Assert.assertNotNull(entity);
+    // In leveldb, the domain Id is still null
+    Assert.assertNull(entity.getDomainId());
+    // Updating is not executed
+    Assert.assertEquals(0, entity.getOtherInfo().size());
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a114dd6/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 242478c..6f15b92 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
@@ -210,6 +210,18 @@ public class TimelineStoreTestUtils {
     assertEquals(entityId7, response.getErrors().get(0).getEntityId());
     assertEquals(TimelinePutError.FORBIDDEN_RELATION,
         response.getErrors().get(0).getErrorCode());
+
+    if (store instanceof LeveldbTimelineStore) {
+      LeveldbTimelineStore leveldb = (LeveldbTimelineStore) store;
+      entities.setEntities(Collections.singletonList(createEntity(
+          "OLD_ENTITY_ID_1", "OLD_ENTITY_TYPE_1", 63l, null, null, null, null,
+          null)));
+      leveldb.putWithNoDomainId(entities);
+      entities.setEntities(Collections.singletonList(createEntity(
+          "OLD_ENTITY_ID_2", "OLD_ENTITY_TYPE_1", 64l, null, null, null, null,
+          null)));
+      leveldb.putWithNoDomainId(entities);
+    }
   }
 
   /**


[02/30] hadoop git commit: YARN-2818. Removed the now unnecessary user entity injection from Timeline service given we now have domains. Contributed by Zhijie Shen.

Posted by vi...@apache.org.
YARN-2818. Removed the now unnecessary user entity injection from Timeline service given we now have domains. 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/f5b19bed
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f5b19bed
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f5b19bed

Branch: refs/heads/HDFS-EC
Commit: f5b19bed7d71979dc8685b03152188902b6e45e9
Parents: 10f9f51
Author: Vinod Kumar Vavilapalli <vi...@apache.org>
Authored: Thu Nov 6 11:48:40 2014 -0800
Committer: Vinod Kumar Vavilapalli <vi...@apache.org>
Committed: Thu Nov 6 11:48:40 2014 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../server/timeline/TimelineDataManager.java    | 71 +-------------------
 .../webapp/TestTimelineWebServices.java         |  6 +-
 3 files changed, 5 insertions(+), 75 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f5b19bed/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index aefc59f..0fe957c 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -429,6 +429,9 @@ Release 2.6.0 - UNRELEASED
     YARN-2770. Added functionality to renew/cancel TimeLineDelegationToken.
     (Zhijie Shen via jianhe)
 
+    YARN-2818. Removed the now unnecessary user entity injection from Timeline
+    service given we now have domains. (Zhijie Shen via vinodkv)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f5b19bed/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.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/TimelineDataManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java
index 3b6aafa..7ef0a67 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java
@@ -109,7 +109,6 @@ public class TimelineDataManager extends AbstractService {
       EnumSet<Field> fields,
       UserGroupInformation callerUGI) throws YarnException, IOException {
     TimelineEntities entities = null;
-    boolean modified = extendFields(fields);
     entities = store.getEntities(
         entityType,
         limit,
@@ -130,13 +129,6 @@ public class TimelineDataManager extends AbstractService {
           if (!timelineACLsManager.checkAccess(
               callerUGI, ApplicationAccessType.VIEW_APP, entity)) {
             entitiesItr.remove();
-          } else {
-            // clean up system data
-            if (modified) {
-              entity.setPrimaryFilters(null);
-            } else {
-              cleanupOwnerInfo(entity);
-            }
           }
         } catch (YarnException e) {
           LOG.error("Error when verifying access for user " + callerUGI
@@ -166,7 +158,6 @@ public class TimelineDataManager extends AbstractService {
       EnumSet<Field> fields,
       UserGroupInformation callerUGI) throws YarnException, IOException {
     TimelineEntity entity = null;
-    boolean modified = extendFields(fields);
     entity =
         store.getEntity(entityId, entityType, fields);
     if (entity != null) {
@@ -174,13 +165,6 @@ public class TimelineDataManager extends AbstractService {
       if (!timelineACLsManager.checkAccess(
           callerUGI, ApplicationAccessType.VIEW_APP, entity)) {
         entity = null;
-      } else {
-        // clean up the system data
-        if (modified) {
-          entity.setPrimaryFilters(null);
-        } else {
-          cleanupOwnerInfo(entity);
-        }
       }
     }
     return entity;
@@ -283,8 +267,7 @@ public class TimelineDataManager extends AbstractService {
         }
       } catch (Exception e) {
         // Skip the entity which already exists and was put by others
-        LOG.error("Skip the timeline entity: " + entityID + ", because "
-            + e.getMessage());
+        LOG.error("Skip the timeline entity: " + entityID, e);
         TimelinePutResponse.TimelinePutError error =
             new TimelinePutResponse.TimelinePutError();
         error.setEntityId(entityID.getId());
@@ -295,28 +278,6 @@ public class TimelineDataManager extends AbstractService {
         continue;
       }
 
-      // inject owner information for the access check if this is the first
-      // time to post the entity, in case it's the admin who is updating
-      // the timeline data.
-      try {
-        if (existingEntity == null) {
-          injectOwnerInfo(entity, callerUGI.getShortUserName());
-        }
-      } catch (YarnException e) {
-        // Skip the entity which messes up the primary filter and record the
-        // error
-        LOG.error("Skip the timeline entity: " + entityID + ", because "
-            + e.getMessage());
-        TimelinePutResponse.TimelinePutError error =
-            new TimelinePutResponse.TimelinePutError();
-        error.setEntityId(entityID.getId());
-        error.setEntityType(entityID.getType());
-        error.setErrorCode(
-            TimelinePutResponse.TimelinePutError.SYSTEM_FILTER_CONFLICT);
-        errors.add(error);
-        continue;
-      }
-
       entityIDs.add(entityID);
       entitiesToPut.addEntity(entity);
       if (LOG.isDebugEnabled()) {
@@ -394,34 +355,4 @@ public class TimelineDataManager extends AbstractService {
     }
   }
 
-  private static boolean extendFields(EnumSet<Field> fieldEnums) {
-    boolean modified = false;
-    if (fieldEnums != null && !fieldEnums.contains(Field.PRIMARY_FILTERS)) {
-      fieldEnums.add(Field.PRIMARY_FILTERS);
-      modified = true;
-    }
-    return modified;
-  }
-
-  private static void injectOwnerInfo(TimelineEntity timelineEntity,
-      String owner) throws YarnException {
-    if (timelineEntity.getPrimaryFilters() != null &&
-        timelineEntity.getPrimaryFilters().containsKey(
-            TimelineStore.SystemFilter.ENTITY_OWNER.toString())) {
-      throw new YarnException(
-          "User should not use the timeline system filter key: "
-              + TimelineStore.SystemFilter.ENTITY_OWNER);
-    }
-    timelineEntity.addPrimaryFilter(
-        TimelineStore.SystemFilter.ENTITY_OWNER
-            .toString(), owner);
-  }
-
-  private static void cleanupOwnerInfo(TimelineEntity timelineEntity) {
-    if (timelineEntity.getPrimaryFilters() != null) {
-      timelineEntity.getPrimaryFilters().remove(
-          TimelineStore.SystemFilter.ENTITY_OWNER.toString());
-    }
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f5b19bed/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestTimelineWebServices.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/webapp/TestTimelineWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestTimelineWebServices.java
index 8a7f27a..fe2ed5c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestTimelineWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestTimelineWebServices.java
@@ -443,11 +443,7 @@ public class TestTimelineWebServices extends JerseyTest {
         .post(ClientResponse.class, entities);
     TimelinePutResponse putResposne =
         response.getEntity(TimelinePutResponse.class);
-    Assert.assertEquals(1, putResposne.getErrors().size());
-    List<TimelinePutError> errors = putResposne.getErrors();
-    Assert.assertEquals(
-        TimelinePutResponse.TimelinePutError.SYSTEM_FILTER_CONFLICT,
-        errors.get(0).getErrorCode());
+    Assert.assertEquals(0, putResposne.getErrors().size());
   }
 
   @Test


[30/30] hadoop git commit: Merge remote-tracking branch 'origin/trunk' into HDFS-EC

Posted by vi...@apache.org.
Merge remote-tracking branch 'origin/trunk' into HDFS-EC


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

Branch: refs/heads/HDFS-EC
Commit: 3a1b3f82dffe91cd4b946132300eef2dc3f25737
Parents: 4ce3a13 9ba8d8c
Author: Vinayakumar B <vi...@intel.com>
Authored: Sun Nov 9 22:23:04 2014 +0530
Committer: Vinayakumar B <vi...@intel.com>
Committed: Sun Nov 9 22:23:04 2014 +0530

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  10 +
 .../java/org/apache/hadoop/fs/FileContext.java  |   3 +
 .../apache/hadoop/util/LimitInputStream.java    | 109 ++++++
 .../security/TestShellBasedIdMapping.java       |   5 +
 .../org/apache/hadoop/util/TestWinUtils.java    |   2 +-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  20 +-
 .../sasl/SaslDataTransferServer.java            |  13 +-
 .../hadoop/hdfs/server/balancer/Balancer.java   |  81 +++--
 .../hadoop/hdfs/server/balancer/Dispatcher.java |  13 +-
 .../hdfs/server/balancer/NameNodeConnector.java |   6 +
 .../blockmanagement/MutableBlockCollection.java |  38 --
 .../hdfs/server/datanode/DataXceiver.java       |   3 +-
 .../server/namenode/FSImageFormatProtobuf.java  |   2 +-
 .../tools/offlineImageViewer/FSImageLoader.java |   2 +-
 .../FileDistributionCalculator.java             |   2 +-
 .../offlineImageViewer/PBImageXmlWriter.java    |   2 +-
 .../hdfs/server/balancer/TestBalancer.java      |  86 ++++-
 hadoop-mapreduce-project/CHANGES.txt            |   6 +
 .../v2/app/job/impl/TaskAttemptImpl.java        |   8 +-
 .../java/org/apache/hadoop/mapred/Merger.java   |  12 +-
 .../apache/hadoop/mapreduce/CryptoUtils.java    |   3 +-
 .../apache/hadoop/mapreduce/JobSubmitter.java   |   9 +-
 .../mapreduce/task/reduce/TestMerger.java       |  80 +++--
 .../org/apache/hadoop/mapred/YARNRunner.java    |   5 +-
 .../apache/hadoop/mapreduce/v2/TestMRJobs.java  |  41 ++-
 .../apache/hadoop/mapreduce/v2/TestUberAM.java  |  40 +--
 hadoop-project/src/site/site.xml                |   1 +
 hadoop-yarn-project/CHANGES.txt                 |  45 +++
 hadoop-yarn-project/hadoop-yarn/bin/yarn        |   6 +
 hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd    |   6 +
 .../dev-support/findbugs-exclude.xml            |   1 +
 .../distributedshell/TestDistributedShell.java  | 138 +------
 .../TestDistributedShellWithNodeLabels.java     | 165 +++++++++
 .../apache/hadoop/yarn/client/cli/QueueCLI.java | 152 ++++++++
 .../hadoop/yarn/client/cli/TestYarnCLI.java     |  78 ++++
 .../nodelabels/CommonNodeLabelsManager.java     |  25 +-
 .../apache/hadoop/yarn/util/ConverterUtils.java |   7 +
 .../hadoop/yarn/util/TestConverterUtils.java    |  14 +
 .../apache/hadoop/registry/cli/RegistryCli.java |  25 +-
 .../registry/client/binding/JsonSerDeser.java   | 142 ++++----
 .../client/binding/RegistryTypeUtils.java       | 166 ++++++---
 .../registry/client/binding/RegistryUtils.java  |   7 +-
 .../client/exceptions/NoRecordException.java    |  10 +-
 .../impl/zk/RegistryOperationsService.java      |  12 +-
 .../registry/client/types/AddressTypes.java     |   2 +
 .../hadoop/registry/client/types/Endpoint.java  | 131 +++++--
 .../registry/client/types/ProtocolTypes.java    |   7 +-
 .../registry/client/types/ServiceRecord.java    |  26 +-
 .../client/types/ServiceRecordHeader.java       |  59 ---
 .../src/main/tla/yarnregistry.tla               |  94 +++--
 .../hadoop/registry/RegistryTestHelper.java     |  36 +-
 .../client/binding/TestMarshalling.java         |  72 ++--
 .../operations/TestRegistryOperations.java      |   5 +-
 .../server/timeline/LeveldbTimelineStore.java   |  46 ++-
 .../server/timeline/TimelineDataManager.java    |  85 +----
 .../timeline/TestLeveldbTimelineStore.java      |  81 ++++-
 .../timeline/TestTimelineDataManager.java       | 152 ++++++++
 .../server/timeline/TimelineStoreTestUtils.java |  12 +
 .../webapp/TestTimelineWebServices.java         |   6 +-
 .../nodemanager/NodeStatusUpdaterImpl.java      |  27 +-
 .../launcher/ContainerLaunch.java               |  12 +-
 .../nodemanager/TestNodeStatusUpdater.java      |  54 ++-
 .../server/resourcemanager/ResourceManager.java |  32 +-
 .../rmapp/attempt/RMAppAttemptImpl.java         |   4 +
 .../scheduler/capacity/AbstractCSQueue.java     |  32 +-
 .../CapacitySchedulerConfiguration.java         |  17 +-
 .../resourcemanager/webapp/RMWebServices.java   | 180 +++++++++-
 .../dao/ApplicationSubmissionContextInfo.java   |  23 ++
 .../webapp/dao/NodeLabelsInfo.java              |  52 +++
 .../webapp/dao/NodeToLabelsInfo.java            |  41 +++
 .../resourcemanager/TestRMProxyUsersConf.java   |   9 +-
 .../server/resourcemanager/TestRMRestart.java   |   5 +-
 .../nodelabels/TestRMNodeLabelsManager.java     |  29 +-
 .../capacity/TestContainerAllocation.java       |   5 +
 .../scheduler/capacity/TestQueueParsing.java    | 226 +++++++++++-
 .../webapp/TestRMWebServicesNodeLabels.java     | 357 +++++++++++++++++++
 .../src/site/apt/NodeManagerRestart.apt.vm      |  86 +++++
 .../src/site/markdown/registry/yarn-registry.md | 233 +++++++++---
 78 files changed, 2939 insertions(+), 900 deletions(-)
----------------------------------------------------------------------



[29/30] hadoop git commit: HDFS-7382. DataNode in secure mode may throw NullPointerException if client connects before DataNode registers itself with NameNode. Contributed by Chris Nauroth.

Posted by vi...@apache.org.
HDFS-7382. DataNode in secure mode may throw NullPointerException if client connects before DataNode registers itself with NameNode. Contributed by Chris Nauroth.


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

Branch: refs/heads/HDFS-EC
Commit: 9ba8d8c7eb65eeb6fe673f04e493d9eedd95a822
Parents: 6caa810
Author: cnauroth <cn...@apache.org>
Authored: Sat Nov 8 22:24:57 2014 -0800
Committer: cnauroth <cn...@apache.org>
Committed: Sat Nov 8 22:24:57 2014 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt            |  3 +++
 .../datatransfer/sasl/SaslDataTransferServer.java      | 13 +++++++------
 .../hadoop/hdfs/server/datanode/DataXceiver.java       |  3 ++-
 3 files changed, 12 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ba8d8c7/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 0595764..6bde9bc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1412,6 +1412,9 @@ Release 2.6.0 - UNRELEASED
 
     HDFS-7226. Fix TestDNFencing.testQueueingWithAppend. (Yongjun Zhang via jing9)
 
+    HDFS-7382. DataNode in secure mode may throw NullPointerException if client
+    connects before DataNode registers itself with NameNode. (cnauroth)
+
 Release 2.5.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ba8d8c7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferServer.java
index 9f94534..3fa7727 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferServer.java
@@ -94,12 +94,14 @@ public class SaslDataTransferServer {
    * @param peer connection peer
    * @param underlyingOut connection output stream
    * @param underlyingIn connection input stream
+   * @param int xferPort data transfer port of DataNode accepting connection
    * @param datanodeId ID of DataNode accepting connection
    * @return new pair of streams, wrapped after SASL negotiation
    * @throws IOException for any error
    */
   public IOStreamPair receive(Peer peer, OutputStream underlyingOut,
-      InputStream underlyingIn, DatanodeID datanodeId) throws IOException {
+      InputStream underlyingIn, int xferPort, DatanodeID datanodeId)
+      throws IOException {
     if (dnConf.getEncryptDataTransfer()) {
       LOG.debug(
         "SASL server doing encrypted handshake for peer = {}, datanodeId = {}",
@@ -110,16 +112,16 @@ public class SaslDataTransferServer {
         "SASL server skipping handshake in unsecured configuration for "
         + "peer = {}, datanodeId = {}", peer, datanodeId);
       return new IOStreamPair(underlyingIn, underlyingOut);
-    } else if (datanodeId.getXferPort() < 1024) {
+    } else if (xferPort < 1024) {
       LOG.debug(
-        "SASL server skipping handshake in unsecured configuration for "
+        "SASL server skipping handshake in secured configuration for "
         + "peer = {}, datanodeId = {}", peer, datanodeId);
       return new IOStreamPair(underlyingIn, underlyingOut);
     } else if (dnConf.getSaslPropsResolver() != null) {
       LOG.debug(
         "SASL server doing general handshake for peer = {}, datanodeId = {}",
         peer, datanodeId);
-      return getSaslStreams(peer, underlyingOut, underlyingIn, datanodeId);
+      return getSaslStreams(peer, underlyingOut, underlyingIn);
     } else if (dnConf.getIgnoreSecurePortsForTesting()) {
       // It's a secured cluster using non-privileged ports, but no SASL.  The
       // only way this can happen is if the DataNode has
@@ -271,12 +273,11 @@ public class SaslDataTransferServer {
    * @param peer connection peer
    * @param underlyingOut connection output stream
    * @param underlyingIn connection input stream
-   * @param datanodeId ID of DataNode accepting connection
    * @return new pair of streams, wrapped after SASL negotiation
    * @throws IOException for any error
    */
   private IOStreamPair getSaslStreams(Peer peer, OutputStream underlyingOut,
-      InputStream underlyingIn, final DatanodeID datanodeId) throws IOException {
+      InputStream underlyingIn) throws IOException {
     if (peer.hasSecureChannel() ||
         dnConf.getTrustedChannelResolver().isTrusted(getPeerAddress(peer))) {
       return new IOStreamPair(underlyingIn, underlyingOut);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ba8d8c7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
index 2a45a42..a235c20 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
@@ -182,7 +182,8 @@ class DataXceiver extends Receiver implements Runnable {
       InputStream input = socketIn;
       try {
         IOStreamPair saslStreams = datanode.saslServer.receive(peer, socketOut,
-          socketIn, datanode.getDatanodeId());
+          socketIn, datanode.getXferAddress().getPort(),
+          datanode.getDatanodeId());
         input = new BufferedInputStream(saslStreams.in,
           HdfsConstants.SMALL_BUFFER_SIZE);
         socketOut = saslStreams.out;


[22/30] hadoop git commit: YARN-2825. Container leak on NM. Contributed by Jian He

Posted by vi...@apache.org.
YARN-2825. Container leak on NM. Contributed by Jian He


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

Branch: refs/heads/HDFS-EC
Commit: c3d475070a1ec54c4b05923f4782cef204effd2c
Parents: 68db5b3
Author: Jason Lowe <jl...@apache.org>
Authored: Fri Nov 7 23:16:37 2014 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Fri Nov 7 23:16:37 2014 +0000

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  2 +
 .../nodemanager/NodeStatusUpdaterImpl.java      | 27 ++++++----
 .../nodemanager/TestNodeStatusUpdater.java      | 54 ++++++++++++++++++--
 3 files changed, 71 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3d47507/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 6e5f59a..e4b116d 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -906,6 +906,8 @@ Release 2.6.0 - UNRELEASED
     YARN-2753. Fixed a bunch of bugs in the NodeLabelsManager classes. (Zhihai xu
     via vinodkv)
 
+    YARN-2825. Container leak on NM (Jian He via jlowe)
+
 Release 2.5.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3d47507/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.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/NodeStatusUpdaterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
index ebbe503..fcdd2c1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
@@ -66,8 +66,8 @@ import org.apache.hadoop.yarn.server.api.records.NodeAction;
 import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
 import org.apache.hadoop.yarn.server.api.records.NodeStatus;
 import org.apache.hadoop.yarn.server.nodemanager.NodeManager.NMContext;
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
 import org.apache.hadoop.yarn.util.YarnVersionInfo;
@@ -115,6 +115,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
   private Runnable statusUpdaterRunnable;
   private Thread  statusUpdater;
   private long rmIdentifier = ResourceManagerConstants.RM_INVALID_IDENTIFIER;
+  Set<ContainerId> pendingContainersToRemove = new HashSet<ContainerId>();
 
   public NodeStatusUpdaterImpl(Context context, Dispatcher dispatcher,
       NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics) {
@@ -446,19 +447,27 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
 
   @VisibleForTesting
   @Private
-  public void removeCompletedContainersFromContext(
+  public void removeOrTrackCompletedContainersFromContext(
       List<ContainerId> containerIds) throws IOException {
     Set<ContainerId> removedContainers = new HashSet<ContainerId>();
 
-    // If the AM has pulled the completedContainer it can be removed
-    for (ContainerId containerId : containerIds) {
-      context.getContainers().remove(containerId);
-      removedContainers.add(containerId);
+    pendingContainersToRemove.addAll(containerIds);
+    Iterator<ContainerId> iter = pendingContainersToRemove.iterator();
+    while (iter.hasNext()) {
+      ContainerId containerId = iter.next();
+      // remove the container only if the container is at DONE state
+      Container nmContainer = context.getContainers().get(containerId);
+      if (nmContainer != null && nmContainer.getContainerState().equals(
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState.DONE)) {
+        context.getContainers().remove(containerId);
+        removedContainers.add(containerId);
+        iter.remove();
+      }
     }
 
     if (!removedContainers.isEmpty()) {
-      LOG.info("Removed completed containers from NM context: " +
-          removedContainers);
+      LOG.info("Removed completed containers from NM context: "
+          + removedContainers);
     }
   }
 
@@ -601,7 +610,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
             // because these completed containers will be reported back to RM
             // when NM re-registers with RM.
             // Only remove the cleanedup containers that are acked
-            removeCompletedContainersFromContext(response
+            removeOrTrackCompletedContainersFromContext(response
                   .getContainersToBeRemovedFromNM());
 
             lastHeartBeatID = response.getResponseId();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3d47507/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.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/TestNodeStatusUpdater.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
index 5c2dd2c..925a249 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
@@ -30,9 +30,11 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.CyclicBarrier;
 import java.util.concurrent.TimeUnit;
@@ -866,18 +868,57 @@ public class TestNodeStatusUpdater {
       public ContainerState getCurrentState() {
         return ContainerState.COMPLETE;
       }
+
+      @Override
+      public org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState getContainerState() {
+        return org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState.DONE;
+      }
     };
 
+    ContainerId runningContainerId =
+        ContainerId.newInstance(appAttemptId, 3);
+    Token runningContainerToken =
+        BuilderUtils.newContainerToken(runningContainerId, "anyHost",
+          1234, "anyUser", BuilderUtils.newResource(1024, 1), 0, 123,
+          "password".getBytes(), 0);
+    Container runningContainer =
+        new ContainerImpl(conf, null, null, null, null, null,
+          BuilderUtils.newContainerTokenIdentifier(runningContainerToken)) {
+          @Override
+          public ContainerState getCurrentState() {
+            return ContainerState.RUNNING;
+          }
+
+          @Override
+          public org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState getContainerState() {
+            return org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState.RUNNING;
+          }
+        };
+
     nm.getNMContext().getApplications().putIfAbsent(appId,
         mock(Application.class));
     nm.getNMContext().getContainers().put(cId, anyCompletedContainer);
-    Assert.assertEquals(1, nodeStatusUpdater.getContainerStatuses().size());
+    nm.getNMContext().getContainers()
+      .put(runningContainerId, runningContainer);
+
+    Assert.assertEquals(2, nodeStatusUpdater.getContainerStatuses().size());
 
     List<ContainerId> ackedContainers = new ArrayList<ContainerId>();
     ackedContainers.add(cId);
+    ackedContainers.add(runningContainerId);
 
-    nodeStatusUpdater.removeCompletedContainersFromContext(ackedContainers);
-    Assert.assertTrue(nodeStatusUpdater.getContainerStatuses().isEmpty());
+    nodeStatusUpdater.removeOrTrackCompletedContainersFromContext(ackedContainers);
+
+    Set<ContainerId> containerIdSet = new HashSet<ContainerId>();
+    for (ContainerStatus status : nodeStatusUpdater.getContainerStatuses()) {
+      containerIdSet.add(status.getContainerId());
+    }
+
+    Assert.assertTrue(nodeStatusUpdater.getContainerStatuses().size() == 1);
+    // completed container is removed;
+    Assert.assertFalse(containerIdSet.contains(anyCompletedContainer));
+    // running container is not removed;
+    Assert.assertTrue(containerIdSet.contains(runningContainerId));
   }
 
   @Test
@@ -1467,6 +1508,13 @@ public class TestNodeStatusUpdater {
     when(container.getCurrentState()).thenReturn(containerStatus.getState());
     when(container.getContainerId()).thenReturn(
       containerStatus.getContainerId());
+    if (containerStatus.getState().equals(ContainerState.COMPLETE)) {
+      when(container.getContainerState())
+        .thenReturn(org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState.DONE);
+    } else if (containerStatus.getState().equals(ContainerState.RUNNING)) {
+      when(container.getContainerState())
+      .thenReturn(org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState.RUNNING);
+    }
     return container;
   }
 


[09/30] hadoop git commit: Revert "HADOOP-11274. ConcurrentModificationException in Configuration Copy Constructor. Contributed by Junping Du."

Posted by vi...@apache.org.
Revert "HADOOP-11274. ConcurrentModificationException in Configuration Copy Constructor. Contributed by Junping Du."

This reverts commit 16b34824673f5a50d464727b8fad98470e5e984a.


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

Branch: refs/heads/HDFS-EC
Commit: ba0a42cf678f1bb53c2ad22e766b3b1a07b3800b
Parents: ae71a67
Author: Jing Zhao <ji...@apache.org>
Authored: Thu Nov 6 18:27:59 2014 -0800
Committer: Jing Zhao <ji...@apache.org>
Committed: Thu Nov 6 18:27:59 2014 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 -
 .../org/apache/hadoop/conf/Configuration.java   | 85 +++++++++-----------
 2 files changed, 40 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ba0a42cf/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 679b5bb..8587f12 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -910,9 +910,6 @@ Release 2.6.0 - UNRELEASED
     HADOOP-11253. Hadoop streaming test TestStreamXmlMultipleRecords fails on
     Windows. (Varun Vasudev via wheat9)
 
-    HADOOP-11274. ConcurrentModificationException in Configuration Copy Constructor.
-    (Junping Du via jing9)
-
     BREAKDOWN OF HDFS-6134 AND HADOOP-10150 SUBTASKS AND RELATED JIRAS
   
       HADOOP-10734. Implement high-performance secure random number sources.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ba0a42cf/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
index 16d5499..a3fae19 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
@@ -690,26 +690,26 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
    */
   @SuppressWarnings("unchecked")
   public Configuration(Configuration other) {
-    synchronized(other) {
-      this.resources = (ArrayList<Resource>) other.resources.clone();
-      if (other.properties != null) {
-        this.properties = (Properties)other.properties.clone();
-      }
-
-      if (other.overlay!=null) {
-        this.overlay = (Properties)other.overlay.clone();
-      }
-
-      this.updatingResource = new HashMap<String, String[]>(other.updatingResource);
-      this.finalParameters = new HashSet<String>(other.finalParameters);
-
-      this.classLoader = other.classLoader;
-      this.loadDefaults = other.loadDefaults;
-      setQuietMode(other.getQuietMode());
-    }
+   this.resources = (ArrayList<Resource>) other.resources.clone();
+   synchronized(other) {
+     if (other.properties != null) {
+       this.properties = (Properties)other.properties.clone();
+     }
+
+     if (other.overlay!=null) {
+       this.overlay = (Properties)other.overlay.clone();
+     }
+
+     this.updatingResource = new HashMap<String, String[]>(other.updatingResource);
+     this.finalParameters = new HashSet<String>(other.finalParameters);
+   }
+   
     synchronized(Configuration.class) {
       REGISTRY.put(this, null);
     }
+    this.classLoader = other.classLoader;
+    this.loadDefaults = other.loadDefaults;
+    setQuietMode(other.getQuietMode());
   }
   
   /**
@@ -1025,28 +1025,26 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     getProps().setProperty(name, value);
     String newSource = (source == null ? "programmatically" : source);
 
-    synchronized (this) {
-      if (!isDeprecated(name)) {
-        updatingResource.put(name, new String[] {newSource});
-        String[] altNames = getAlternativeNames(name);
-        if(altNames != null) {
-          for(String n: altNames) {
-            if(!n.equals(name)) {
-              getOverlay().setProperty(n, value);
-              getProps().setProperty(n, value);
-              updatingResource.put(n, new String[] {newSource});
-            }
+    if (!isDeprecated(name)) {
+      updatingResource.put(name, new String[] {newSource});
+      String[] altNames = getAlternativeNames(name);
+      if(altNames != null) {
+        for(String n: altNames) {
+          if(!n.equals(name)) {
+            getOverlay().setProperty(n, value);
+            getProps().setProperty(n, value);
+            updatingResource.put(n, new String[] {newSource});
           }
         }
       }
-      else {
-        String[] names = handleDeprecation(deprecationContext.get(), name);
-        String altSource = "because " + name + " is deprecated";
-        for(String n : names) {
-          getOverlay().setProperty(n, value);
-          getProps().setProperty(n, value);
-          updatingResource.put(n, new String[] {altSource});
-        }
+    }
+    else {
+      String[] names = handleDeprecation(deprecationContext.get(), name);
+      String altSource = "because " + name + " is deprecated";
+      for(String n : names) {
+        getOverlay().setProperty(n, value);
+        getProps().setProperty(n, value);
+        updatingResource.put(n, new String[] {altSource});
       }
     }
   }
@@ -2279,7 +2277,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
    *
    * @return final parameter set.
    */
-  public synchronized Set<String> getFinalParameters() {
+  public Set<String> getFinalParameters() {
     return new HashSet<String>(finalParameters);
   }
 
@@ -2542,18 +2540,14 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     if (value != null) {
       if (!finalParameters.contains(attr)) {
         properties.setProperty(attr, value);
-        synchronized(this) {
-          updatingResource.put(attr, source);
-        }
+        updatingResource.put(attr, source);
       } else if (!value.equals(properties.getProperty(attr))) {
         LOG.warn(name+":an attempt to override final parameter: "+attr
             +";  Ignoring.");
       }
     }
     if (finalParameter) {
-      synchronized(this) {
-        finalParameters.add(attr);
-      }
+      finalParameters.add(attr);
     }
   }
 
@@ -2747,7 +2741,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
   }
 
   @Override
-  public synchronized void readFields(DataInput in) throws IOException {
+  public void readFields(DataInput in) throws IOException {
     clear();
     int size = WritableUtils.readVInt(in);
     for(int i=0; i < size; ++i) {
@@ -2759,8 +2753,9 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     }
   }
 
+  //@Override
   @Override
-  public synchronized void write(DataOutput out) throws IOException {
+  public void write(DataOutput out) throws IOException {
     Properties props = getProps();
     WritableUtils.writeVInt(out, props.size());
     for(Map.Entry<Object, Object> item: props.entrySet()) {


[19/30] hadoop git commit: HDFS-7379. TestBalancer#testBalancerWithRamDisk creates test files incorrectly. Contributed by Xiaoyu Yao.

Posted by vi...@apache.org.
HDFS-7379. TestBalancer#testBalancerWithRamDisk creates test files incorrectly. 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/57760c06
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/57760c06
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/57760c06

Branch: refs/heads/HDFS-EC
Commit: 57760c0663288a7611c3609891ef92f1abf4bb53
Parents: 06b7979
Author: Haohui Mai <wh...@apache.org>
Authored: Fri Nov 7 13:39:09 2014 -0800
Committer: Haohui Mai <wh...@apache.org>
Committed: Fri Nov 7 13:39:09 2014 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                       | 3 +++
 .../java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/57760c06/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 8735d61..0595764 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1043,6 +1043,9 @@ Release 2.6.0 - UNRELEASED
     HDFS-7364. Balancer always shows zero Bytes Already Moved.
     (Tsz Wo Nicholas Sze via jing9)
 
+    HDFS-7379. TestBalancer#testBalancerWithRamDisk creates test files
+    incorrectly. (Xiaoyu Yao via wheat9)
+
     BREAKDOWN OF HDFS-6134 AND HADOOP-10150 SUBTASKS AND RELATED JIRAS
   
       HDFS-6387. HDFS CLI admin tool for creating & deleting an

http://git-wip-us.apache.org/repos/asf/hadoop/blob/57760c06/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 698aa69..6955fcd 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
@@ -1213,7 +1213,7 @@ public class TestBalancer {
       DFSTestUtil.createFile(fs, path1, true,
         DEFAULT_RAM_DISK_BLOCK_SIZE, 4 * DEFAULT_RAM_DISK_BLOCK_SIZE,
         DEFAULT_RAM_DISK_BLOCK_SIZE, REPL_FACT, SEED, true);
-      DFSTestUtil.createFile(fs, path1, true,
+      DFSTestUtil.createFile(fs, path2, true,
         DEFAULT_RAM_DISK_BLOCK_SIZE, 1 * DEFAULT_RAM_DISK_BLOCK_SIZE,
         DEFAULT_RAM_DISK_BLOCK_SIZE, REPL_FACT, SEED, true);
 


[10/30] hadoop git commit: YARN-2647. Added a queue CLI for getting queue information. Contributed by Sunil Govind.

Posted by vi...@apache.org.
YARN-2647. Added a queue CLI for getting queue information. Contributed by Sunil Govind.


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

Branch: refs/heads/HDFS-EC
Commit: 61effcbebc819ee1bd3d60f5444c39c972494a32
Parents: ba0a42c
Author: Vinod Kumar Vavilapalli <vi...@apache.org>
Authored: Thu Nov 6 20:48:31 2014 -0800
Committer: Vinod Kumar Vavilapalli <vi...@apache.org>
Committed: Thu Nov 6 20:48:31 2014 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 hadoop-yarn-project/hadoop-yarn/bin/yarn        |   6 +
 hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd    |   6 +
 .../apache/hadoop/yarn/client/cli/QueueCLI.java | 148 +++++++++++++++++++
 .../hadoop/yarn/client/cli/TestYarnCLI.java     |  61 ++++++++
 5 files changed, 224 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/61effcbe/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index f6b39e3..9dffb22 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -192,6 +192,9 @@ Release 2.6.0 - UNRELEASED
     YARN-2198. Remove the need to run NodeManager as privileged account for
     Windows Secure Container Executor. (Remus Rusanu via jianhe)
 
+    YARN-2647. Added a queue CLI for getting queue information. (Sunil Govind via
+    vinodkv)
+
   IMPROVEMENTS
 
     YARN-2197. Add a link to YARN CHANGES.txt in the left side of doc

http://git-wip-us.apache.org/repos/asf/hadoop/blob/61effcbe/hadoop-yarn-project/hadoop-yarn/bin/yarn
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/bin/yarn b/hadoop-yarn-project/hadoop-yarn/bin/yarn
index bfcb190..0d83674 100644
--- a/hadoop-yarn-project/hadoop-yarn/bin/yarn
+++ b/hadoop-yarn-project/hadoop-yarn/bin/yarn
@@ -30,6 +30,7 @@ function hadoop_usage
   echo "  node                                  prints node report(s)"
   echo "  nodemanager                           run a nodemanager on each slave"
   echo "  proxyserver                           run the web app proxy server"
+  echo "  queue                                 prints queue information"
   echo "  resourcemanager                       run the ResourceManager"
   echo "  resourcemanager -format-state-store   deletes the RMStateStore"
   echo "  rmadmin                               admin tools"
@@ -127,6 +128,11 @@ case "${COMMAND}" in
       JAVA_HEAP_MAX="-Xmx${YARN_PROXYSERVER_HEAPSIZE}m"
     fi
   ;;
+  queue)
+    CLASS=org.apache.hadoop.yarn.client.cli.QueueCLI
+    hadoop_debug "Append YARN_CLIENT_OPTS onto YARN_OPTS"
+    YARN_OPTS="${YARN_OPTS} ${YARN_CLIENT_OPTS}"
+  ;;
   resourcemanager)
     daemon="true"
     CLASS='org.apache.hadoop.yarn.server.resourcemanager.ResourceManager'

http://git-wip-us.apache.org/repos/asf/hadoop/blob/61effcbe/hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd b/hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd
index 0cf0b40..7b7fdc4 100644
--- a/hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd
+++ b/hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd
@@ -196,6 +196,11 @@ goto :eof
   set YARN_OPTS=%YARN_OPTS% %YARN_CLIENT_OPTS%
   goto :eof
 
+:queue
+  set CLASS=org.apache.hadoop.yarn.client.cli.QueueCLI
+  set YARN_OPTS=%YARN_OPTS% %YARN_CLIENT_OPTS%
+  goto :eof
+
 :resourcemanager
   set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR%\rm-config\log4j.properties
   set CLASS=org.apache.hadoop.yarn.server.resourcemanager.ResourceManager
@@ -298,6 +303,7 @@ goto :eof
   @echo   applicationattempt   prints applicationattempt(s) report
   @echo   container            prints container(s) report
   @echo   node                 prints node report(s)
+  @echo   queue                prints queue information
   @echo   logs                 dump container logs
   @echo   classpath            prints the class path needed to get the
   @echo                        Hadoop jar and the required libraries

http://git-wip-us.apache.org/repos/asf/hadoop/blob/61effcbe/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/QueueCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/QueueCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/QueueCLI.java
new file mode 100644
index 0000000..3c74f13
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/QueueCLI.java
@@ -0,0 +1,148 @@
+/**
+ * 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.client.cli;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.text.DecimalFormat;
+import java.util.Set;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.MissingArgumentException;
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.hadoop.yarn.api.records.QueueInfo;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+import com.google.common.annotations.VisibleForTesting;
+
+@Private
+@Unstable
+public class QueueCLI extends YarnCLI {
+  public static final String QUEUE = "queue";
+
+  public static void main(String[] args) throws Exception {
+    QueueCLI cli = new QueueCLI();
+    cli.setSysOutPrintStream(System.out);
+    cli.setSysErrPrintStream(System.err);
+    int res = ToolRunner.run(cli, args);
+    cli.stop();
+    System.exit(res);
+  }
+
+  @Override
+  public int run(String[] args) throws Exception {
+    Options opts = new Options();
+    int exitCode = -1;
+    if (args.length > 0) {
+      opts.addOption(STATUS_CMD, true,
+          "List queue information about given queue.");
+      opts.addOption(HELP_CMD, false, "Displays help for all commands.");
+      opts.getOption(STATUS_CMD).setArgName("Queue Name");
+    } else {
+      syserr.println("Invalid Command usage. Command must start with 'queue'");
+      return exitCode;
+    }
+
+    CommandLine cliParser = null;
+    try {
+      cliParser = new GnuParser().parse(opts, args);
+    } catch (MissingArgumentException ex) {
+      sysout.println("Missing argument for options");
+      printUsage(opts);
+      return exitCode;
+    }
+
+    if (cliParser.hasOption(STATUS_CMD)) {
+      if (args.length != 3) {
+        printUsage(opts);
+        return exitCode;
+      }
+      listQueue(cliParser.getOptionValue(STATUS_CMD));
+    } else if (cliParser.hasOption(HELP_CMD)) {
+      printUsage(opts);
+      return 0;
+    } else {
+      syserr.println("Invalid Command Usage : ");
+      printUsage(opts);
+    }
+    return 0;
+  }
+
+  /**
+   * It prints the usage of the command
+   * 
+   * @param opts
+   */
+  @VisibleForTesting
+  void printUsage(Options opts) {
+    new HelpFormatter().printHelp(QUEUE, opts);
+  }
+  
+  /**
+   * Lists the Queue Information matching the given queue name
+   * 
+   * @param queueName
+   * @throws YarnException
+   * @throws IOException
+   */
+  private void listQueue(String queueName) throws YarnException, IOException {
+    PrintWriter writer = new PrintWriter(sysout);
+
+    QueueInfo queueInfo = client.getQueueInfo(queueName);
+    writer.println("Queue Information : ");
+    printQueueInfo(writer, queueInfo);
+    writer.flush();
+  }
+
+  private void printQueueInfo(PrintWriter writer, QueueInfo queueInfo) {
+    writer.print("Queue Name : ");
+    writer.println(queueInfo.getQueueName());
+
+    writer.print("\tState : ");
+    writer.println(queueInfo.getQueueState());
+    DecimalFormat df = new DecimalFormat("#.0");
+    writer.print("\tCapacity : ");
+    writer.println(df.format(queueInfo.getCapacity() * 100) + "%");
+    writer.print("\tCurrent Capacity : ");
+    writer.println(df.format(queueInfo.getCurrentCapacity() * 100) + "%");
+    writer.print("\tMaximum Capacity : ");
+    writer.println(df.format(queueInfo.getMaximumCapacity() * 100) + "%");
+    writer.print("\tDefault Node Label expression : ");
+    if (null != queueInfo.getDefaultNodeLabelExpression()) {
+      writer.println(queueInfo.getDefaultNodeLabelExpression());
+    } else {
+      writer.println();
+    }
+
+    Set<String> nodeLabels = queueInfo.getAccessibleNodeLabels();
+    StringBuilder labelList = new StringBuilder();
+    writer.print("\tAccessible Node Labels : ");
+    for (String nodeLabel : nodeLabels) {
+      if (labelList.length() > 0) {
+        labelList.append(',');
+      }
+      labelList.append(nodeLabel);
+    }
+    writer.println(labelList.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/61effcbe/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
index 19ded9c..adb58e8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
@@ -58,6 +58,8 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.QueueInfo;
+import org.apache.hadoop.yarn.api.records.QueueState;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
@@ -1235,6 +1237,57 @@ public class TestYarnCLI {
     Assert.assertEquals(String.format("Missing argument for options%n%1s",
         createNodeCLIHelpMessage()), sysOutStream.toString());
   }
+  
+  @Test
+  public void testGetQueueInfo() throws Exception {
+    QueueCLI cli = createAndGetQueueCLI();
+    Set<String> nodeLabels = new HashSet<String>();
+    nodeLabels.add("GPU");
+    nodeLabels.add("JDK_7");
+    QueueInfo queueInfo = QueueInfo.newInstance("queueA", 0.4f, 0.8f, 0.5f,
+        null, null, QueueState.RUNNING, nodeLabels, "GPU");
+    when(client.getQueueInfo(any(String.class))).thenReturn(queueInfo);
+    int result = cli.run(new String[] { "queue", "-status", "queueA" });
+    assertEquals(0, result);
+    verify(client).getQueueInfo("queueA");
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    PrintWriter pw = new PrintWriter(baos);
+    pw.println("Queue Information : ");
+    pw.println("Queue Name : " + "queueA");
+    pw.println("\tState : " + "RUNNING");
+    pw.println("\tCapacity : " + "40.0%");
+    pw.println("\tCurrent Capacity : " + "50.0%");
+    pw.println("\tMaximum Capacity : " + "80.0%");
+    pw.println("\tDefault Node Label expression : " + "GPU");
+    pw.println("\tAccessible Node Labels : " + "JDK_7,GPU");
+    pw.close();
+    String queueInfoStr = baos.toString("UTF-8");
+    Assert.assertEquals(queueInfoStr, sysOutStream.toString());
+  }
+  
+  @Test
+  public void testGetQueueInfoWithEmptyNodeLabel() throws Exception {
+    QueueCLI cli = createAndGetQueueCLI();
+    QueueInfo queueInfo = QueueInfo.newInstance("queueA", 0.4f, 0.8f, 0.5f,
+        null, null, QueueState.RUNNING, null, null);
+    when(client.getQueueInfo(any(String.class))).thenReturn(queueInfo);
+    int result = cli.run(new String[] { "queue", "-status", "queueA" });
+    assertEquals(0, result);
+    verify(client).getQueueInfo("queueA");
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    PrintWriter pw = new PrintWriter(baos);
+    pw.println("Queue Information : ");
+    pw.println("Queue Name : " + "queueA");
+    pw.println("\tState : " + "RUNNING");
+    pw.println("\tCapacity : " + "40.0%");
+    pw.println("\tCurrent Capacity : " + "50.0%");
+    pw.println("\tMaximum Capacity : " + "80.0%");
+    pw.println("\tDefault Node Label expression : ");
+    pw.println("\tAccessible Node Labels : ");
+    pw.close();
+    String queueInfoStr = baos.toString("UTF-8");
+    Assert.assertEquals(queueInfoStr, sysOutStream.toString());
+  }
 
   private void verifyUsageInfo(YarnCLI cli) throws Exception {
     cli.setSysErrPrintStream(sysErr);
@@ -1272,6 +1325,14 @@ public class TestYarnCLI {
     cli.setSysOutPrintStream(sysOut);
     return cli;
   }
+  
+  private QueueCLI createAndGetQueueCLI() {
+    QueueCLI cli = new QueueCLI();
+    cli.setClient(client);
+    cli.setSysOutPrintStream(sysOut);
+    cli.setSysErrPrintStream(sysErr);
+    return cli;
+  }
 
   private String createApplicationCLIHelpMessage() throws IOException {
     ByteArrayOutputStream baos = new ByteArrayOutputStream();


[11/30] hadoop git commit: HDFS-7221. Update CHANGES.txt to indicate fix in 2.6.0.

Posted by vi...@apache.org.
HDFS-7221. Update CHANGES.txt to indicate fix in 2.6.0.


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

Branch: refs/heads/HDFS-EC
Commit: e7f1c0482e5dff8a1549ace1fc2b366941170c58
Parents: 61effcb
Author: cnauroth <cn...@apache.org>
Authored: Thu Nov 6 22:46:22 2014 -0800
Committer: cnauroth <cn...@apache.org>
Committed: Thu Nov 6 22:46:22 2014 -0800

----------------------------------------------------------------------
 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/e7f1c048/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 5a7dac0..0e64d69 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -287,8 +287,6 @@ Release 2.7.0 - UNRELEASED
     HDFS-7266. HDFS Peercache enabled check should not lock on object (awang
     via cmccabe)
 
-    HDFS-7221. TestDNFencingWithReplication fails consistently. (Charles Lamb via wang)
-
     HDFS-7254. Add documentation for hot swaping DataNode drives (Lei Xu via
     Colin P. McCabe)
 
@@ -719,6 +717,8 @@ Release 2.6.0 - UNRELEASED
     HDFS-7233. NN logs unnecessary org.apache.hadoop.hdfs.protocol.UnresolvedPathException.
     (Rushabh S Shah via jing9)
 
+    HDFS-7221. TestDNFencingWithReplication fails consistently. (Charles Lamb via wang)
+
   OPTIMIZATIONS
 
     HDFS-6690. Deduplicate xattr names in memory. (wang)


[16/30] hadoop git commit: YARN-2824. Fixed Capacity Scheduler to not crash when some node-labels are not mapped to queues by making default capacities per label to be zero. Contributed by Wangda Tan.

Posted by vi...@apache.org.
YARN-2824. Fixed Capacity Scheduler to not crash when some node-labels are not mapped to queues by making default capacities per label to be zero. 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/2ac1be7d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2ac1be7d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2ac1be7d

Branch: refs/heads/HDFS-EC
Commit: 2ac1be7dec4aef001e3162e364249933b2c4a6c4
Parents: 1e97f2f
Author: Vinod Kumar Vavilapalli <vi...@apache.org>
Authored: Fri Nov 7 10:39:37 2014 -0800
Committer: Vinod Kumar Vavilapalli <vi...@apache.org>
Committed: Fri Nov 7 10:39:37 2014 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  4 +
 .../dev-support/findbugs-exclude.xml            |  1 +
 .../distributedshell/TestDistributedShell.java  |  2 +
 .../scheduler/capacity/AbstractCSQueue.java     | 28 ++++---
 .../CapacitySchedulerConfiguration.java         | 17 +---
 .../capacity/TestContainerAllocation.java       |  5 ++
 .../scheduler/capacity/TestQueueParsing.java    | 84 ++++++++++++++++++--
 7 files changed, 107 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ac1be7d/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 111aaaa..c48fb4f 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -894,6 +894,10 @@ Release 2.6.0 - UNRELEASED
 
     YARN-2810. TestRMProxyUsersConf fails on Windows VMs. (Varun Vasudev via xgong)
 
+    YARN-2824. Fixed Capacity Scheduler to not crash when some node-labels are
+    not mapped to queues by making default capacities per label to be zero.
+    (Wangda Tan via vinodkv)
+
 Release 2.5.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ac1be7d/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
index e6da24c..45d7294 100644
--- a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
@@ -202,6 +202,7 @@
       <Field name="accessibleLabels" />
       <Field name="absoluteNodeLabelCapacities" />
       <Field name="reservationsContinueLooking" />
+      <Field name="absoluteCapacityByNodeLabels" />
     </Or>
     <Bug pattern="IS2_INCONSISTENT_SYNC" />
   </Match>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ac1be7d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
index 0ded5bd..eb0fb94 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
@@ -86,6 +86,8 @@ public class TestDistributedShell {
 
     // Setup queue access to node labels
     conf.set("yarn.scheduler.capacity.root.accessible-node-labels", "x");
+    conf.set("yarn.scheduler.capacity.root.accessible-node-labels.x.capacity",
+        "100");
     conf.set("yarn.scheduler.capacity.root.default.accessible-node-labels", "x");
     conf.set(
         "yarn.scheduler.capacity.root.default.accessible-node-labels.x.capacity",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ac1be7d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.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/AbstractCSQueue.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/AbstractCSQueue.java
index c612846..fc0fbb4 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/AbstractCSQueue.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/AbstractCSQueue.java
@@ -395,16 +395,15 @@ public abstract class AbstractCSQueue implements CSQueue {
   
   @Private
   public float getCapacityByNodeLabel(String label) {
-    if (null == parent) {
-      return 1f;
-    }
-    
     if (StringUtils.equals(label, RMNodeLabelsManager.NO_LABEL)) {
+      if (null == parent) {
+        return 1f;
+      }
       return getCapacity();
     }
     
     if (!capacitiyByNodeLabels.containsKey(label)) {
-      return 0;
+      return 0f;
     } else {
       return capacitiyByNodeLabels.get(label);
     }
@@ -412,18 +411,17 @@ public abstract class AbstractCSQueue implements CSQueue {
   
   @Private
   public float getAbsoluteCapacityByNodeLabel(String label) {
-    if (null == parent) {
-      return 1; 
-    }
-    
     if (StringUtils.equals(label, RMNodeLabelsManager.NO_LABEL)) {
+      if (null == parent) {
+        return 1f; 
+      }
       return getAbsoluteCapacity();
     }
     
-    if (!absoluteMaxCapacityByNodeLabels.containsKey(label)) {
-      return 0;
+    if (!absoluteCapacityByNodeLabels.containsKey(label)) {
+      return 0f;
     } else {
-      return absoluteMaxCapacityByNodeLabels.get(label);
+      return absoluteCapacityByNodeLabels.get(label);
     }
   }
   
@@ -433,7 +431,11 @@ public abstract class AbstractCSQueue implements CSQueue {
       return getAbsoluteMaximumCapacity();
     }
     
-    return getAbsoluteCapacityByNodeLabel(label);
+    if (!absoluteMaxCapacityByNodeLabels.containsKey(label)) {
+      return 0f;
+    } else {
+      return absoluteMaxCapacityByNodeLabels.get(label);
+    }
   }
   
   @Private

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ac1be7d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.java
index b36172c..23bf381 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/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.java
@@ -461,19 +461,8 @@ public class CapacitySchedulerConfiguration extends Configuration {
 
     for (String label : labels.contains(CommonNodeLabelsManager.ANY) ? mgr
         .getClusterNodeLabels() : labels) {
-      // capacity of all labels in each queue should be 1
-      if (org.apache.commons.lang.StringUtils.equals(ROOT, queue)) {
-        nodeLabelCapacities.put(label, 1.0f);
-        continue;
-      }
       String capacityPropertyName = getNodeLabelPrefix(queue, label) + CAPACITY;
-      float capacity = getFloat(capacityPropertyName, UNDEFINED);
-      if (capacity == UNDEFINED) {
-        throw new IllegalArgumentException("Configuration issue: "
-            + " node-label=" + label + " is accessible from queue=" + queue
-            + " but has no capacity set, you should set " 
-            + capacityPropertyName + " in range of [0, 100].");
-      }
+      float capacity = getFloat(capacityPropertyName, 0f);
       if (capacity < MINIMUM_CAPACITY_VALUE
           || capacity > MAXIMUM_CAPACITY_VALUE) {
         throw new IllegalArgumentException("Illegal capacity of " + capacity
@@ -501,9 +490,7 @@ public class CapacitySchedulerConfiguration extends Configuration {
         .getClusterNodeLabels() : labels) {
       float maxCapacity =
           getFloat(getNodeLabelPrefix(queue, label) + MAXIMUM_CAPACITY,
-              UNDEFINED);
-      maxCapacity = (maxCapacity == DEFAULT_MAXIMUM_CAPACITY_VALUE) ? 
-          MAXIMUM_CAPACITY_VALUE : maxCapacity;
+              100f);
       if (maxCapacity < MINIMUM_CAPACITY_VALUE
           || maxCapacity > MAXIMUM_CAPACITY_VALUE) {
         throw new IllegalArgumentException("Illegal " + "capacity of "

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ac1be7d/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 b90df8e..0c32c0c 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
@@ -340,6 +340,8 @@ public class TestContainerAllocation {
     
     // Define top-level queues
     conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b", "c"});
+    conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100);
+    conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "y", 100);
 
     final String A = CapacitySchedulerConfiguration.ROOT + ".a";
     conf.setCapacity(A, 10);
@@ -403,6 +405,9 @@ public class TestContainerAllocation {
     
     // Define top-level queues
     conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"});
+    conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100);
+    conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "y", 100);
+    conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "z", 100);
 
     final String A = CapacitySchedulerConfiguration.ROOT + ".a";
     conf.setCapacity(A, 10);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ac1be7d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueParsing.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/TestQueueParsing.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/TestQueueParsing.java
index 42db030..cf2e5ce 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/TestQueueParsing.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/TestQueueParsing.java
@@ -18,10 +18,6 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
 import java.io.IOException;
 
 import org.apache.commons.logging.Log;
@@ -51,8 +47,9 @@ public class TestQueueParsing {
   
   @Before
   public void setup() {
-    nodeLabelManager = mock(RMNodeLabelsManager.class);
-    when(nodeLabelManager.containsNodeLabel(any(String.class))).thenReturn(true);
+    nodeLabelManager = new MemoryRMNodeLabelsManager();
+    nodeLabelManager.init(new YarnConfiguration());
+    nodeLabelManager.start();
   }
   
   @Test
@@ -255,6 +252,8 @@ public class TestQueueParsing {
   private void setupQueueConfigurationWithLabels(CapacitySchedulerConfiguration conf) {
     // Define top-level queues
     conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"});
+    conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "red", 100);
+    conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "blue", 100);
 
     final String A = CapacitySchedulerConfiguration.ROOT + ".a";
     conf.setCapacity(A, 10);
@@ -271,6 +270,7 @@ public class TestQueueParsing {
     conf.setQueues(A, new String[] {"a1", "a2"});
     conf.setAccessibleNodeLabels(A, ImmutableSet.of("red", "blue"));
     conf.setCapacityByLabel(A, "red", 50);
+    conf.setMaximumCapacityByLabel(A, "red", 50);
     conf.setCapacityByLabel(A, "blue", 50);
     
     conf.setCapacity(A1, 30);
@@ -282,6 +282,7 @@ public class TestQueueParsing {
     conf.setMaximumCapacity(A2, 85);
     conf.setAccessibleNodeLabels(A2, ImmutableSet.of("red"));
     conf.setCapacityByLabel(A2, "red", 50);
+    conf.setMaximumCapacityByLabel(A2, "red", 60);
     
     final String B1 = B + ".b1";
     final String B2 = B + ".b2";
@@ -311,6 +312,8 @@ public class TestQueueParsing {
       CapacitySchedulerConfiguration conf) {
     // Define top-level queues
     conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"});
+    conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "red", 100);
+    conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "blue", 100);
 
     // Set A configuration
     final String A = CapacitySchedulerConfiguration.ROOT + ".a";
@@ -364,6 +367,7 @@ public class TestQueueParsing {
   
   @Test
   public void testQueueParsingReinitializeWithLabels() throws IOException {
+    nodeLabelManager.addToCluserNodeLabels(ImmutableSet.of("red", "blue"));
     CapacitySchedulerConfiguration csConf =
         new CapacitySchedulerConfiguration();
     setupQueueConfigurationWithoutLabels(csConf);
@@ -410,6 +414,22 @@ public class TestQueueParsing {
     // queue-B2 inherits "red"/"blue"
     Assert.assertTrue(capacityScheduler.getQueue("b2")
         .getAccessibleNodeLabels().containsAll(ImmutableSet.of("red", "blue")));
+    
+    // check capacity of A2
+    CSQueue qA2 = capacityScheduler.getQueue("a2");
+    Assert.assertEquals(0.7, qA2.getCapacity(), DELTA);
+    Assert.assertEquals(0.5, qA2.getCapacityByNodeLabel("red"), DELTA);
+    Assert.assertEquals(0.07, qA2.getAbsoluteCapacity(), DELTA);
+    Assert.assertEquals(0.25, qA2.getAbsoluteCapacityByNodeLabel("red"), DELTA);
+    Assert.assertEquals(0.1275, qA2.getAbsoluteMaximumCapacity(), DELTA);
+    Assert.assertEquals(0.3, qA2.getAbsoluteMaximumCapacityByNodeLabel("red"), DELTA);
+    
+    // check capacity of B3
+    CSQueue qB3 = capacityScheduler.getQueue("b3");
+    Assert.assertEquals(0.18, qB3.getAbsoluteCapacity(), DELTA);
+    Assert.assertEquals(0.125, qB3.getAbsoluteCapacityByNodeLabel("red"), DELTA);
+    Assert.assertEquals(0.35, qB3.getAbsoluteMaximumCapacity(), DELTA);
+    Assert.assertEquals(1, qB3.getAbsoluteMaximumCapacityByNodeLabel("red"), DELTA);
   }
   
   private void
@@ -435,6 +455,8 @@ public class TestQueueParsing {
   
   @Test
   public void testQueueParsingWithLabels() throws IOException {
+    nodeLabelManager.addToCluserNodeLabels(ImmutableSet.of("red", "blue"));
+    
     YarnConfiguration conf = new YarnConfiguration();
     CapacitySchedulerConfiguration csConf =
         new CapacitySchedulerConfiguration(conf);
@@ -457,6 +479,8 @@ public class TestQueueParsing {
   
   @Test
   public void testQueueParsingWithLabelsInherit() throws IOException {
+    nodeLabelManager.addToCluserNodeLabels(ImmutableSet.of("red", "blue"));
+
     YarnConfiguration conf = new YarnConfiguration();
     CapacitySchedulerConfiguration csConf =
         new CapacitySchedulerConfiguration(conf);
@@ -587,4 +611,52 @@ public class TestQueueParsing {
     ServiceOperations.stopQuietly(capacityScheduler);
     ServiceOperations.stopQuietly(nodeLabelsManager);
   }
+  
+  @Test
+  public void testQueueParsingWithUnusedLabels() throws IOException {
+    final ImmutableSet<String> labels = ImmutableSet.of("red", "blue");
+    
+    // Initialize a cluster with labels, but doesn't use them, reinitialize
+    // shouldn't fail
+    nodeLabelManager.addToCluserNodeLabels(labels);
+
+    CapacitySchedulerConfiguration csConf =
+        new CapacitySchedulerConfiguration();
+    setupQueueConfiguration(csConf);
+    csConf.setAccessibleNodeLabels(CapacitySchedulerConfiguration.ROOT, labels);
+    YarnConfiguration conf = new YarnConfiguration(csConf);
+
+    CapacityScheduler capacityScheduler = new CapacityScheduler();
+    capacityScheduler.setConf(conf);
+    RMContextImpl rmContext =
+        new RMContextImpl(null, null, null, null, null, null,
+            new RMContainerTokenSecretManager(csConf),
+            new NMTokenSecretManagerInRM(csConf),
+            new ClientToAMTokenSecretManagerInRM(), null);
+    rmContext.setNodeLabelManager(nodeLabelManager);
+    capacityScheduler.setRMContext(rmContext);
+    capacityScheduler.init(conf);
+    capacityScheduler.start();
+    capacityScheduler.reinitialize(conf, rmContext);
+    
+    // check root queue's capacity by label -- they should be all zero
+    CSQueue root = capacityScheduler.getQueue(CapacitySchedulerConfiguration.ROOT);
+    Assert.assertEquals(0, root.getCapacityByNodeLabel("red"), DELTA);
+    Assert.assertEquals(0, root.getCapacityByNodeLabel("blue"), DELTA);
+
+    CSQueue a = capacityScheduler.getQueue("a");
+    Assert.assertEquals(0.10, a.getAbsoluteCapacity(), DELTA);
+    Assert.assertEquals(0.15, a.getAbsoluteMaximumCapacity(), DELTA);
+
+    CSQueue b1 = capacityScheduler.getQueue("b1");
+    Assert.assertEquals(0.2 * 0.5, b1.getAbsoluteCapacity(), DELTA);
+    Assert.assertEquals("Parent B has no MAX_CAP", 0.85,
+        b1.getAbsoluteMaximumCapacity(), DELTA);
+
+    CSQueue c12 = capacityScheduler.getQueue("c12");
+    Assert.assertEquals(0.7 * 0.5 * 0.45, c12.getAbsoluteCapacity(), DELTA);
+    Assert.assertEquals(0.7 * 0.55 * 0.7, c12.getAbsoluteMaximumCapacity(),
+        DELTA);
+    capacityScheduler.stop();
+  }
 }


[28/30] hadoop git commit: HADOOP-11286. Copied LimitInputStream from guava-0.14 to hadoop to avoid issues with newer versions of guava in applications. Contributed by Christopher Tubbs.

Posted by vi...@apache.org.
HADOOP-11286. Copied LimitInputStream from guava-0.14 to hadoop to avoid issues with newer versions of guava in applications. Contributed by Christopher Tubbs.


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

Branch: refs/heads/HDFS-EC
Commit: 6caa8100d5d2547e34356dc279fd5e65b81a925a
Parents: 737d928
Author: Arun C. Murthy <ac...@apache.org>
Authored: Sat Nov 8 15:39:56 2014 -0800
Committer: Arun C. Murthy <ac...@apache.org>
Committed: Sat Nov 8 15:39:56 2014 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |   4 +
 .../apache/hadoop/util/LimitInputStream.java    | 109 +++++++++++++++++++
 .../server/namenode/FSImageFormatProtobuf.java  |   2 +-
 .../tools/offlineImageViewer/FSImageLoader.java |   2 +-
 .../FileDistributionCalculator.java             |   2 +-
 .../offlineImageViewer/PBImageXmlWriter.java    |   2 +-
 .../apache/hadoop/mapreduce/CryptoUtils.java    |   3 +-
 7 files changed, 118 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6caa8100/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 550b009..7ecee21 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -656,6 +656,10 @@ Release 2.6.0 - UNRELEASED
     HADOOP-11254. Changed visibility of AccessControlList to be public for
     consumption by ecosystem. (Zhijie Shen via vinodkv)
 
+    HADOOP-11286. Copied LimitInputStream from guava-0.14 to hadoop to avoid
+    issues with newer versions of guava in applications. (Christopher Tubbs
+    via acmurthy)
+
   OPTIMIZATIONS
 
     HADOOP-10838. Byte array native checksumming. (James Thomas via todd)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6caa8100/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LimitInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LimitInputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LimitInputStream.java
new file mode 100644
index 0000000..c94a517
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LimitInputStream.java
@@ -0,0 +1,109 @@
+/*
+ *
+ * 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.util;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import java.io.FilterInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+/**
+ * Copied from guava source code v15 (LimitedInputStream)
+ * Guava deprecated LimitInputStream in v14 and removed it in v15. Copying this class here
+ * allows to be compatible with guava 11 to 15+.
+ * 
+ * Originally: org.apache.hadoop.hbase.io.LimitInputStream
+ */
+@Unstable
+public final class LimitInputStream extends FilterInputStream {
+  private long left;
+  private long mark = -1;
+
+  public LimitInputStream(InputStream in, long limit) {
+    super(in);
+    checkNotNull(in);
+    checkArgument(limit >= 0, "limit must be non-negative");
+    left = limit;
+  }
+
+  @Override
+  public int available() throws IOException {
+    return (int) Math.min(in.available(), left);
+  }
+
+  // it's okay to mark even if mark isn't supported, as reset won't work
+  @Override
+  public synchronized void mark(int readLimit) {
+    in.mark(readLimit);
+    mark = left;
+  }
+
+  @Override
+  public int read() throws IOException {
+    if (left == 0) {
+      return -1;
+    }
+
+    int result = in.read();
+    if (result != -1) {
+      --left;
+    }
+    return result;
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    if (left == 0) {
+      return -1;
+    }
+
+    len = (int) Math.min(len, left);
+    int result = in.read(b, off, len);
+    if (result != -1) {
+      left -= result;
+    }
+    return result;
+  }
+
+  @Override
+  public synchronized void reset() throws IOException {
+    if (!in.markSupported()) {
+      throw new IOException("Mark not supported");
+    }
+    if (mark == -1) {
+      throw new IOException("Mark not set");
+    }
+
+    in.reset();
+    left = mark;
+  }
+
+  @Override
+  public long skip(long n) throws IOException {
+    n = Math.min(n, left);
+    long skipped = in.skip(n);
+    left -= skipped;
+    return skipped;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6caa8100/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
index e0d5a5f..4387cff 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
@@ -62,11 +62,11 @@ import org.apache.hadoop.hdfs.util.MD5FileUtils;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.io.compress.CompressorStream;
+import org.apache.hadoop.util.LimitInputStream;
 import org.apache.hadoop.util.Time;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
-import com.google.common.io.LimitInputStream;
 import com.google.protobuf.CodedOutputStream;
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6caa8100/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageLoader.java
index b68d842..ff665e7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageLoader.java
@@ -47,12 +47,12 @@ import org.apache.hadoop.hdfs.server.namenode.FSImageUtil;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto;
 import org.apache.hadoop.hdfs.server.namenode.INodeId;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.LimitInputStream;
 import org.codehaus.jackson.map.ObjectMapper;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
-import com.google.common.io.LimitInputStream;
 
 /**
  * FSImageLoader loads fsimage and provide methods to return JSON formatted

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6caa8100/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FileDistributionCalculator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FileDistributionCalculator.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FileDistributionCalculator.java
index c8033dd..61c3650 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FileDistributionCalculator.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FileDistributionCalculator.java
@@ -31,9 +31,9 @@ import org.apache.hadoop.hdfs.server.namenode.FSImageUtil;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.LimitInputStream;
 
 import com.google.common.base.Preconditions;
-import com.google.common.io.LimitInputStream;
 
 /**
  * This is the tool for analyzing file sizes in the namespace image. In order to

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6caa8100/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageXmlWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageXmlWriter.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageXmlWriter.java
index fa8c59d..3e3f021 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageXmlWriter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageXmlWriter.java
@@ -51,9 +51,9 @@ import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SnapshotSection;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.StringTableSection;
 import org.apache.hadoop.hdfs.util.XMLUtils;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.LimitInputStream;
 
 import com.google.common.collect.Lists;
-import com.google.common.io.LimitInputStream;
 
 /**
  * PBImageXmlWriter walks over an fsimage structure and writes out

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6caa8100/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/CryptoUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/CryptoUtils.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/CryptoUtils.java
index 7d8a496..184cdf0 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/CryptoUtils.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/CryptoUtils.java
@@ -37,8 +37,7 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.security.TokenCache;
 import org.apache.hadoop.security.UserGroupInformation;
-
-import com.google.common.io.LimitInputStream;
+import org.apache.hadoop.util.LimitInputStream;
 
 /**
  * This class provides utilities to make it easier to work with Cryptographic


[27/30] hadoop git commit: YARN-2607. Fixed issues in TestDistributedShell. Contributed by Wangda Tan.

Posted by vi...@apache.org.
YARN-2607. Fixed issues in TestDistributedShell. 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/737d9284
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/737d9284
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/737d9284

Branch: refs/heads/HDFS-EC
Commit: 737d9284c109dac20ff423f30c62f6abe2db10f7
Parents: 9a4e0d3
Author: Vinod Kumar Vavilapalli <vi...@apache.org>
Authored: Sat Nov 8 11:00:57 2014 -0800
Committer: Vinod Kumar Vavilapalli <vi...@apache.org>
Committed: Sat Nov 8 11:00:57 2014 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   2 +
 .../distributedshell/TestDistributedShell.java  | 140 +---------------
 .../TestDistributedShellWithNodeLabels.java     | 165 +++++++++++++++++++
 3 files changed, 176 insertions(+), 131 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/737d9284/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 748ffe0..9adfb8c 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -920,6 +920,8 @@ Release 2.6.0 - UNRELEASED
     YARN-2826. Fixed user-groups mappings' refresh bug caused by YARN-2826.
     (Wangda Tan via vinodkv)
 
+    YARN-2607. Fixed issues in TestDistributedShell. (Wangda Tan via vinodkv)
+
 Release 2.5.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/737d9284/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
index eb0fb94..1d3a104 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
@@ -30,9 +30,7 @@ import java.net.InetAddress;
 import java.net.URL;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.HashSet;
 import java.util.List;
-import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.commons.logging.Log;
@@ -44,70 +42,37 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.util.JarFinder;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
-import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
 import org.apache.hadoop.yarn.client.api.YarnClient;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.MiniYARNCluster;
-import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
-import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-import com.google.common.collect.ImmutableMap;
-
 public class TestDistributedShell {
 
   private static final Log LOG =
       LogFactory.getLog(TestDistributedShell.class);
 
-  protected MiniYARNCluster yarnCluster = null;
-  private int numNodeManager = 1;
-  
-  private YarnConfiguration conf = null;
+  protected MiniYARNCluster yarnCluster = null;  
+  protected YarnConfiguration conf = null;
+  private static final int NUM_NMS = 1;
 
   protected final static String APPMASTER_JAR =
       JarFinder.getJar(ApplicationMaster.class);
-  
-  private void initializeNodeLabels() throws IOException {
-    RMContext rmContext = yarnCluster.getResourceManager(0).getRMContext();
-
-    // Setup node labels
-    RMNodeLabelsManager labelsMgr = rmContext.getNodeLabelManager();
-    Set<String> labels = new HashSet<String>();
-    labels.add("x");
-    labelsMgr.addToCluserNodeLabels(labels);
-
-    // Setup queue access to node labels
-    conf.set("yarn.scheduler.capacity.root.accessible-node-labels", "x");
-    conf.set("yarn.scheduler.capacity.root.accessible-node-labels.x.capacity",
-        "100");
-    conf.set("yarn.scheduler.capacity.root.default.accessible-node-labels", "x");
-    conf.set(
-        "yarn.scheduler.capacity.root.default.accessible-node-labels.x.capacity",
-        "100");
-
-    rmContext.getScheduler().reinitialize(conf, rmContext);
-
-    // Fetch node-ids from yarn cluster
-    NodeId[] nodeIds = new NodeId[numNodeManager];
-    for (int i = 0; i < numNodeManager; i++) {
-      NodeManager mgr = this.yarnCluster.getNodeManager(i);
-      nodeIds[i] = mgr.getNMContext().getNodeId();
-    }
-
-    // Set label x to NM[1]
-    labelsMgr.addLabelsToNode(ImmutableMap.of(nodeIds[1], labels));
-  }
 
   @Before
   public void setup() throws Exception {
+    setupInternal(NUM_NMS);
+  }
+
+  protected void setupInternal(int numNodeManager) throws Exception {
+
     LOG.info("Starting up YARN cluster");
     
     conf = new YarnConfiguration();
@@ -115,7 +80,6 @@ public class TestDistributedShell {
     conf.set("yarn.log.dir", "target");
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
     conf.set(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class.getName());
-    numNodeManager = 2;
     
     if (yarnCluster == null) {
       yarnCluster =
@@ -127,9 +91,6 @@ public class TestDistributedShell {
       
       waitForNMsToRegister();
       
-      // currently only capacity scheduler support node labels,
-      initializeNodeLabels();
-      
       URL url = Thread.currentThread().getContextClassLoader().getResource("yarn-site.xml");
       if (url == null) {
         throw new RuntimeException("Could not find 'yarn-site.xml' dummy file in classpath");
@@ -807,7 +768,7 @@ public class TestDistributedShell {
     int sec = 60;
     while (sec >= 0) {
       if (yarnCluster.getResourceManager().getRMContext().getRMNodes().size() 
-          >= numNodeManager) {
+          >= NUM_NMS) {
         break;
       }
       Thread.sleep(1000);
@@ -940,88 +901,5 @@ public class TestDistributedShell {
     }
     return numOfWords;
   }
-  
-  @Test(timeout=90000)
-  public void testDSShellWithNodeLabelExpression() throws Exception {
-    // Start NMContainerMonitor
-    NMContainerMonitor mon = new NMContainerMonitor();
-    Thread t = new Thread(mon);
-    t.start();
-
-    // Submit a job which will sleep for 60 sec
-    String[] args = {
-        "--jar",
-        APPMASTER_JAR,
-        "--num_containers",
-        "4",
-        "--shell_command",
-        "sleep",
-        "--shell_args",
-        "15",
-        "--master_memory",
-        "512",
-        "--master_vcores",
-        "2",
-        "--container_memory",
-        "128",
-        "--container_vcores",
-        "1",
-        "--node_label_expression",
-        "x"
-    };
-
-    LOG.info("Initializing DS Client");
-    final Client client =
-        new Client(new Configuration(yarnCluster.getConfig()));
-    boolean initSuccess = client.init(args);
-    Assert.assertTrue(initSuccess);
-    LOG.info("Running DS Client");
-    boolean result = client.run();
-    LOG.info("Client run completed. Result=" + result);
-    
-    t.interrupt();
-    
-    // Check maximum number of containers on each NMs
-    int[] maxRunningContainersOnNMs = mon.getMaxRunningContainersReport();
-    // Check no container allocated on NM[0]
-    Assert.assertEquals(0, maxRunningContainersOnNMs[0]);
-    // Check there're some containers allocated on NM[1]
-    Assert.assertTrue(maxRunningContainersOnNMs[1] > 0);
-  }
-  
-  /**
-   * Monitor containers running on NMs
-   */
-  private class NMContainerMonitor implements Runnable {
-    // The interval of milliseconds of sampling (500ms)
-    final static int SAMPLING_INTERVAL_MS = 500;
-    
-    // The maximum number of containers running on each NMs
-    int[] maxRunningContainersOnNMs = new int[numNodeManager];
-
-    @Override
-    public void run() {
-      while (true) {
-        for (int i = 0; i < numNodeManager; i++) {
-          int nContainers =
-              yarnCluster.getNodeManager(i).getNMContext().getContainers()
-                  .size();
-          if (nContainers > maxRunningContainersOnNMs[i]) {
-            maxRunningContainersOnNMs[i] = nContainers;
-          }
-        }
-        try {
-          Thread.sleep(SAMPLING_INTERVAL_MS);
-        } catch (InterruptedException e) {
-          e.printStackTrace();
-          break;
-        }
-      }
-    }
-    
-    public int[] getMaxRunningContainersReport() {
-      return maxRunningContainersOnNMs;
-    }
-  }
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/737d9284/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShellWithNodeLabels.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShellWithNodeLabels.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShellWithNodeLabels.java
new file mode 100644
index 0000000..c04b7fe
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShellWithNodeLabels.java
@@ -0,0 +1,165 @@
+/**
+ * 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.applications.distributedshell;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+
+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.NodeId;
+import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.common.collect.ImmutableMap;
+
+public class TestDistributedShellWithNodeLabels {
+  private static final Log LOG =
+      LogFactory.getLog(TestDistributedShellWithNodeLabels.class);
+  
+  static final int NUM_NMS = 2;
+  TestDistributedShell distShellTest;
+ 
+  @Before
+  public void setup() throws Exception {
+    distShellTest = new TestDistributedShell();
+    distShellTest.setupInternal(NUM_NMS);
+  }
+  
+  private void initializeNodeLabels() throws IOException {
+    RMContext rmContext = distShellTest.yarnCluster.getResourceManager(0).getRMContext();
+
+    // Setup node labels
+    RMNodeLabelsManager labelsMgr = rmContext.getNodeLabelManager();
+    Set<String> labels = new HashSet<String>();
+    labels.add("x");
+    labelsMgr.addToCluserNodeLabels(labels);
+
+    // Setup queue access to node labels
+    distShellTest.conf.set("yarn.scheduler.capacity.root.accessible-node-labels", "x");
+    distShellTest.conf.set("yarn.scheduler.capacity.root.accessible-node-labels.x.capacity",
+        "100");
+    distShellTest.conf.set("yarn.scheduler.capacity.root.default.accessible-node-labels", "x");
+    distShellTest.conf.set(
+        "yarn.scheduler.capacity.root.default.accessible-node-labels.x.capacity",
+        "100");
+
+    rmContext.getScheduler().reinitialize(distShellTest.conf, rmContext);
+
+    // Fetch node-ids from yarn cluster
+    NodeId[] nodeIds = new NodeId[NUM_NMS];
+    for (int i = 0; i < NUM_NMS; i++) {
+      NodeManager mgr = distShellTest.yarnCluster.getNodeManager(i);
+      nodeIds[i] = mgr.getNMContext().getNodeId();
+    }
+
+    // Set label x to NM[1]
+    labelsMgr.addLabelsToNode(ImmutableMap.of(nodeIds[1], labels));
+  }
+  
+  @Test(timeout=90000)
+  public void testDSShellWithNodeLabelExpression() throws Exception {
+    initializeNodeLabels();
+    
+    // Start NMContainerMonitor
+    NMContainerMonitor mon = new NMContainerMonitor();
+    Thread t = new Thread(mon);
+    t.start();
+
+    // Submit a job which will sleep for 60 sec
+    String[] args = {
+        "--jar",
+        TestDistributedShell.APPMASTER_JAR,
+        "--num_containers",
+        "4",
+        "--shell_command",
+        "sleep",
+        "--shell_args",
+        "15",
+        "--master_memory",
+        "512",
+        "--master_vcores",
+        "2",
+        "--container_memory",
+        "128",
+        "--container_vcores",
+        "1",
+        "--node_label_expression",
+        "x"
+    };
+
+    LOG.info("Initializing DS Client");
+    final Client client =
+        new Client(new Configuration(distShellTest.yarnCluster.getConfig()));
+    boolean initSuccess = client.init(args);
+    Assert.assertTrue(initSuccess);
+    LOG.info("Running DS Client");
+    boolean result = client.run();
+    LOG.info("Client run completed. Result=" + result);
+    
+    t.interrupt();
+    
+    // Check maximum number of containers on each NMs
+    int[] maxRunningContainersOnNMs = mon.getMaxRunningContainersReport();
+    // Check no container allocated on NM[0]
+    Assert.assertEquals(0, maxRunningContainersOnNMs[0]);
+    // Check there're some containers allocated on NM[1]
+    Assert.assertTrue(maxRunningContainersOnNMs[1] > 0);
+  }
+  
+  /**
+   * Monitor containers running on NMs
+   */
+  class NMContainerMonitor implements Runnable {
+    // The interval of milliseconds of sampling (500ms)
+    final static int SAMPLING_INTERVAL_MS = 500;
+
+    // The maximum number of containers running on each NMs
+    int[] maxRunningContainersOnNMs = new int[NUM_NMS];
+
+    @Override
+    public void run() {
+      while (true) {
+        for (int i = 0; i < NUM_NMS; i++) {
+          int nContainers =
+              distShellTest.yarnCluster.getNodeManager(i).getNMContext()
+                  .getContainers().size();
+          if (nContainers > maxRunningContainersOnNMs[i]) {
+            maxRunningContainersOnNMs[i] = nContainers;
+          }
+        }
+        try {
+          Thread.sleep(SAMPLING_INTERVAL_MS);
+        } catch (InterruptedException e) {
+          e.printStackTrace();
+          break;
+        }
+      }
+    }
+
+    public int[] getMaxRunningContainersReport() {
+      return maxRunningContainersOnNMs;
+    }
+  }
+}


[04/30] hadoop git commit: MAPREDUCE-5958. Wrong reduce task progress if map output is compressed. Contributed by Emilio Coppa and Jason Lowe.

Posted by vi...@apache.org.
MAPREDUCE-5958. Wrong reduce task progress if map output is compressed. Contributed by Emilio Coppa and Jason Lowe.


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

Branch: refs/heads/HDFS-EC
Commit: 8f701ae07a0b1dc70b8e1eb8d4a5c35c0a1e76da
Parents: 1670578
Author: Kihwal Lee <ki...@apache.org>
Authored: Thu Nov 6 15:53:40 2014 -0600
Committer: Kihwal Lee <ki...@apache.org>
Committed: Thu Nov 6 15:53:40 2014 -0600

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |  3 +
 .../java/org/apache/hadoop/mapred/Merger.java   | 12 +--
 .../mapreduce/task/reduce/TestMerger.java       | 80 ++++++++++++++------
 3 files changed, 64 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8f701ae0/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index fd42f82..573408e 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -462,6 +462,9 @@ Release 2.6.0 - UNRELEASED
     MAPREDUCE-5960. JobSubmitter's check whether job.jar is local is incorrect
     with no authority in job jar path. (Gera Shegalov via jlowe)
 
+    MAPREDUCE-5958. Wrong reduce task progress if map output is compressed
+    (Emilio Coppa and jlowe via kihwal)
+
 Release 2.5.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8f701ae0/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Merger.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Merger.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Merger.java
index 9285516..b44e742 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Merger.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Merger.java
@@ -515,9 +515,9 @@ public class Merger {
     }
 
     private void adjustPriorityQueue(Segment<K, V> reader) throws IOException{
-      long startPos = reader.getPosition();
+      long startPos = reader.getReader().bytesRead;
       boolean hasNext = reader.nextRawKey();
-      long endPos = reader.getPosition();
+      long endPos = reader.getReader().bytesRead;
       totalBytesProcessed += endPos - startPos;
       mergeProgress.set(totalBytesProcessed * progPerByte);
       if (hasNext) {
@@ -543,7 +543,7 @@ public class Merger {
         }
       }
       minSegment = top();
-      long startPos = minSegment.getPosition();
+      long startPos = minSegment.getReader().bytesRead;
       key = minSegment.getKey();
       if (!minSegment.inMemory()) {
         //When we load the value from an inmemory segment, we reset
@@ -560,7 +560,7 @@ public class Merger {
       } else {
         minSegment.getValue(value);
       }
-      long endPos = minSegment.getPosition();
+      long endPos = minSegment.getReader().bytesRead;
       totalBytesProcessed += endPos - startPos;
       mergeProgress.set(totalBytesProcessed * progPerByte);
       return true;
@@ -638,9 +638,9 @@ public class Merger {
             // Initialize the segment at the last possible moment;
             // this helps in ensuring we don't use buffers until we need them
             segment.init(readsCounter);
-            long startPos = segment.getPosition();
+            long startPos = segment.getReader().bytesRead;
             boolean hasNext = segment.nextRawKey();
-            long endPos = segment.getPosition();
+            long endPos = segment.getReader().bytesRead;
             
             if (hasNext) {
               startBytes += endPos - startPos;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8f701ae0/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestMerger.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestMerger.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestMerger.java
index c5ab420..651dd38 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestMerger.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestMerger.java
@@ -18,13 +18,12 @@
 package org.apache.hadoop.mapreduce.task.reduce;
 
 import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
-import static org.mockito.Mockito.doAnswer;
 
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
-import java.security.PrivilegedAction;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Iterator;
@@ -32,9 +31,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
 
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.junit.Assert;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalDirAllocator;
@@ -43,14 +41,15 @@ import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.RawComparator;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.Counters.Counter;
-import org.apache.hadoop.mapred.IFile.Reader;
 import org.apache.hadoop.mapred.IFile;
+import org.apache.hadoop.mapred.IFile.Reader;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.MROutputFiles;
 import org.apache.hadoop.mapred.Merger;
 import org.apache.hadoop.mapred.Merger.Segment;
 import org.apache.hadoop.mapred.RawKeyValueIterator;
 import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapreduce.CryptoUtils;
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.mapreduce.MRJobConfig;
@@ -58,21 +57,17 @@ import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.security.TokenCache;
-import org.apache.hadoop.mapreduce.CryptoUtils;
-import org.apache.hadoop.mapreduce.task.reduce.MergeManagerImpl;
 import org.apache.hadoop.mapreduce.task.reduce.MergeManagerImpl.CompressAwarePath;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Progress;
 import org.apache.hadoop.util.Progressable;
-import org.junit.After;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
-import com.google.common.collect.Lists;
-
 public class TestMerger {
 
   private Configuration conf;
@@ -254,7 +249,7 @@ public class TestMerger {
     testMergeShouldReturnProperProgress(getUncompressedSegments());
   }
 
-  @SuppressWarnings( { "deprecation", "unchecked" })
+  @SuppressWarnings( { "unchecked" })
   public void testMergeShouldReturnProperProgress(
       List<Segment<Text, Text>> segments) throws IOException {
     Path tmpDir = new Path("localpath");
@@ -267,7 +262,38 @@ public class TestMerger {
     RawKeyValueIterator mergeQueue = Merger.merge(conf, fs, keyClass,
         valueClass, segments, 2, tmpDir, comparator, getReporter(),
         readsCounter, writesCounter, mergePhase);
-    Assert.assertEquals(1.0f, mergeQueue.getProgress().get(), 0.0f);
+    final float epsilon = 0.00001f;
+
+    // Reading 6 keys total, 3 each in 2 segments, so each key read moves the
+    // progress forward 1/6th of the way. Initially the first keys from each
+    // segment have been read as part of the merge setup, so progress = 2/6.
+    Assert.assertEquals(2/6.0f, mergeQueue.getProgress().get(), epsilon);
+
+    // The first next() returns one of the keys already read during merge setup
+    Assert.assertTrue(mergeQueue.next());
+    Assert.assertEquals(2/6.0f, mergeQueue.getProgress().get(), epsilon);
+
+    // Subsequent next() calls should read one key and move progress
+    Assert.assertTrue(mergeQueue.next());
+    Assert.assertEquals(3/6.0f, mergeQueue.getProgress().get(), epsilon);
+    Assert.assertTrue(mergeQueue.next());
+    Assert.assertEquals(4/6.0f, mergeQueue.getProgress().get(), epsilon);
+
+    // At this point we've exhausted all of the keys in one segment
+    // so getting the next key will return the already cached key from the
+    // other segment
+    Assert.assertTrue(mergeQueue.next());
+    Assert.assertEquals(4/6.0f, mergeQueue.getProgress().get(), epsilon);
+
+    // Subsequent next() calls should read one key and move progress
+    Assert.assertTrue(mergeQueue.next());
+    Assert.assertEquals(5/6.0f, mergeQueue.getProgress().get(), epsilon);
+    Assert.assertTrue(mergeQueue.next());
+    Assert.assertEquals(1.0f, mergeQueue.getProgress().get(), epsilon);
+
+    // Now there should be no more input
+    Assert.assertFalse(mergeQueue.next());
+    Assert.assertEquals(1.0f, mergeQueue.getProgress().get(), epsilon);
   }
 
   private Progressable getReporter() {
@@ -281,7 +307,7 @@ public class TestMerger {
 
   private List<Segment<Text, Text>> getUncompressedSegments() throws IOException {
     List<Segment<Text, Text>> segments = new ArrayList<Segment<Text, Text>>();
-    for (int i = 1; i < 1; i++) {
+    for (int i = 0; i < 2; i++) {
       segments.add(getUncompressedSegment(i));
     }
     return segments;
@@ -289,44 +315,51 @@ public class TestMerger {
 
   private List<Segment<Text, Text>> getCompressedSegments() throws IOException {
     List<Segment<Text, Text>> segments = new ArrayList<Segment<Text, Text>>();
-    for (int i = 1; i < 1; i++) {
+    for (int i = 0; i < 2; i++) {
       segments.add(getCompressedSegment(i));
     }
     return segments;
   }
 
   private Segment<Text, Text> getUncompressedSegment(int i) throws IOException {
-    return new Segment<Text, Text>(getReader(i), false);
+    return new Segment<Text, Text>(getReader(i, false), false);
   }
 
   private Segment<Text, Text> getCompressedSegment(int i) throws IOException {
-    return new Segment<Text, Text>(getReader(i), false, 3000l);
+    return new Segment<Text, Text>(getReader(i, true), false, 3000l);
   }
 
   @SuppressWarnings("unchecked")
-  private Reader<Text, Text> getReader(int i) throws IOException {
+  private Reader<Text, Text> getReader(int i, boolean isCompressedInput)
+      throws IOException {
     Reader<Text, Text> readerMock = mock(Reader.class);
+    when(readerMock.getLength()).thenReturn(30l);
     when(readerMock.getPosition()).thenReturn(0l).thenReturn(10l).thenReturn(
         20l);
     when(
         readerMock.nextRawKey(any(DataInputBuffer.class)))
-        .thenAnswer(getKeyAnswer("Segment" + i));
+        .thenAnswer(getKeyAnswer("Segment" + i, isCompressedInput));
     doAnswer(getValueAnswer("Segment" + i)).when(readerMock).nextRawValue(
         any(DataInputBuffer.class));
 
     return readerMock;
   }
 
-  private Answer<?> getKeyAnswer(final String segmentName) {
+  private Answer<?> getKeyAnswer(final String segmentName,
+      final boolean isCompressedInput) {
     return new Answer<Object>() {
       int i = 0;
 
+      @SuppressWarnings("unchecked")
       public Boolean answer(InvocationOnMock invocation) {
-        Object[] args = invocation.getArguments();
-        DataInputBuffer key = (DataInputBuffer) args[0];
-        if (i++ == 2) {
+        if (i++ == 3) {
           return false;
         }
+        Reader<Text,Text> mock = (Reader<Text,Text>) invocation.getMock();
+        int multiplier = isCompressedInput ? 100 : 1;
+        mock.bytesRead += 10 * multiplier;
+        Object[] args = invocation.getArguments();
+        DataInputBuffer key = (DataInputBuffer) args[0];
         key.reset(("Segment Key " + segmentName + i).getBytes(), 20);
         return true;
       }
@@ -340,9 +373,6 @@ public class TestMerger {
       public Void answer(InvocationOnMock invocation) {
         Object[] args = invocation.getArguments();
         DataInputBuffer key = (DataInputBuffer) args[0];
-        if (i++ == 2) {
-          return null;
-        }
         key.reset(("Segment Value " + segmentName + i).getBytes(), 20);
         return null;
       }


[06/30] hadoop git commit: HADOOP-11274. ConcurrentModificationException in Configuration Copy Constructor. Contributed by Junping Du.

Posted by vi...@apache.org.
HADOOP-11274. ConcurrentModificationException in Configuration Copy Constructor. Contributed by Junping Du.


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

Branch: refs/heads/HDFS-EC
Commit: 16b34824673f5a50d464727b8fad98470e5e984a
Parents: 75b820c
Author: Jing Zhao <ji...@apache.org>
Authored: Thu Nov 6 16:07:50 2014 -0800
Committer: Jing Zhao <ji...@apache.org>
Committed: Thu Nov 6 16:07:50 2014 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 +
 .../org/apache/hadoop/conf/Configuration.java   | 85 +++++++++++---------
 2 files changed, 48 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/16b34824/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 8587f12..679b5bb 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -910,6 +910,9 @@ Release 2.6.0 - UNRELEASED
     HADOOP-11253. Hadoop streaming test TestStreamXmlMultipleRecords fails on
     Windows. (Varun Vasudev via wheat9)
 
+    HADOOP-11274. ConcurrentModificationException in Configuration Copy Constructor.
+    (Junping Du via jing9)
+
     BREAKDOWN OF HDFS-6134 AND HADOOP-10150 SUBTASKS AND RELATED JIRAS
   
       HADOOP-10734. Implement high-performance secure random number sources.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/16b34824/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
index a3fae19..16d5499 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
@@ -690,26 +690,26 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
    */
   @SuppressWarnings("unchecked")
   public Configuration(Configuration other) {
-   this.resources = (ArrayList<Resource>) other.resources.clone();
-   synchronized(other) {
-     if (other.properties != null) {
-       this.properties = (Properties)other.properties.clone();
-     }
-
-     if (other.overlay!=null) {
-       this.overlay = (Properties)other.overlay.clone();
-     }
-
-     this.updatingResource = new HashMap<String, String[]>(other.updatingResource);
-     this.finalParameters = new HashSet<String>(other.finalParameters);
-   }
-   
+    synchronized(other) {
+      this.resources = (ArrayList<Resource>) other.resources.clone();
+      if (other.properties != null) {
+        this.properties = (Properties)other.properties.clone();
+      }
+
+      if (other.overlay!=null) {
+        this.overlay = (Properties)other.overlay.clone();
+      }
+
+      this.updatingResource = new HashMap<String, String[]>(other.updatingResource);
+      this.finalParameters = new HashSet<String>(other.finalParameters);
+
+      this.classLoader = other.classLoader;
+      this.loadDefaults = other.loadDefaults;
+      setQuietMode(other.getQuietMode());
+    }
     synchronized(Configuration.class) {
       REGISTRY.put(this, null);
     }
-    this.classLoader = other.classLoader;
-    this.loadDefaults = other.loadDefaults;
-    setQuietMode(other.getQuietMode());
   }
   
   /**
@@ -1025,26 +1025,28 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     getProps().setProperty(name, value);
     String newSource = (source == null ? "programmatically" : source);
 
-    if (!isDeprecated(name)) {
-      updatingResource.put(name, new String[] {newSource});
-      String[] altNames = getAlternativeNames(name);
-      if(altNames != null) {
-        for(String n: altNames) {
-          if(!n.equals(name)) {
-            getOverlay().setProperty(n, value);
-            getProps().setProperty(n, value);
-            updatingResource.put(n, new String[] {newSource});
+    synchronized (this) {
+      if (!isDeprecated(name)) {
+        updatingResource.put(name, new String[] {newSource});
+        String[] altNames = getAlternativeNames(name);
+        if(altNames != null) {
+          for(String n: altNames) {
+            if(!n.equals(name)) {
+              getOverlay().setProperty(n, value);
+              getProps().setProperty(n, value);
+              updatingResource.put(n, new String[] {newSource});
+            }
           }
         }
       }
-    }
-    else {
-      String[] names = handleDeprecation(deprecationContext.get(), name);
-      String altSource = "because " + name + " is deprecated";
-      for(String n : names) {
-        getOverlay().setProperty(n, value);
-        getProps().setProperty(n, value);
-        updatingResource.put(n, new String[] {altSource});
+      else {
+        String[] names = handleDeprecation(deprecationContext.get(), name);
+        String altSource = "because " + name + " is deprecated";
+        for(String n : names) {
+          getOverlay().setProperty(n, value);
+          getProps().setProperty(n, value);
+          updatingResource.put(n, new String[] {altSource});
+        }
       }
     }
   }
@@ -2277,7 +2279,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
    *
    * @return final parameter set.
    */
-  public Set<String> getFinalParameters() {
+  public synchronized Set<String> getFinalParameters() {
     return new HashSet<String>(finalParameters);
   }
 
@@ -2540,14 +2542,18 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     if (value != null) {
       if (!finalParameters.contains(attr)) {
         properties.setProperty(attr, value);
-        updatingResource.put(attr, source);
+        synchronized(this) {
+          updatingResource.put(attr, source);
+        }
       } else if (!value.equals(properties.getProperty(attr))) {
         LOG.warn(name+":an attempt to override final parameter: "+attr
             +";  Ignoring.");
       }
     }
     if (finalParameter) {
-      finalParameters.add(attr);
+      synchronized(this) {
+        finalParameters.add(attr);
+      }
     }
   }
 
@@ -2741,7 +2747,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
   }
 
   @Override
-  public void readFields(DataInput in) throws IOException {
+  public synchronized void readFields(DataInput in) throws IOException {
     clear();
     int size = WritableUtils.readVInt(in);
     for(int i=0; i < size; ++i) {
@@ -2753,9 +2759,8 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     }
   }
 
-  //@Override
   @Override
-  public void write(DataOutput out) throws IOException {
+  public synchronized void write(DataOutput out) throws IOException {
     Properties props = getProps();
     WritableUtils.writeVInt(out, props.size());
     for(Map.Entry<Object, Object> item: props.entrySet()) {


[08/30] hadoop git commit: HDFS-7364. Balancer always shows zero Bytes Already Moved. Contributed by Tsz Wo Nicholas Sze.

Posted by vi...@apache.org.
HDFS-7364. Balancer always shows zero Bytes Already Moved. 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/ae71a671
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ae71a671
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ae71a671

Branch: refs/heads/HDFS-EC
Commit: ae71a671a3b4b454aa393c2974b6f1f16dd61405
Parents: a3839a9
Author: Jing Zhao <ji...@apache.org>
Authored: Thu Nov 6 17:48:36 2014 -0800
Committer: Jing Zhao <ji...@apache.org>
Committed: Thu Nov 6 17:48:36 2014 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../hadoop/hdfs/server/balancer/Balancer.java   | 81 ++++++++++++-------
 .../hadoop/hdfs/server/balancer/Dispatcher.java | 13 ++-
 .../hdfs/server/balancer/NameNodeConnector.java |  6 ++
 .../hdfs/server/balancer/TestBalancer.java      | 84 ++++++++++++++++++--
 5 files changed, 142 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae71a671/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 1a41251..5a7dac0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1042,6 +1042,9 @@ Release 2.6.0 - UNRELEASED
     file descriptors when SASL is enabled on DataTransferProtocol.
     (Chris Nauroth via wheat9)
 
+    HDFS-7364. Balancer always shows zero Bytes Already Moved.
+    (Tsz Wo Nicholas Sze via jing9)
+
     BREAKDOWN OF HDFS-6134 AND HADOOP-10150 SUBTASKS AND RELATED JIRAS
   
       HDFS-6387. HDFS CLI admin tool for creating & deleting an

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae71a671/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
index 2a19537..dba1e2d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
@@ -27,7 +27,6 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Date;
-import java.util.Formatter;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
@@ -162,7 +161,7 @@ import com.google.common.base.Preconditions;
 public class Balancer {
   static final Log LOG = LogFactory.getLog(Balancer.class);
 
-  private static final Path BALANCER_ID_PATH = new Path("/system/balancer.id");
+  static final Path BALANCER_ID_PATH = new Path("/system/balancer.id");
 
   private static final long GB = 1L << 30; //1GB
   private static final long MAX_SIZE_TO_MOVE = 10*GB;
@@ -459,7 +458,7 @@ public class Balancer {
   }
 
   /* reset all fields in a balancer preparing for the next iteration */
-  private void resetData(Configuration conf) {
+  void resetData(Configuration conf) {
     this.overUtilized.clear();
     this.aboveAvgUtilized.clear();
     this.belowAvgUtilized.clear();
@@ -467,16 +466,47 @@ public class Balancer {
     this.policy.reset();
     dispatcher.reset(conf);;
   }
-  
+
+  static class Result {
+    final ExitStatus exitStatus;
+    final long bytesLeftToMove;
+    final long bytesBeingMoved;
+    final long bytesAlreadyMoved;
+
+    Result(ExitStatus exitStatus, long bytesLeftToMove, long bytesBeingMoved,
+        long bytesAlreadyMoved) {
+      this.exitStatus = exitStatus;
+      this.bytesLeftToMove = bytesLeftToMove;
+      this.bytesBeingMoved = bytesBeingMoved;
+      this.bytesAlreadyMoved = bytesAlreadyMoved;
+    }
+
+    void print(int iteration, PrintStream out) {
+      out.printf("%-24s %10d  %19s  %18s  %17s%n",
+          DateFormat.getDateTimeInstance().format(new Date()), iteration,
+          StringUtils.byteDesc(bytesAlreadyMoved),
+          StringUtils.byteDesc(bytesLeftToMove),
+          StringUtils.byteDesc(bytesBeingMoved));
+    }
+  }
+
+  Result newResult(ExitStatus exitStatus, long bytesLeftToMove, long bytesBeingMoved) {
+    return new Result(exitStatus, bytesLeftToMove, bytesBeingMoved,
+        dispatcher.getBytesMoved());
+  }
+
+  Result newResult(ExitStatus exitStatus) {
+    return new Result(exitStatus, -1, -1, dispatcher.getBytesMoved());
+  }
+
   /** Run an iteration for all datanodes. */
-  private ExitStatus run(int iteration, Formatter formatter,
-      Configuration conf) {
+  Result runOneIteration() {
     try {
       final List<DatanodeStorageReport> reports = dispatcher.init();
       final long bytesLeftToMove = init(reports);
       if (bytesLeftToMove == 0) {
         System.out.println("The cluster is balanced. Exiting...");
-        return ExitStatus.SUCCESS;
+        return newResult(ExitStatus.SUCCESS, bytesLeftToMove, -1);
       } else {
         LOG.info( "Need to move "+ StringUtils.byteDesc(bytesLeftToMove)
             + " to make the cluster balanced." );
@@ -487,22 +517,14 @@ public class Balancer {
        * in this iteration. Maximum bytes to be moved per node is
        * Min(1 Band worth of bytes,  MAX_SIZE_TO_MOVE).
        */
-      final long bytesToMove = chooseStorageGroups();
-      if (bytesToMove == 0) {
+      final long bytesBeingMoved = chooseStorageGroups();
+      if (bytesBeingMoved == 0) {
         System.out.println("No block can be moved. Exiting...");
-        return ExitStatus.NO_MOVE_BLOCK;
+        return newResult(ExitStatus.NO_MOVE_BLOCK, bytesLeftToMove, bytesBeingMoved);
       } else {
-        LOG.info( "Will move " + StringUtils.byteDesc(bytesToMove) +
+        LOG.info( "Will move " + StringUtils.byteDesc(bytesBeingMoved) +
             " in this iteration");
       }
-
-      formatter.format("%-24s %10d  %19s  %18s  %17s%n",
-          DateFormat.getDateTimeInstance().format(new Date()),
-          iteration,
-          StringUtils.byteDesc(dispatcher.getBytesMoved()),
-          StringUtils.byteDesc(bytesLeftToMove),
-          StringUtils.byteDesc(bytesToMove)
-          );
       
       /* For each pair of <source, target>, start a thread that repeatedly 
        * decide a block to be moved and its proxy source, 
@@ -511,19 +533,19 @@ public class Balancer {
        * Exit no byte has been moved for 5 consecutive iterations.
        */
       if (!dispatcher.dispatchAndCheckContinue()) {
-        return ExitStatus.NO_MOVE_PROGRESS;
+        return newResult(ExitStatus.NO_MOVE_PROGRESS, bytesLeftToMove, bytesBeingMoved);
       }
 
-      return ExitStatus.IN_PROGRESS;
+      return newResult(ExitStatus.IN_PROGRESS, bytesLeftToMove, bytesBeingMoved);
     } catch (IllegalArgumentException e) {
       System.out.println(e + ".  Exiting ...");
-      return ExitStatus.ILLEGAL_ARGUMENTS;
+      return newResult(ExitStatus.ILLEGAL_ARGUMENTS);
     } catch (IOException e) {
       System.out.println(e + ".  Exiting ...");
-      return ExitStatus.IO_EXCEPTION;
+      return newResult(ExitStatus.IO_EXCEPTION);
     } catch (InterruptedException e) {
       System.out.println(e + ".  Exiting ...");
-      return ExitStatus.INTERRUPTED;
+      return newResult(ExitStatus.INTERRUPTED);
     } finally {
       dispatcher.shutdownNow();
     }
@@ -545,7 +567,6 @@ public class Balancer {
     LOG.info("namenodes  = " + namenodes);
     LOG.info("parameters = " + p);
     
-    final Formatter formatter = new Formatter(System.out);
     System.out.println("Time Stamp               Iteration#  Bytes Already Moved  Bytes Left To Move  Bytes Being Moved");
     
     List<NameNodeConnector> connectors = Collections.emptyList();
@@ -559,14 +580,16 @@ public class Balancer {
         Collections.shuffle(connectors);
         for(NameNodeConnector nnc : connectors) {
           final Balancer b = new Balancer(nnc, p, conf);
-          final ExitStatus r = b.run(iteration, formatter, conf);
+          final Result r = b.runOneIteration();
+          r.print(iteration, System.out);
+
           // clean all lists
           b.resetData(conf);
-          if (r == ExitStatus.IN_PROGRESS) {
+          if (r.exitStatus == ExitStatus.IN_PROGRESS) {
             done = false;
-          } else if (r != ExitStatus.SUCCESS) {
+          } else if (r.exitStatus != ExitStatus.SUCCESS) {
             //must be an error statue, return.
-            return r.getExitCode();
+            return r.exitStatus.getExitCode();
           }
         }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae71a671/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 8b881e9..6ede40a 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
@@ -41,9 +41,7 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
-import java.util.concurrent.atomic.AtomicLong;
 
-import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -76,6 +74,7 @@ import org.apache.hadoop.util.HostsFileReader;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 
 /** Dispatching block replica moves between datanodes. */
@@ -119,8 +118,6 @@ public class Dispatcher {
   /** The maximum number of concurrent blocks moves at a datanode */
   private final int maxConcurrentMovesPerNode;
 
-  private final AtomicLong bytesMoved = new AtomicLong();
-
   private static class GlobalBlockMap {
     private final Map<Block, DBlock> map = new HashMap<Block, DBlock>();
 
@@ -313,7 +310,7 @@ public class Dispatcher {
 
         sendRequest(out, eb, accessToken);
         receiveResponse(in);
-        bytesMoved.addAndGet(block.getNumBytes());
+        nnc.getBytesMoved().addAndGet(block.getNumBytes());
         LOG.info("Successfully moved " + this);
       } catch (IOException e) {
         LOG.warn("Failed to move " + this + ": " + e.getMessage());
@@ -803,7 +800,7 @@ public class Dispatcher {
   }
   
   long getBytesMoved() {
-    return bytesMoved.get();
+    return nnc.getBytesMoved().get();
   }
 
   long bytesToMove() {
@@ -889,7 +886,7 @@ public class Dispatcher {
    * @return the total number of bytes successfully moved in this iteration.
    */
   private long dispatchBlockMoves() throws InterruptedException {
-    final long bytesLastMoved = bytesMoved.get();
+    final long bytesLastMoved = getBytesMoved();
     final Future<?>[] futures = new Future<?>[sources.size()];
 
     final Iterator<Source> i = sources.iterator();
@@ -915,7 +912,7 @@ public class Dispatcher {
     // wait for all block moving to be done
     waitForMoveCompletion(targets);
 
-    return bytesMoved.get() - bytesLastMoved;
+    return getBytesMoved() - bytesLastMoved;
   }
 
   /** The sleeping period before checking if block move is completed again */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae71a671/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java
index 9162531..e01d57d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java
@@ -28,6 +28,7 @@ import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -108,6 +109,7 @@ public class NameNodeConnector implements Closeable {
   private final Path idPath;
   private final OutputStream out;
   private final List<Path> targetPaths;
+  private final AtomicLong bytesMoved = new AtomicLong();
 
   private int notChangedIterations = 0;
 
@@ -148,6 +150,10 @@ public class NameNodeConnector implements Closeable {
     return blockpoolID;
   }
 
+  AtomicLong getBytesMoved() {
+    return bytesMoved;
+  }
+
   /** @return blocks with locations. */
   public BlocksWithLocations getBlocks(DatanodeInfo datanode, long size)
       throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae71a671/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 793675f..698aa69 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
@@ -17,7 +17,12 @@
  */
 package org.apache.hadoop.hdfs.server.balancer;
 
-import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_LAZY_WRITER_INTERVAL_SEC;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_RAM_DISK_LOW_WATERMARK_BYTES;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC;
 import static org.apache.hadoop.hdfs.StorageType.DEFAULT;
 import static org.apache.hadoop.hdfs.StorageType.RAM_DISK;
 import static org.junit.Assert.assertEquals;
@@ -31,6 +36,7 @@ import java.net.URI;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Random;
@@ -44,7 +50,15 @@ import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.*;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.NameNodeProxies;
+import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -53,8 +67,10 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.balancer.Balancer.Cli;
 import org.apache.hadoop.hdfs.server.balancer.Balancer.Parameters;
+import org.apache.hadoop.hdfs.server.balancer.Balancer.Result;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Tool;
@@ -65,8 +81,8 @@ import org.junit.Test;
  * This class tests if a balancer schedules tasks correctly.
  */
 public class TestBalancer {
-  private static final Log LOG = LogFactory.getLog(
-  "org.apache.hadoop.hdfs.TestBalancer");
+  private static final Log LOG = LogFactory.getLog(TestBalancer.class);
+
   static {
     ((Log4JLogger)Balancer.LOG).getLogger().setLevel(Level.ALL);
   }
@@ -478,7 +494,7 @@ public class TestBalancer {
     LOG.info("racks      = " +  Arrays.asList(racks)); 
     LOG.info("newCapacity= " +  newCapacity); 
     LOG.info("newRack    = " +  newRack); 
-    LOG.info("useTool    = " +  useTool); 
+    LOG.info("useTool    = " +  useTool);
     assertEquals(capacities.length, racks.length);
     int numOfDatanodes = capacities.length;
     cluster = new MiniDFSCluster.Builder(conf)
@@ -584,7 +600,7 @@ public class TestBalancer {
 
     // start rebalancing
     Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
-    final int r = Balancer.run(namenodes, p, conf);
+    final int r = runBalancer(namenodes, p, conf);
     if (conf.getInt(DFSConfigKeys.DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY, 
         DFSConfigKeys.DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_DEFAULT) ==0) {
       assertEquals(ExitStatus.NO_MOVE_PROGRESS.getExitCode(), r);
@@ -593,10 +609,63 @@ public class TestBalancer {
       assertEquals(ExitStatus.SUCCESS.getExitCode(), r);
     }
     waitForHeartBeat(totalUsedSpace, totalCapacity, client, cluster);
-    LOG.info("Rebalancing with default ctor.");
+    LOG.info("  .");
     waitForBalancer(totalUsedSpace, totalCapacity, client, cluster, p, excludedNodes);
   }
 
+  private static int runBalancer(Collection<URI> namenodes, final Parameters p,
+      Configuration conf) throws IOException, InterruptedException {
+    final long sleeptime =
+        conf.getLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
+            DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT) * 2000 +
+        conf.getLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY,
+            DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT) * 1000;
+    LOG.info("namenodes  = " + namenodes);
+    LOG.info("parameters = " + p);
+    LOG.info("Print stack trace", new Throwable());
+
+    System.out.println("Time Stamp               Iteration#  Bytes Already Moved  Bytes Left To Move  Bytes Being Moved");
+
+    List<NameNodeConnector> connectors = Collections.emptyList();
+    try {
+      connectors = NameNodeConnector.newNameNodeConnectors(namenodes, 
+            Balancer.class.getSimpleName(), Balancer.BALANCER_ID_PATH, conf);
+    
+      boolean done = false;
+      for(int iteration = 0; !done; iteration++) {
+        done = true;
+        Collections.shuffle(connectors);
+        for(NameNodeConnector nnc : connectors) {
+          final Balancer b = new Balancer(nnc, p, conf);
+          final Result r = b.runOneIteration();
+          r.print(iteration, System.out);
+
+          // clean all lists
+          b.resetData(conf);
+          if (r.exitStatus == ExitStatus.IN_PROGRESS) {
+            done = false;
+          } else if (r.exitStatus != ExitStatus.SUCCESS) {
+            //must be an error statue, return.
+            return r.exitStatus.getExitCode();
+          } else {
+            if (iteration > 0) {
+              assertTrue(r.bytesAlreadyMoved > 0);
+            }
+          }
+        }
+
+        if (!done) {
+          Thread.sleep(sleeptime);
+        }
+      }
+    } finally {
+      for(NameNodeConnector nnc : connectors) {
+        IOUtils.cleanup(LOG, nnc);
+      }
+    }
+    return ExitStatus.SUCCESS.getExitCode();
+  }
+
   private void runBalancerCli(Configuration conf,
       long totalUsedSpace, long totalCapacity,
       Balancer.Parameters p, boolean useFile, int expectedExcludedNodes) throws Exception {
@@ -1118,7 +1187,6 @@ public class TestBalancer {
     initConfWithRamDisk(conf);
 
     final int defaultRamDiskCapacity = 10;
-    final int defaultDiskCapacity = 100;
     final long ramDiskStorageLimit =
       ((long) defaultRamDiskCapacity * DEFAULT_RAM_DISK_BLOCK_SIZE) +
       (DEFAULT_RAM_DISK_BLOCK_SIZE - 1);


[17/30] hadoop git commit: YARN-2827. Fixed bugs in "yarn queue" CLI. Contributed by Wangda Tan.

Posted by vi...@apache.org.
YARN-2827. Fixed bugs in "yarn queue" CLI. 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/a71e9302
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a71e9302
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a71e9302

Branch: refs/heads/HDFS-EC
Commit: a71e9302a9dbbb702c8c50a4fdaa508f912561d6
Parents: 2ac1be7
Author: Vinod Kumar Vavilapalli <vi...@apache.org>
Authored: Fri Nov 7 11:56:11 2014 -0800
Committer: Vinod Kumar Vavilapalli <vi...@apache.org>
Committed: Fri Nov 7 11:56:11 2014 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  2 +
 .../apache/hadoop/yarn/client/cli/QueueCLI.java | 40 +++++++++++---------
 .../hadoop/yarn/client/cli/TestYarnCLI.java     | 21 +++++++++-
 3 files changed, 43 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a71e9302/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index c48fb4f..a26bfb2 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -898,6 +898,8 @@ Release 2.6.0 - UNRELEASED
     not mapped to queues by making default capacities per label to be zero.
     (Wangda Tan via vinodkv)
 
+    YARN-2827. Fixed bugs in "yarn queue" CLI. (Wangda Tan via vinodkv)
+
 Release 2.5.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a71e9302/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/QueueCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/QueueCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/QueueCLI.java
index 3c74f13..4d50e7f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/QueueCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/QueueCLI.java
@@ -52,16 +52,11 @@ public class QueueCLI extends YarnCLI {
   @Override
   public int run(String[] args) throws Exception {
     Options opts = new Options();
-    int exitCode = -1;
-    if (args.length > 0) {
-      opts.addOption(STATUS_CMD, true,
-          "List queue information about given queue.");
-      opts.addOption(HELP_CMD, false, "Displays help for all commands.");
-      opts.getOption(STATUS_CMD).setArgName("Queue Name");
-    } else {
-      syserr.println("Invalid Command usage. Command must start with 'queue'");
-      return exitCode;
-    }
+
+    opts.addOption(STATUS_CMD, true,
+        "List queue information about given queue.");
+    opts.addOption(HELP_CMD, false, "Displays help for all commands.");
+    opts.getOption(STATUS_CMD).setArgName("Queue Name");
 
     CommandLine cliParser = null;
     try {
@@ -69,23 +64,23 @@ public class QueueCLI extends YarnCLI {
     } catch (MissingArgumentException ex) {
       sysout.println("Missing argument for options");
       printUsage(opts);
-      return exitCode;
+      return -1;
     }
 
     if (cliParser.hasOption(STATUS_CMD)) {
-      if (args.length != 3) {
+      if (args.length != 2) {
         printUsage(opts);
-        return exitCode;
+        return -1;
       }
-      listQueue(cliParser.getOptionValue(STATUS_CMD));
+      return listQueue(cliParser.getOptionValue(STATUS_CMD));
     } else if (cliParser.hasOption(HELP_CMD)) {
       printUsage(opts);
       return 0;
     } else {
       syserr.println("Invalid Command Usage : ");
       printUsage(opts);
+      return -1;
     }
-    return 0;
   }
 
   /**
@@ -105,13 +100,22 @@ public class QueueCLI extends YarnCLI {
    * @throws YarnException
    * @throws IOException
    */
-  private void listQueue(String queueName) throws YarnException, IOException {
+  private int listQueue(String queueName) throws YarnException, IOException {
+    int rc;
     PrintWriter writer = new PrintWriter(sysout);
 
     QueueInfo queueInfo = client.getQueueInfo(queueName);
-    writer.println("Queue Information : ");
-    printQueueInfo(writer, queueInfo);
+    if (queueInfo != null) {
+      writer.println("Queue Information : ");
+      printQueueInfo(writer, queueInfo);
+      rc = 0;
+    } else {
+      writer.println("Cannot get queue from RM by queueName = " + queueName
+          + ", please check.");
+      rc = -1;
+    }
     writer.flush();
+    return rc;
   }
 
   private void printQueueInfo(PrintWriter writer, QueueInfo queueInfo) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a71e9302/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
index adb58e8..d87277a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
@@ -1247,7 +1247,7 @@ public class TestYarnCLI {
     QueueInfo queueInfo = QueueInfo.newInstance("queueA", 0.4f, 0.8f, 0.5f,
         null, null, QueueState.RUNNING, nodeLabels, "GPU");
     when(client.getQueueInfo(any(String.class))).thenReturn(queueInfo);
-    int result = cli.run(new String[] { "queue", "-status", "queueA" });
+    int result = cli.run(new String[] { "-status", "queueA" });
     assertEquals(0, result);
     verify(client).getQueueInfo("queueA");
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
@@ -1271,7 +1271,7 @@ public class TestYarnCLI {
     QueueInfo queueInfo = QueueInfo.newInstance("queueA", 0.4f, 0.8f, 0.5f,
         null, null, QueueState.RUNNING, null, null);
     when(client.getQueueInfo(any(String.class))).thenReturn(queueInfo);
-    int result = cli.run(new String[] { "queue", "-status", "queueA" });
+    int result = cli.run(new String[] { "-status", "queueA" });
     assertEquals(0, result);
     verify(client).getQueueInfo("queueA");
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
@@ -1288,6 +1288,23 @@ public class TestYarnCLI {
     String queueInfoStr = baos.toString("UTF-8");
     Assert.assertEquals(queueInfoStr, sysOutStream.toString());
   }
+  
+  @Test
+  public void testGetQueueInfoWithNonExistedQueue() throws Exception {
+    String queueName = "non-existed-queue";
+    QueueCLI cli = createAndGetQueueCLI();
+    when(client.getQueueInfo(any(String.class))).thenReturn(null);
+    int result = cli.run(new String[] { "-status", queueName });
+    assertEquals(-1, result);;
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    PrintWriter pw = new PrintWriter(baos);
+    pw.println("Cannot get queue from RM by queueName = " + queueName
+        + ", please check.");
+    pw.close();
+    String queueInfoStr = baos.toString("UTF-8");
+    Assert.assertEquals(queueInfoStr, sysOutStream.toString());
+  }
+
 
   private void verifyUsageInfo(YarnCLI cli) throws Exception {
     cli.setSysErrPrintStream(sysErr);


[05/30] hadoop git commit: HDFS-7365. Remove hdfs.server.blockmanagement.MutableBlockCollection. Contributed by Li Lu.

Posted by vi...@apache.org.
HDFS-7365. Remove hdfs.server.blockmanagement.MutableBlockCollection. Contributed by Li Lu.


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

Branch: refs/heads/HDFS-EC
Commit: 75b820cca9d4e709b9e8d40635ff0406528ad4ba
Parents: 8f701ae
Author: Haohui Mai <wh...@apache.org>
Authored: Thu Nov 6 13:53:37 2014 -0800
Committer: Haohui Mai <wh...@apache.org>
Committed: Thu Nov 6 13:56:31 2014 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../blockmanagement/MutableBlockCollection.java | 38 --------------------
 2 files changed, 3 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b820cc/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 1cdfeb1..1a41251 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -349,6 +349,9 @@ Release 2.7.0 - UNRELEASED
 
     HDFS-7336. Unused member DFSInputStream.buffersize. (Milan Desai via shv)
 
+    HDFS-7365. Remove hdfs.server.blockmanagement.MutableBlockCollection.
+    (Li Lu via wheat9)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b820cc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/MutableBlockCollection.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/MutableBlockCollection.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/MutableBlockCollection.java
deleted file mode 100644
index 5d63fef..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/MutableBlockCollection.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/**
- * 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.blockmanagement;
-
-import java.io.IOException;
-
-/** 
- * This interface is used by the block manager to expose a
- * few characteristics of a collection of Block/BlockUnderConstruction.
- */
-public interface MutableBlockCollection extends BlockCollection {
-  /**
-   * Set the block at the given index.
-   */
-  public void setBlock(int index, BlockInfo blk);
-
-  /**
-   * Convert the last block of the collection to an under-construction block
-   * and set the locations.
-   */
-  public BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock,
-      DatanodeStorageInfo[] storages) throws IOException;
-}


[18/30] hadoop git commit: YARN-2803. MR distributed cache not working correctly on Windows after NodeManager privileged account changes. Contributed by Craig Welch.

Posted by vi...@apache.org.
YARN-2803. MR distributed cache not working correctly on Windows after NodeManager privileged account changes. 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/06b79794
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/06b79794
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/06b79794

Branch: refs/heads/HDFS-EC
Commit: 06b797947c980d7d21864eb8b700cf565756aac1
Parents: a71e930
Author: cnauroth <cn...@apache.org>
Authored: Fri Nov 7 12:29:39 2014 -0800
Committer: cnauroth <cn...@apache.org>
Committed: Fri Nov 7 12:29:39 2014 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                         |  3 +++
 .../containermanager/launcher/ContainerLaunch.java      | 12 +++++++++++-
 2 files changed, 14 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/06b79794/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index a26bfb2..97b67b5 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -900,6 +900,9 @@ Release 2.6.0 - UNRELEASED
 
     YARN-2827. Fixed bugs in "yarn queue" CLI. (Wangda Tan via vinodkv)
 
+    YARN-2803. MR distributed cache not working correctly on Windows after
+    NodeManager privileged account changes. (Craig Welch via cnauroth)
+
 Release 2.5.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06b79794/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.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/launcher/ContainerLaunch.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
index 57e3bb9..8fc5ea3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
@@ -71,6 +71,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Cont
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
+import org.apache.hadoop.yarn.server.nodemanager.WindowsSecureContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.util.ProcessIdFileReader;
 import org.apache.hadoop.yarn.util.Apps;
 import org.apache.hadoop.yarn.util.AuxiliaryServiceHelper;
@@ -767,8 +768,17 @@ public class ContainerLaunch implements Callable<Integer> {
           System.getenv());
         mergedEnv.putAll(environment);
         
+        // this is hacky and temporary - it's to preserve the windows secure
+        // behavior but enable non-secure windows to properly build the class
+        // path for access to job.jar/lib/xyz and friends (see YARN-2803)
+        Path jarDir;
+        if (exec instanceof WindowsSecureContainerExecutor) {
+          jarDir = nmPrivateClasspathJarDir;
+        } else {
+          jarDir = pwd; 
+        }
         String[] jarCp = FileUtil.createJarWithClassPath(
-          newClassPath.toString(), nmPrivateClasspathJarDir, pwd, mergedEnv);
+          newClassPath.toString(), jarDir, pwd, mergedEnv);
         // In a secure cluster the classpath jar must be localized to grant access
         Path localizedClassPathJar = exec.localizeClasspathJar(
             new Path(jarCp[0]), pwd, container.getUser());


[03/30] hadoop git commit: YARN-2768 Improved Yarn Registry service record structure (stevel)

Posted by vi...@apache.org.
YARN-2768 Improved Yarn Registry service record structure (stevel)


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

Branch: refs/heads/HDFS-EC
Commit: 1670578018b3210d518408530858a869e37b23cb
Parents: f5b19be
Author: Steve Loughran <st...@apache.org>
Authored: Thu Nov 6 20:21:25 2014 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Nov 6 20:22:22 2014 +0000

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   2 +
 .../apache/hadoop/registry/cli/RegistryCli.java |  25 +-
 .../registry/client/binding/JsonSerDeser.java   | 142 +++++------
 .../client/binding/RegistryTypeUtils.java       | 166 ++++++++-----
 .../registry/client/binding/RegistryUtils.java  |   7 +-
 .../client/exceptions/NoRecordException.java    |  10 +-
 .../impl/zk/RegistryOperationsService.java      |  12 +-
 .../registry/client/types/AddressTypes.java     |   2 +
 .../hadoop/registry/client/types/Endpoint.java  | 131 ++++++++---
 .../registry/client/types/ProtocolTypes.java    |   7 +-
 .../registry/client/types/ServiceRecord.java    |  26 +--
 .../client/types/ServiceRecordHeader.java       |  59 -----
 .../src/main/tla/yarnregistry.tla               |  94 ++++++--
 .../hadoop/registry/RegistryTestHelper.java     |  36 ++-
 .../client/binding/TestMarshalling.java         |  72 ++++--
 .../operations/TestRegistryOperations.java      |   5 +-
 .../src/site/markdown/registry/yarn-registry.md | 233 +++++++++++++++----
 17 files changed, 623 insertions(+), 406 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/16705780/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 0fe957c..0c5fc4c 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -740,6 +740,8 @@ Release 2.6.0 - UNRELEASED
     YARN-2677 registry punycoding of usernames doesn't fix all usernames to be
     DNS-valid (stevel)
 
+    YARN-2768 Improved Yarn Registry service record structure (stevel)
+
     ---
 
     YARN-2598 GHS should show N/A instead of null for the inaccessible information

http://git-wip-us.apache.org/repos/asf/hadoop/blob/16705780/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/cli/RegistryCli.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/cli/RegistryCli.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/cli/RegistryCli.java
index 863039e..bf2b5e5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/cli/RegistryCli.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/cli/RegistryCli.java
@@ -24,6 +24,7 @@ import java.io.PrintStream;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.List;
+import java.util.Map;
 
 import com.google.common.base.Preconditions;
 import org.apache.commons.cli.CommandLine;
@@ -174,24 +175,22 @@ public class RegistryCli extends Configured implements Tool {
 				ServiceRecord record = registry.resolve(argsList.get(1));
 
 				for (Endpoint endpoint : record.external) {
-					if ((endpoint.protocolType.equals(ProtocolTypes.PROTOCOL_WEBUI))
-							|| (endpoint.protocolType.equals(ProtocolTypes.PROTOCOL_REST))) {
-						sysout.print(" Endpoint(ProtocolType="
-								+ endpoint.protocolType + ", Api="
-								+ endpoint.api + "); Uris are: ");
-					} else {
-						sysout.print(" Endpoint(ProtocolType="
+						sysout.println(" Endpoint(ProtocolType="
 								+ endpoint.protocolType + ", Api="
 								+ endpoint.api + ");"
 								+ " Addresses(AddressType="
 								+ endpoint.addressType + ") are: ");
 
-					}
-					for (List<String> a : endpoint.addresses) {
-						sysout.print(a + " ");
-					}
-					sysout.println();
-				}
+          for (Map<String, String> address : endpoint.addresses) {
+            sysout.println("  [ ");
+            for (Map.Entry<String, String> entry : address.entrySet()) {
+              sysout.println("    " + entry.getKey()
+                             + ": \"" + entry.getValue() + "\"");
+            }
+            sysout.println("  ]");
+          }
+          sysout.println();
+        }
 				return 0;
       } catch (Exception e) {
         syserr.println(analyzeException("resolve", e, argsList));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/16705780/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/JsonSerDeser.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/JsonSerDeser.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/JsonSerDeser.java
index e086e36..af4e4f4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/JsonSerDeser.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/JsonSerDeser.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.registry.client.binding;
 
 import com.google.common.base.Preconditions;
+import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -45,8 +46,6 @@ import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
 
 /**
  * Support for marshalling objects to and from JSON.
@@ -62,30 +61,30 @@ public class JsonSerDeser<T> {
 
   private static final Logger LOG = LoggerFactory.getLogger(JsonSerDeser.class);
   private static final String UTF_8 = "UTF-8";
-  public static final String E_NO_SERVICE_RECORD = "No service record at path";
+  public static final String E_NO_DATA = "No data at path";
+  public static final String E_DATA_TOO_SHORT = "Data at path too short";
+  public static final String E_MISSING_MARKER_STRING =
+      "Missing marker string: ";
 
   private final Class<T> classType;
   private final ObjectMapper mapper;
-  private final byte[] header;
 
   /**
    * Create an instance bound to a specific type
    * @param classType class to marshall
-   * @param header byte array to use as header
    */
-  public JsonSerDeser(Class<T> classType, byte[] header) {
+  public JsonSerDeser(Class<T> classType) {
     Preconditions.checkArgument(classType != null, "null classType");
-    Preconditions.checkArgument(header != null, "null header");
     this.classType = classType;
     this.mapper = new ObjectMapper();
     mapper.configure(DeserializationConfig.Feature.FAIL_ON_UNKNOWN_PROPERTIES,
         false);
-    // make an immutable copy to keep findbugs happy.
-    byte[] h = new byte[header.length];
-    System.arraycopy(header, 0, h, 0, header.length);
-    this.header = h;
   }
 
+  /**
+   * Get the simple name of the class type to be marshalled
+   * @return the name of the class being marshalled
+   */
   public String getName() {
     return classType.getSimpleName();
   }
@@ -183,7 +182,7 @@ public class JsonSerDeser<T> {
     if (count != len) {
       throw new EOFException(path.toString() + ": read finished prematurely");
     }
-    return fromBytes(path.toString(), b, 0);
+    return fromBytes(path.toString(), b);
   }
 
   /**
@@ -206,8 +205,7 @@ public class JsonSerDeser<T> {
    * @throws IOException on any failure
    */
   private void writeJsonAsBytes(T instance,
-      DataOutputStream dataOutputStream) throws
-      IOException {
+      DataOutputStream dataOutputStream) throws IOException {
     try {
       byte[] b = toBytes(instance);
       dataOutputStream.write(b);
@@ -228,36 +226,50 @@ public class JsonSerDeser<T> {
   }
 
   /**
-   * Convert JSON To bytes, inserting the header
-   * @param instance instance to convert
-   * @return a byte array
-   * @throws IOException
+   * Deserialize from a byte array
+   * @param path path the data came from
+   * @param bytes byte array
+   * @throws IOException all problems
+   * @throws EOFException not enough data
+   * @throws InvalidRecordException if the parsing failed -the record is invalid
    */
-  public byte[] toByteswithHeader(T instance) throws IOException {
-    byte[] body = toBytes(instance);
-
-    ByteBuffer buffer = ByteBuffer.allocate(body.length + header.length);
-    buffer.put(header);
-    buffer.put(body);
-    return buffer.array();
+  public T fromBytes(String path, byte[] bytes) throws IOException,
+      InvalidRecordException {
+    return fromBytes(path, bytes, "");
   }
 
   /**
-   * Deserialize from a byte array
+   * Deserialize from a byte array, optionally checking for a marker string.
+   * <p>
+   * If the marker parameter is supplied (and not empty), then its presence
+   * will be verified before the JSON parsing takes place; it is a fast-fail
+   * check. If not found, an {@link InvalidRecordException} exception will be
+   * raised
    * @param path path the data came from
    * @param bytes byte array
-   * @return offset in the array to read from
+   * @param marker an optional string which, if set, MUST be present in the
+   * UTF-8 parsed payload.
+   * @return The parsed record
    * @throws IOException all problems
    * @throws EOFException not enough data
-   * @throws InvalidRecordException if the parsing failed -the record is invalid
+   * @throws InvalidRecordException if the JSON parsing failed.
+   * @throws NoRecordException if the data is not considered a record: either
+   * it is too short or it did not contain the marker string.
    */
-  public T fromBytes(String path, byte[] bytes, int offset) throws IOException,
-      InvalidRecordException {
-    int data = bytes.length - offset;
-    if (data <= 0) {
-      throw new EOFException("No data at " + path);
+  public T fromBytes(String path, byte[] bytes, String marker)
+      throws IOException, NoRecordException, InvalidRecordException {
+    int len = bytes.length;
+    if (len == 0 ) {
+      throw new NoRecordException(path, E_NO_DATA);
+    }
+    if (StringUtils.isNotEmpty(marker) && len < marker.length()) {
+      throw new NoRecordException(path, E_DATA_TOO_SHORT);
+    }
+    String json = new String(bytes, 0, len, UTF_8);
+    if (StringUtils.isNotEmpty(marker)
+        && !json.contains(marker)) {
+      throw new NoRecordException(path, E_MISSING_MARKER_STRING + marker);
     }
-    String json = new String(bytes, offset, data, UTF_8);
     try {
       return fromJson(json);
     } catch (JsonProcessingException e) {
@@ -266,52 +278,7 @@ public class JsonSerDeser<T> {
   }
 
   /**
-   * Read from a byte array to a type, checking the header first
-   * @param path source of data
-   * @param buffer buffer
-   * @return the parsed structure
-   * Null if the record was too short or the header did not match
-   * @throws IOException on a failure
-   * @throws NoRecordException if header checks implied there was no record
-   * @throws InvalidRecordException if record parsing failed
-   */
-  @SuppressWarnings("unchecked")
-  public T fromBytesWithHeader(String path, byte[] buffer) throws IOException {
-    int hlen = header.length;
-    int blen = buffer.length;
-    if (hlen > 0) {
-      if (blen < hlen) {
-        throw new NoRecordException(path, E_NO_SERVICE_RECORD);
-      }
-      byte[] magic = Arrays.copyOfRange(buffer, 0, hlen);
-      if (!Arrays.equals(header, magic)) {
-        LOG.debug("start of entry does not match service record header at {}",
-            path);
-        throw new NoRecordException(path, E_NO_SERVICE_RECORD);
-      }
-    }
-    return fromBytes(path, buffer, hlen);
-  }
-
-  /**
-   * Check if a buffer has a header which matches this record type
-   * @param buffer buffer
-   * @return true if there is a match
-   * @throws IOException
-   */
-  public boolean headerMatches(byte[] buffer) throws IOException {
-    int hlen = header.length;
-    int blen = buffer.length;
-    boolean matches = false;
-    if (blen > hlen) {
-      byte[] magic = Arrays.copyOfRange(buffer, 0, hlen);
-      matches = Arrays.equals(header, magic);
-    }
-    return matches;
-  }
-
-  /**
-   * Convert an object to a JSON string
+   * Convert an instance to a JSON string
    * @param instance instance to convert
    * @return a JSON string description
    * @throws JsonParseException parse problems
@@ -324,4 +291,19 @@ public class JsonSerDeser<T> {
     return mapper.writeValueAsString(instance);
   }
 
+  /**
+   * Convert an instance to a string form for output. This is a robust
+   * operation which will convert any JSON-generating exceptions into
+   * error text.
+   * @param instance non-null instance
+   * @return a JSON string
+   */
+  public String toString(T instance) {
+    Preconditions.checkArgument(instance != null, "Null instance argument");
+    try {
+      return toJson(instance);
+    } catch (IOException e) {
+      return "Failed to convert to a string: " + e;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/16705780/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryTypeUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryTypeUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryTypeUtils.java
index b4254a3..ec59d59 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryTypeUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryTypeUtils.java
@@ -22,17 +22,19 @@ import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.registry.client.exceptions.InvalidRecordException;
-import org.apache.hadoop.registry.client.types.AddressTypes;
+import static org.apache.hadoop.registry.client.types.AddressTypes.*;
 import org.apache.hadoop.registry.client.types.Endpoint;
 import org.apache.hadoop.registry.client.types.ProtocolTypes;
+import org.apache.hadoop.registry.client.types.ServiceRecord;
 
 import java.net.InetSocketAddress;
 import java.net.MalformedURLException;
 import java.net.URI;
 import java.net.URL;
 import java.util.ArrayList;
-import java.util.Arrays;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 /**
  * Static methods to work with registry types —primarily endpoints and the
@@ -94,79 +96,66 @@ public class RegistryTypeUtils {
     Preconditions.checkArgument(protocolType != null, "null protocolType");
     Preconditions.checkArgument(hostname != null, "null hostname");
     return new Endpoint(api,
-        AddressTypes.ADDRESS_HOSTNAME_AND_PORT,
+        ADDRESS_HOSTNAME_AND_PORT,
         protocolType,
-        tuplelist(hostname, Integer.toString(port)));
+        hostnamePortPair(hostname, port));
   }
 
   /**
    * Create an IPC endpoint
    * @param api API
-   * @param protobuf flag to indicate whether or not the IPC uses protocol
-   * buffers
    * @param address the address as a tuple of (hostname, port)
    * @return the new endpoint
    */
-  public static Endpoint ipcEndpoint(String api,
-      boolean protobuf, List<String> address) {
-    ArrayList<List<String>> addressList = new ArrayList<List<String>>();
-    if (address != null) {
-      addressList.add(address);
-    }
+  public static Endpoint ipcEndpoint(String api, InetSocketAddress address) {
     return new Endpoint(api,
-        AddressTypes.ADDRESS_HOSTNAME_AND_PORT,
-        protobuf ? ProtocolTypes.PROTOCOL_HADOOP_IPC_PROTOBUF
-                 : ProtocolTypes.PROTOCOL_HADOOP_IPC,
-        addressList);
+        ADDRESS_HOSTNAME_AND_PORT,
+        ProtocolTypes.PROTOCOL_HADOOP_IPC,
+        address== null ? null: hostnamePortPair(address));
   }
 
   /**
-   * Create a single-element list of tuples from the input.
-   * that is, an input ("a","b","c") is converted into a list
-   * in the form [["a","b","c"]]
-   * @param t1 tuple elements
-   * @return a list containing a single tuple
+   * Create a single entry map
+   * @param key map entry key
+   * @param val map entry value
+   * @return a 1 entry map.
    */
-  public static List<List<String>> tuplelist(String... t1) {
-    List<List<String>> outer = new ArrayList<List<String>>();
-    outer.add(tuple(t1));
-    return outer;
+  public static Map<String, String> map(String key, String val) {
+    Map<String, String> map = new HashMap<String, String>(1);
+    map.put(key, val);
+    return map;
   }
 
   /**
-   * Create a tuples from the input.
-   * that is, an input ("a","b","c") is converted into a list
-   * in the form ["a","b","c"]
-   * @param t1 tuple elements
-   * @return a single tuple as a list
+   * Create a URI
+   * @param uri value
+   * @return a 1 entry map.
    */
-  public static List<String> tuple(String... t1) {
-    return Arrays.asList(t1);
+  public static Map<String, String> uri(String uri) {
+    return map(ADDRESS_URI, uri);
   }
 
   /**
-   * Create a tuples from the input, converting all to Strings in the process
-   * that is, an input ("a", 7, true) is converted into a list
-   * in the form ["a","7,"true"]
-   * @param t1 tuple elements
-   * @return a single tuple as a list
+   * Create a (hostname, port) address pair
+   * @param hostname hostname
+   * @param port port
+   * @return a 1 entry map.
    */
-  public static List<String> tuple(Object... t1) {
-    List<String> l = new ArrayList<String>(t1.length);
-    for (Object t : t1) {
-      l.add(t.toString());
-    }
-    return l;
+  public static Map<String, String> hostnamePortPair(String hostname, int port) {
+    Map<String, String> map =
+        map(ADDRESS_HOSTNAME_FIELD, hostname);
+    map.put(ADDRESS_PORT_FIELD, Integer.toString(port));
+    return map;
   }
 
   /**
-   * Convert a socket address pair into a string tuple, (host, port).
-   * TODO JDK7: move to InetAddress.getHostString() to avoid DNS lookups.
-   * @param address an address
-   * @return an element for the address list
+   * Create a (hostname, port) address pair
+   * @param address socket address whose hostname and port are used for the
+   * generated address.
+   * @return a 1 entry map.
    */
-  public static List<String> marshall(InetSocketAddress address) {
-    return tuple(address.getHostName(), address.getPort());
+  public static Map<String, String> hostnamePortPair(InetSocketAddress address) {
+    return hostnamePortPair(address.getHostName(), address.getPort());
   }
 
   /**
@@ -199,25 +188,37 @@ public class RegistryTypeUtils {
     if (epr == null) {
       return null;
     }
-    requireAddressType(AddressTypes.ADDRESS_URI, epr);
-    List<List<String>> addresses = epr.addresses;
+    requireAddressType(ADDRESS_URI, epr);
+    List<Map<String, String>> addresses = epr.addresses;
     if (addresses.size() < 1) {
       throw new InvalidRecordException(epr.toString(),
           "No addresses in endpoint");
     }
     List<String> results = new ArrayList<String>(addresses.size());
-    for (List<String> address : addresses) {
-      if (address.size() != 1) {
-        throw new InvalidRecordException(epr.toString(),
-            "Address payload invalid: wrong element count: " +
-            address.size());
-      }
-      results.add(address.get(0));
+    for (Map<String, String> address : addresses) {
+      results.add(getAddressField(address, ADDRESS_URI));
     }
     return results;
   }
 
   /**
+   * Get a specific field from an address -raising an exception if
+   * the field is not present
+   * @param address address to query
+   * @param field field to resolve
+   * @return the resolved value. Guaranteed to be non-null.
+   * @throws InvalidRecordException if the field did not resolve
+   */
+  public static String getAddressField(Map<String, String> address,
+      String field) throws InvalidRecordException {
+    String val = address.get(field);
+    if (val == null) {
+      throw new InvalidRecordException("", "Missing address field: " + field);
+    }
+    return val;
+  }
+
+  /**
    * Get the address URLs. Guranteed to return at least one address.
    * @param epr endpoint
    * @return the address as a URL
@@ -237,4 +238,53 @@ public class RegistryTypeUtils {
     }
     return results;
   }
+
+  /**
+   * Validate the record by checking for null fields and other invalid
+   * conditions
+   * @param path path for exceptions
+   * @param record record to validate. May be null
+   * @throws InvalidRecordException on invalid entries
+   */
+  public static void validateServiceRecord(String path, ServiceRecord record)
+      throws InvalidRecordException {
+    if (record == null) {
+      throw new InvalidRecordException(path, "Null record");
+    }
+    if (!ServiceRecord.RECORD_TYPE.equals(record.type)) {
+      throw new InvalidRecordException(path,
+          "invalid record type field: \"" + record.type + "\"");
+    }
+
+    if (record.external != null) {
+      for (Endpoint endpoint : record.external) {
+        validateEndpoint(path, endpoint);
+      }
+    }
+    if (record.internal != null) {
+      for (Endpoint endpoint : record.internal) {
+        validateEndpoint(path, endpoint);
+      }
+    }
+  }
+
+  /**
+   * Validate the endpoint by checking for null fields and other invalid
+   * conditions
+   * @param path path for exceptions
+   * @param endpoint endpoint to validate. May be null
+   * @throws InvalidRecordException on invalid entries
+   */
+  public static void validateEndpoint(String path, Endpoint endpoint)
+      throws InvalidRecordException {
+    if (endpoint == null) {
+      throw new InvalidRecordException(path, "Null endpoint");
+    }
+    try {
+      endpoint.validate();
+    } catch (RuntimeException e) {
+      throw new InvalidRecordException(path, e.toString());
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/16705780/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryUtils.java
index 8caf400..68dc84e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryUtils.java
@@ -33,7 +33,6 @@ import org.apache.hadoop.registry.client.exceptions.NoRecordException;
 import org.apache.hadoop.registry.client.impl.zk.RegistryInternalConstants;
 import org.apache.hadoop.registry.client.types.RegistryPathStatus;
 import org.apache.hadoop.registry.client.types.ServiceRecord;
-import org.apache.hadoop.registry.client.types.ServiceRecordHeader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -314,7 +313,7 @@ public class RegistryUtils {
       Collection<RegistryPathStatus> stats) throws IOException {
     Map<String, ServiceRecord> results = new HashMap<String, ServiceRecord>(stats.size());
     for (RegistryPathStatus stat : stats) {
-      if (stat.size > ServiceRecordHeader.getLength()) {
+      if (stat.size > ServiceRecord.RECORD_TYPE.length()) {
         // maybe has data
         String path = join(parentpath, stat.path);
         try {
@@ -344,7 +343,6 @@ public class RegistryUtils {
    * <p>
    * @param operations operation support for fetches
    * @param parentpath path of the parent of all the entries
-   * @param stats a map of name:value mappings.
    * @return a possibly empty map of fullpath:record.
    * @throws IOException for any IO Operation that wasn't ignored.
    */
@@ -362,7 +360,6 @@ public class RegistryUtils {
    * <p>
    * @param operations operation support for fetches
    * @param parentpath path of the parent of all the entries
-   * @param stats a map of name:value mappings.
    * @return a possibly empty map of fullpath:record.
    * @throws IOException for any IO Operation that wasn't ignored.
    */
@@ -382,7 +379,7 @@ public class RegistryUtils {
    */
   public static class ServiceRecordMarshal extends JsonSerDeser<ServiceRecord> {
     public ServiceRecordMarshal() {
-      super(ServiceRecord.class, ServiceRecordHeader.getData());
+      super(ServiceRecord.class);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/16705780/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/NoRecordException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/NoRecordException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/NoRecordException.java
index 160433f..b81b9d4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/NoRecordException.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/NoRecordException.java
@@ -21,17 +21,11 @@ package org.apache.hadoop.registry.client.exceptions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.registry.client.types.ServiceRecord;
-import org.apache.hadoop.registry.client.types.ServiceRecordHeader;
 
 /**
  * Raised if there is no {@link ServiceRecord} resolved at the end
- * of the specified path, for reasons such as:
- * <ul>
- *   <li>There wasn't enough data to contain a Service Record.</li>
- *   <li>The start of the data did not match the {@link ServiceRecordHeader}
- *   header.</li>
- * </ul>
- *
+ * of the specified path.
+ * <p>
  * There may be valid data of some form at the end of the path, but it does
  * not appear to be a Service Record.
  */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/16705780/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistryOperationsService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistryOperationsService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistryOperationsService.java
index 7c01bdf..271ab25 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistryOperationsService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistryOperationsService.java
@@ -24,9 +24,11 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.registry.client.api.BindFlags;
 import org.apache.hadoop.registry.client.api.RegistryOperations;
 
+import org.apache.hadoop.registry.client.binding.RegistryTypeUtils;
 import org.apache.hadoop.registry.client.binding.RegistryUtils;
 import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
 import org.apache.hadoop.registry.client.exceptions.InvalidPathnameException;
+import org.apache.hadoop.registry.client.exceptions.NoRecordException;
 import org.apache.hadoop.registry.client.types.RegistryPathStatus;
 import org.apache.hadoop.registry.client.types.ServiceRecord;
 import org.apache.zookeeper.CreateMode;
@@ -103,10 +105,12 @@ public class RegistryOperationsService extends CuratorService
       int flags) throws IOException {
     Preconditions.checkArgument(record != null, "null record");
     validatePath(path);
+    // validate the record before putting it
+    RegistryTypeUtils.validateServiceRecord(path, record);
     LOG.info("Bound at {} : {}", path, record);
 
     CreateMode mode = CreateMode.PERSISTENT;
-    byte[] bytes = serviceRecordMarshal.toByteswithHeader(record);
+    byte[] bytes = serviceRecordMarshal.toBytes(record);
     zkSet(path, mode, bytes, getClientAcls(),
         ((flags & BindFlags.OVERWRITE) != 0));
   }
@@ -114,7 +118,11 @@ public class RegistryOperationsService extends CuratorService
   @Override
   public ServiceRecord resolve(String path) throws IOException {
     byte[] bytes = zkRead(path);
-    return serviceRecordMarshal.fromBytesWithHeader(path, bytes);
+
+    ServiceRecord record = serviceRecordMarshal.fromBytes(path,
+        bytes, ServiceRecord.RECORD_TYPE);
+    RegistryTypeUtils.validateServiceRecord(path, record);
+    return record;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/16705780/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/AddressTypes.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/AddressTypes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/AddressTypes.java
index 192819c..36dbf0c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/AddressTypes.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/AddressTypes.java
@@ -38,6 +38,8 @@ public interface AddressTypes {
    * </pre>
    */
   public static final String ADDRESS_HOSTNAME_AND_PORT = "host/port";
+  public static final String ADDRESS_HOSTNAME_FIELD = "host";
+  public static final String ADDRESS_PORT_FIELD = "port";
 
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/16705780/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/Endpoint.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/Endpoint.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/Endpoint.java
index 51418d9..e4effb4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/Endpoint.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/Endpoint.java
@@ -21,14 +21,16 @@ package org.apache.hadoop.registry.client.types;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.registry.client.binding.JsonSerDeser;
 import org.apache.hadoop.registry.client.binding.RegistryTypeUtils;
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
 import java.net.URI;
 import java.util.ArrayList;
-import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 /**
  * Description of a single service/component endpoint.
@@ -67,7 +69,7 @@ public final class Endpoint implements Cloneable {
   /**
    * a list of address tuples —tuples whose format depends on the address type
    */
-  public List<List<String>> addresses;
+  public List<Map<String, String>> addresses;
 
   /**
    * Create an empty instance.
@@ -84,10 +86,11 @@ public final class Endpoint implements Cloneable {
     this.api = that.api;
     this.addressType = that.addressType;
     this.protocolType = that.protocolType;
-    this.addresses = new ArrayList<List<String>>(that.addresses.size());
-    for (List<String> address : addresses) {
-      List<String> addr2 = new ArrayList<String>(address.size());
-      Collections.copy(address, addr2);
+    this.addresses = newAddresses(that.addresses.size());
+    for (Map<String, String> address : that.addresses) {
+      Map<String, String> addr2 = new HashMap<String, String>(address.size());
+      addr2.putAll(address);
+      addresses.add(addr2);
     }
   }
 
@@ -101,17 +104,83 @@ public final class Endpoint implements Cloneable {
   public Endpoint(String api,
       String addressType,
       String protocolType,
-      List<List<String>> addrs) {
+      List<Map<String, String>> addrs) {
     this.api = api;
     this.addressType = addressType;
     this.protocolType = protocolType;
-    this.addresses = new ArrayList<List<String>>();
+    this.addresses = newAddresses(0);
     if (addrs != null) {
       addresses.addAll(addrs);
     }
   }
 
   /**
+   * Build an endpoint with an empty address list
+   * @param api API name
+   * @param addressType address type
+   * @param protocolType protocol type
+   */
+  public Endpoint(String api,
+      String addressType,
+      String protocolType) {
+    this.api = api;
+    this.addressType = addressType;
+    this.protocolType = protocolType;
+    this.addresses = newAddresses(0);
+  }
+
+  /**
+   * Build an endpoint with a single address entry.
+   * <p>
+   * This constructor is superfluous given the varags constructor is equivalent
+   * for a single element argument. However, type-erasure in java generics
+   * causes javac to warn about unchecked generic array creation. This
+   * constructor, which represents the common "one address" case, does
+   * not generate compile-time warnings.
+   * @param api API name
+   * @param addressType address type
+   * @param protocolType protocol type
+   * @param addr address. May be null —in which case it is not added
+   */
+  public Endpoint(String api,
+      String addressType,
+      String protocolType,
+      Map<String, String> addr) {
+    this(api, addressType, protocolType);
+    if (addr != null) {
+      addresses.add(addr);
+    }
+  }
+
+  /**
+   * Build an endpoint with a list of addresses
+   * @param api API name
+   * @param addressType address type
+   * @param protocolType protocol type
+   * @param addrs addresses. Null elements will be skipped
+   */
+  public Endpoint(String api,
+      String addressType,
+      String protocolType,
+      Map<String, String>...addrs) {
+    this(api, addressType, protocolType);
+    for (Map<String, String> addr : addrs) {
+      if (addr!=null) {
+        addresses.add(addr);
+      }
+    }
+  }
+
+  /**
+   * Create a new address structure of the requested size
+   * @param size size to create
+   * @return the new list
+   */
+  private List<Map<String, String>> newAddresses(int size) {
+    return new ArrayList<Map<String, String>>(size);
+  }
+
+  /**
    * Build an endpoint from a list of URIs; each URI
    * is ASCII-encoded and added to the list of addresses.
    * @param api API name
@@ -125,40 +194,16 @@ public final class Endpoint implements Cloneable {
     this.addressType = AddressTypes.ADDRESS_URI;
 
     this.protocolType = protocolType;
-    List<List<String>> addrs = new ArrayList<List<String>>(uris.length);
+    List<Map<String, String>> addrs = newAddresses(uris.length);
     for (URI uri : uris) {
-      addrs.add(RegistryTypeUtils.tuple(uri.toString()));
+      addrs.add(RegistryTypeUtils.uri(uri.toString()));
     }
     this.addresses = addrs;
   }
 
   @Override
   public String toString() {
-    final StringBuilder sb = new StringBuilder("Endpoint{");
-    sb.append("api='").append(api).append('\'');
-    sb.append(", addressType='").append(addressType).append('\'');
-    sb.append(", protocolType='").append(protocolType).append('\'');
-
-    sb.append(", addresses=");
-    if (addresses != null) {
-      sb.append("[ ");
-      for (List<String> address : addresses) {
-        sb.append("[ ");
-        if (address == null) {
-          sb.append("NULL entry in address list");
-        } else {
-          for (String elt : address) {
-            sb.append('"').append(elt).append("\" ");
-          }
-        }
-        sb.append("] ");
-      };
-      sb.append("] ");
-    } else {
-      sb.append("(null) ");
-    }
-    sb.append('}');
-    return sb.toString();
+      return marshalToString.toString(this);
   }
 
   /**
@@ -173,7 +218,7 @@ public final class Endpoint implements Cloneable {
     Preconditions.checkNotNull(addressType, "null addressType field");
     Preconditions.checkNotNull(protocolType, "null protocolType field");
     Preconditions.checkNotNull(addresses, "null addresses field");
-    for (List<String> address : addresses) {
+    for (Map<String, String> address : addresses) {
       Preconditions.checkNotNull(address, "null element in address");
     }
   }
@@ -184,7 +229,19 @@ public final class Endpoint implements Cloneable {
    * @throws CloneNotSupportedException
    */
   @Override
-  protected Object clone() throws CloneNotSupportedException {
+  public Object clone() throws CloneNotSupportedException {
     return super.clone();
   }
+
+
+  /**
+   * Static instance of service record marshalling
+   */
+  private static class Marshal extends JsonSerDeser<Endpoint> {
+    private Marshal() {
+      super(Endpoint.class);
+    }
+  }
+
+  private static final Marshal marshalToString = new Marshal();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/16705780/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ProtocolTypes.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ProtocolTypes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ProtocolTypes.java
index f225cf0..b836b00 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ProtocolTypes.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ProtocolTypes.java
@@ -34,16 +34,11 @@ public interface ProtocolTypes {
   String PROTOCOL_FILESYSTEM = "hadoop/filesystem";
 
   /**
-   * Classic Hadoop IPC : {@value}.
+   * Hadoop IPC,  "classic" or protobuf : {@value}.
    */
   String PROTOCOL_HADOOP_IPC = "hadoop/IPC";
 
   /**
-   * Hadoop protocol buffers IPC: {@value}.
-   */
-  String PROTOCOL_HADOOP_IPC_PROTOBUF = "hadoop/protobuf";
-
-  /**
    * Corba IIOP: {@value}.
    */
   String PROTOCOL_IIOP = "IIOP";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/16705780/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ServiceRecord.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ServiceRecord.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ServiceRecord.java
index 378127f..9403d31 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ServiceRecord.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ServiceRecord.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.registry.client.types;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.registry.client.exceptions.InvalidRecordException;
 import org.codehaus.jackson.annotate.JsonAnyGetter;
 import org.codehaus.jackson.annotate.JsonAnySetter;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
@@ -41,6 +42,17 @@ import java.util.Map;
 public class ServiceRecord implements Cloneable {
 
   /**
+   * A type string which MUST be in the serialized json. This permits
+   * fast discarding of invalid entries
+   */
+  public static final String RECORD_TYPE = "JSONServiceRecord";
+
+  /**
+   * The type field. This must be the string {@link #RECORD_TYPE}
+   */
+  public String type = RECORD_TYPE;
+
+  /**
    * Description string
    */
   public String description;
@@ -233,17 +245,5 @@ public class ServiceRecord implements Cloneable {
     return super.clone();
   }
 
-  /**
-   * Validate the record by checking for null fields and other invalid
-   * conditions
-   * @throws NullPointerException if a field is null when it
-   * MUST be set.
-   * @throws RuntimeException on invalid entries
-   */
-  public void validate() {
-    for (Endpoint endpoint : external) {
-      Preconditions.checkNotNull("null endpoint", endpoint);
-      endpoint.validate();
-    }
-  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/16705780/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ServiceRecordHeader.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ServiceRecordHeader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ServiceRecordHeader.java
deleted file mode 100644
index 2f75dba..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ServiceRecordHeader.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * 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.registry.client.types;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * Service record header; access to the byte array kept private
- * to avoid findbugs warnings of mutability
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class ServiceRecordHeader {
-  /**
-   * Header of a service record:  "jsonservicerec"
-   * By making this over 12 bytes long, we can auto-determine which entries
-   * in a listing are too short to contain a record without getting their data
-   */
-  private static final byte[] RECORD_HEADER = {
-      'j', 's', 'o', 'n',
-      's', 'e', 'r', 'v', 'i', 'c', 'e',
-      'r', 'e', 'c'
-  };
-
-  /**
-   * Get the length of the record header
-   * @return the header length
-   */
-  public static int getLength() {
-    return RECORD_HEADER.length;
-  }
-
-  /**
-   * Get a clone of the record header
-   * @return the new record header.
-   */
-  public static byte[] getData() {
-    byte[] h = new byte[RECORD_HEADER.length];
-    System.arraycopy(RECORD_HEADER, 0, h, 0, RECORD_HEADER.length);
-    return h;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/16705780/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/tla/yarnregistry.tla
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/tla/yarnregistry.tla b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/tla/yarnregistry.tla
index 1c19ade..a950475 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/tla/yarnregistry.tla
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/tla/yarnregistry.tla
@@ -4,6 +4,7 @@ EXTENDS FiniteSets, Sequences, Naturals, TLC
 
 
 (*
+============================================================================
  * 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
@@ -19,6 +20,7 @@ EXTENDS FiniteSets, Sequences, Naturals, TLC
  * 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.
+============================================================================
  *)
 
 (*
@@ -71,13 +73,22 @@ CONSTANTS
     MknodeActions    \* all possible mkdir actions
 
 
+ASSUME PathChars \in STRING
+ASSUME Paths \in STRING
+
+(* Data in records is JSON, hence a string *)
+ASSUME Data \in STRING
+
+----------------------------------------------------------------------------------------
 
 (* the registry*)
 VARIABLE registry
 
+
 (* Sequence of actions to apply to the registry *)
 VARIABLE actions
 
+
 ----------------------------------------------------------------------------------------
 (* Tuple of all variables.  *)
 
@@ -92,7 +103,6 @@ vars == << registry, actions >>
 
 (* Persistence policy *)
 PersistPolicySet == {
-    "",                      \* Undefined; field not present. PERMANENT is implied.
     "permanent",            \* persists until explicitly removed
     "application",          \* persists until the application finishes
     "application-attempt",  \* persists until the application attempt finishes
@@ -104,7 +114,6 @@ TypeInvariant ==
     /\ \A p \in PersistPolicies: p \in PersistPolicySet
 
 
-
 ----------------------------------------------------------------------------------------
 
 
@@ -129,6 +138,14 @@ RegistryEntry == [
     ]
 
 
+(* Define the set of all string to string mappings *)
+
+StringMap == [
+  STRING |-> STRING
+]
+
+
+
 (*
     An endpoint in a service record
 *)
@@ -140,21 +157,14 @@ Endpoint == [
     addresses: Addresses
 ]
 
-(* Attributes are the set of all string to string mappings *)
-
-Attributes == [
-STRING |-> STRING
-]
-
 (*
     A service record
 *)
 ServiceRecord == [
-    \* ID -used when applying the persistence policy
-    yarn_id: STRING,
 
-    \* the persistence policy
-    yarn_persistence: PersistPolicySet,
+    \* This MUST be present: if it is not then the data is not a service record
+    \* This permits shortcut scan & reject of byte arrays without parsing
+    type: "JSONServiceRecord",
 
     \*A description
     description: STRING,
@@ -166,9 +176,34 @@ ServiceRecord == [
     internal: Endpoints,
 
     \* Attributes are a function
-    attributes: Attributes
+    attributes: StringMap
 ]
 
+----------------------------------------------------------------------------------------
+
+(*
+ There is an operation serialize whose internals are not defined,
+ Which converts the service records to JSON
+ *)
+
+CONSTANT serialize(_)
+
+(* A function which returns true iff the byte stream is considered a valid service record. *)
+CONSTANT containsServiceRecord(_)
+
+(* A function to deserialize a string to JSON *)
+CONSTANT deserialize(_)
+
+ASSUME \A json \in STRING: containsServiceRecord(json) \in BOOLEAN
+
+(* Records can be serialized *)
+ASSUME \A r \in ServiceRecord : serialize(r) \in STRING /\ containsServiceRecord(serialize(r))
+
+(* All strings for which containsServiceRecord() holds can be deserialized *)
+ASSUME \A json \in STRING: containsServiceRecord(json) => deserialize(json) \in ServiceRecord
+
+
+
 
 ----------------------------------------------------------------------------------------
 
@@ -304,8 +339,8 @@ validRegistry(R) ==
         \* an entry must be the root entry or have a parent entry
         /\ \A e \in R: isRootEntry(e) \/ exists(R, parent(e.path))
 
-        \* If the entry has data, it must be a service record
-        /\ \A e \in R: (e.data = << >> \/ e.data \in ServiceRecords)
+        \* If the entry has data, it must contain a service record
+        /\ \A e \in R: (e.data = << >> \/ containsServiceRecord(e.data))
 
 
 ----------------------------------------------------------------------------------------
@@ -336,13 +371,13 @@ mknode() adds a new empty entry where there was none before, iff
 *)
 
 mknodeSimple(R, path) ==
-    LET record == [ path |-> path, data |-> <<>>  ]
+    LET entry == [ path |-> path, data |-> <<>>  ]
     IN  \/ exists(R, path)
-        \/ (exists(R, parent(path))  /\ canBind(R, record) /\ (R' = R \union {record} ))
+        \/ (exists(R, parent(path))  /\ canBind(R, entry) /\ (R' = R \union {entry} ))
 
 
 (*
-For all parents, the mknodeSimpl() criteria must apply.
+For all parents, the mknodeSimple() criteria must apply.
 This could be defined recursively, though as TLA+ does not support recursion,
 an alternative is required
 
@@ -350,7 +385,8 @@ an alternative is required
 Because this specification is declaring the final state of a operation, not
 the implemental, all that is needed is to describe those parents.
 
-It declares that the mkdirSimple state applies to the path and all its parents in the set R'
+It declares that the mknodeSimple() state applies to the path and all
+its parents in the set R'
 
 *)
 mknodeWithParents(R, path) ==
@@ -402,7 +438,7 @@ purge(R, path, id, persistence) ==
          => recursiveDelete(R, p2.path)
 
 (*
-resolveRecord() resolves the record at a path or fails.
+resolveEntry() resolves the record entry at a path or fails.
 
 It relies on the fact that if the cardinality of a set is 1, then the CHOOSE operator
 is guaranteed to return the single entry of that set, iff the choice predicate holds.
@@ -411,19 +447,28 @@ Using a predicate of TRUE, it always succeeds, so this function selects
 the sole entry of the resolve operation.
 *)
 
-resolveRecord(R, path) ==
+resolveEntry(R, path) ==
     LET l == resolve(R, path) IN
         /\ Cardinality(l) = 1
         /\ CHOOSE e \in l : TRUE
 
 (*
+ Resolve a record by resolving the entry and deserializing the result
+ *)
+resolveRecord(R, path) ==
+    deserialize(resolveEntry(R, path))
+
+
+(*
 The specific action of putting an entry into a record includes validating the record
 *)
 
 validRecordToBind(path, record) ==
       \* The root entry must have permanent persistence
-     isRootPath(path) => (record.attributes["yarn:persistence"] = "permanent"
-     \/ record.attributes["yarn:persistence"] = "")
+     isRootPath(path) => (
+        record.attributes["yarn:persistence"] = "permanent"
+        \/ record.attributes["yarn:persistence"]
+        \/ record.attributes["yarn:persistence"] = {})
 
 
 (*
@@ -432,13 +477,12 @@ marshalled as the data in the entry
  *)
 bindRecord(R, path, record) ==
     /\ validRecordToBind(path, record)
-    /\ bind(R, [path |-> path, data |-> record])
+    /\ bind(R, [path |-> path, data |-> serialize(record)])
 
 
 ----------------------------------------------------------------------------------------
 
 
-
 (*
 The action queue can only contain one of the sets of action types, and
 by giving each a unique name, those sets are guaranteed to be disjoint

http://git-wip-us.apache.org/repos/asf/hadoop/blob/16705780/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/RegistryTestHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/RegistryTestHelper.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/RegistryTestHelper.java
index 460ecad..91602e1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/RegistryTestHelper.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/RegistryTestHelper.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.registry;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.registry.client.api.RegistryConstants;
 import org.apache.hadoop.registry.client.binding.RegistryUtils;
 import org.apache.hadoop.registry.client.binding.RegistryTypeUtils;
@@ -46,11 +45,7 @@ import java.net.URISyntaxException;
 import java.util.List;
 import java.util.Map;
 
-import static org.apache.hadoop.registry.client.binding.RegistryTypeUtils.inetAddrEndpoint;
-import static org.apache.hadoop.registry.client.binding.RegistryTypeUtils.ipcEndpoint;
-import static org.apache.hadoop.registry.client.binding.RegistryTypeUtils.restEndpoint;
-import static org.apache.hadoop.registry.client.binding.RegistryTypeUtils.tuple;
-import static org.apache.hadoop.registry.client.binding.RegistryTypeUtils.webEndpoint;
+import static org.apache.hadoop.registry.client.binding.RegistryTypeUtils.*;
 
 /**
  * This is a set of static methods to aid testing the registry operations.
@@ -61,18 +56,18 @@ public class RegistryTestHelper extends Assert {
   public static final String SC_HADOOP = "org-apache-hadoop";
   public static final String USER = "devteam/";
   public static final String NAME = "hdfs";
-  public static final String API_WEBHDFS = "org_apache_hadoop_namenode_webhdfs";
-  public static final String API_HDFS = "org_apache_hadoop_namenode_dfs";
+  public static final String API_WEBHDFS = "classpath:org.apache.hadoop.namenode.webhdfs";
+  public static final String API_HDFS = "classpath:org.apache.hadoop.namenode.dfs";
   public static final String USERPATH = RegistryConstants.PATH_USERS + USER;
   public static final String PARENT_PATH = USERPATH + SC_HADOOP + "/";
   public static final String ENTRY_PATH = PARENT_PATH + NAME;
-  public static final String NNIPC = "nnipc";
-  public static final String IPC2 = "IPC2";
+  public static final String NNIPC = "uuid:423C2B93-C927-4050-AEC6-6540E6646437";
+  public static final String IPC2 = "uuid:0663501D-5AD3-4F7E-9419-52F5D6636FCF";
   private static final Logger LOG =
       LoggerFactory.getLogger(RegistryTestHelper.class);
-  public static final String KTUTIL = "ktutil";
   private static final RegistryUtils.ServiceRecordMarshal recordMarshal =
       new RegistryUtils.ServiceRecordMarshal();
+  public static final String HTTP_API = "http://";
 
   /**
    * Assert the path is valid by ZK rules
@@ -148,9 +143,9 @@ public class RegistryTestHelper extends Assert {
     assertEquals(API_WEBHDFS, webhdfs.api);
     assertEquals(AddressTypes.ADDRESS_URI, webhdfs.addressType);
     assertEquals(ProtocolTypes.PROTOCOL_REST, webhdfs.protocolType);
-    List<List<String>> addressList = webhdfs.addresses;
-    List<String> url = addressList.get(0);
-    String addr = url.get(0);
+    List<Map<String, String>> addressList = webhdfs.addresses;
+    Map<String, String> url = addressList.get(0);
+    String addr = url.get("uri");
     assertTrue(addr.contains("http"));
     assertTrue(addr.contains(":8020"));
 
@@ -159,8 +154,9 @@ public class RegistryTestHelper extends Assert {
         nnipc.protocolType);
 
     Endpoint ipc2 = findEndpoint(record, IPC2, false, 1,2);
+    assertNotNull(ipc2);
 
-    Endpoint web = findEndpoint(record, "web", true, 1, 1);
+    Endpoint web = findEndpoint(record, HTTP_API, true, 1, 1);
     assertEquals(1, web.addresses.size());
     assertEquals(1, web.addresses.get(0).size());
   }
@@ -275,14 +271,14 @@ public class RegistryTestHelper extends Assert {
   public static void addSampleEndpoints(ServiceRecord entry, String hostname)
       throws URISyntaxException {
     assertNotNull(hostname);
-    entry.addExternalEndpoint(webEndpoint("web",
+    entry.addExternalEndpoint(webEndpoint(HTTP_API,
         new URI("http", hostname + ":80", "/")));
     entry.addExternalEndpoint(
         restEndpoint(API_WEBHDFS,
             new URI("http", hostname + ":8020", "/")));
 
-    Endpoint endpoint = ipcEndpoint(API_HDFS, true, null);
-    endpoint.addresses.add(tuple(hostname, "8030"));
+    Endpoint endpoint = ipcEndpoint(API_HDFS, null);
+    endpoint.addresses.add(RegistryTypeUtils.hostnamePortPair(hostname, 8030));
     entry.addInternalEndpoint(endpoint);
     InetSocketAddress localhost = new InetSocketAddress("localhost", 8050);
     entry.addInternalEndpoint(
@@ -290,9 +286,7 @@ public class RegistryTestHelper extends Assert {
             8050));
     entry.addInternalEndpoint(
         RegistryTypeUtils.ipcEndpoint(
-            IPC2,
-            true,
-            RegistryTypeUtils.marshall(localhost)));
+            IPC2, localhost));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/16705780/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/client/binding/TestMarshalling.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/client/binding/TestMarshalling.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/client/binding/TestMarshalling.java
index 14e3b1f..f1814d3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/client/binding/TestMarshalling.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/client/binding/TestMarshalling.java
@@ -19,9 +19,9 @@
 package org.apache.hadoop.registry.client.binding;
 
 import org.apache.hadoop.registry.RegistryTestHelper;
+import org.apache.hadoop.registry.client.exceptions.InvalidRecordException;
 import org.apache.hadoop.registry.client.exceptions.NoRecordException;
 import org.apache.hadoop.registry.client.types.ServiceRecord;
-import org.apache.hadoop.registry.client.types.ServiceRecordHeader;
 import org.apache.hadoop.registry.client.types.yarn.PersistencePolicies;
 import org.junit.BeforeClass;
 import org.junit.Rule;
@@ -31,8 +31,6 @@ import org.junit.rules.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.EOFException;
-
 /**
  * Test record marshalling
  */
@@ -44,6 +42,7 @@ public class TestMarshalling extends RegistryTestHelper {
   public final Timeout testTimeout = new Timeout(10000);
   @Rule
   public TestName methodName = new TestName();
+
   private static RegistryUtils.ServiceRecordMarshal marshal;
 
   @BeforeClass
@@ -55,42 +54,65 @@ public class TestMarshalling extends RegistryTestHelper {
   public void testRoundTrip() throws Throwable {
     String persistence = PersistencePolicies.PERMANENT;
     ServiceRecord record = createRecord(persistence);
-    record.set("customkey","customvalue");
-    record.set("customkey2","customvalue2");
+    record.set("customkey", "customvalue");
+    record.set("customkey2", "customvalue2");
+    RegistryTypeUtils.validateServiceRecord("", record);
     LOG.info(marshal.toJson(record));
     byte[] bytes = marshal.toBytes(record);
-    ServiceRecord r2 = marshal.fromBytes("", bytes, 0);
+    ServiceRecord r2 = marshal.fromBytes("", bytes);
     assertMatches(record, r2);
+    RegistryTypeUtils.validateServiceRecord("", r2);
   }
 
-  @Test
-  public void testRoundTripHeaders() throws Throwable {
-    ServiceRecord record = createRecord(PersistencePolicies.CONTAINER);
-    byte[] bytes = marshal.toByteswithHeader(record);
-    ServiceRecord r2 = marshal.fromBytesWithHeader("", bytes);
-    assertMatches(record, r2);
 
+  @Test(expected = NoRecordException.class)
+  public void testUnmarshallNoData() throws Throwable {
+    marshal.fromBytes("src", new byte[]{});
   }
 
   @Test(expected = NoRecordException.class)
-  public void testRoundTripBadHeaders() throws Throwable {
-    ServiceRecord record = createRecord(PersistencePolicies.APPLICATION);
-    byte[] bytes = marshal.toByteswithHeader(record);
-    bytes[1] = 0x01;
-    marshal.fromBytesWithHeader("src", bytes);
+  public void testUnmarshallNotEnoughData() throws Throwable {
+    // this is nominally JSON -but without the service record header
+    marshal.fromBytes("src", new byte[]{'{','}'}, ServiceRecord.RECORD_TYPE);
+  }
+
+  @Test(expected = InvalidRecordException.class)
+  public void testUnmarshallNoBody() throws Throwable {
+    byte[] bytes = "this is not valid JSON at all and should fail".getBytes();
+    marshal.fromBytes("src", bytes);
+  }
+
+  @Test(expected = InvalidRecordException.class)
+  public void testUnmarshallWrongType() throws Throwable {
+    byte[] bytes = "{'type':''}".getBytes();
+    ServiceRecord serviceRecord = marshal.fromBytes("marshalling", bytes);
+    RegistryTypeUtils.validateServiceRecord("validating", serviceRecord);
   }
 
   @Test(expected = NoRecordException.class)
-  public void testUnmarshallHeaderTooShort() throws Throwable {
-    marshal.fromBytesWithHeader("src", new byte[]{'a'});
+  public void testUnmarshallWrongLongType() throws Throwable {
+    ServiceRecord record = new ServiceRecord();
+    record.type = "ThisRecordHasALongButNonMatchingType";
+    byte[] bytes = marshal.toBytes(record);
+    ServiceRecord serviceRecord = marshal.fromBytes("marshalling",
+        bytes, ServiceRecord.RECORD_TYPE);
   }
 
-  @Test(expected = EOFException.class)
-  public void testUnmarshallNoBody() throws Throwable {
-    byte[] bytes = ServiceRecordHeader.getData();
-    marshal.fromBytesWithHeader("src", bytes);
+  @Test(expected = NoRecordException.class)
+  public void testUnmarshallNoType() throws Throwable {
+    ServiceRecord record = new ServiceRecord();
+    record.type = "NoRecord";
+    byte[] bytes = marshal.toBytes(record);
+    ServiceRecord serviceRecord = marshal.fromBytes("marshalling",
+        bytes, ServiceRecord.RECORD_TYPE);
   }
 
+  @Test(expected = InvalidRecordException.class)
+  public void testRecordValidationWrongType() throws Throwable {
+    ServiceRecord record = new ServiceRecord();
+    record.type = "NotAServiceRecordType";
+    RegistryTypeUtils.validateServiceRecord("validating", record);
+  }
 
   @Test
   public void testUnknownFieldsRoundTrip() throws Throwable {
@@ -102,8 +124,8 @@ public class TestMarshalling extends RegistryTestHelper {
     assertEquals("2", record.get("intval"));
     assertNull(record.get("null"));
     assertEquals("defval", record.get("null", "defval"));
-    byte[] bytes = marshal.toByteswithHeader(record);
-    ServiceRecord r2 = marshal.fromBytesWithHeader("", bytes);
+    byte[] bytes = marshal.toBytes(record);
+    ServiceRecord r2 = marshal.fromBytes("", bytes);
     assertEquals("value", r2.get("key"));
     assertEquals("2", r2.get("intval"));
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/16705780/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/operations/TestRegistryOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/operations/TestRegistryOperations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/operations/TestRegistryOperations.java
index 7a7f88c..853d7f1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/operations/TestRegistryOperations.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/operations/TestRegistryOperations.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException;
 import org.apache.hadoop.fs.PathNotFoundException;
 import org.apache.hadoop.registry.AbstractRegistryTest;
 import org.apache.hadoop.registry.client.api.BindFlags;
+import org.apache.hadoop.registry.client.binding.RegistryTypeUtils;
 import org.apache.hadoop.registry.client.binding.RegistryUtils;
 import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
 import org.apache.hadoop.registry.client.exceptions.NoRecordException;
@@ -91,10 +92,8 @@ public class TestRegistryOperations extends AbstractRegistryTest {
             childStats.values());
     assertEquals(1, records.size());
     ServiceRecord record = records.get(ENTRY_PATH);
-    assertNotNull(record);
-    record.validate();
+    RegistryTypeUtils.validateServiceRecord(ENTRY_PATH, record);
     assertMatches(written, record);
-
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/16705780/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/registry/yarn-registry.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/registry/yarn-registry.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/registry/yarn-registry.md
index a2a5009..b38d9fb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/registry/yarn-registry.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/registry/yarn-registry.md
@@ -353,6 +353,10 @@ application.
         <td>Description</td>
       </tr>
       <tr>
+        <td>type: String</td>
+        <td>Always: "JSONServiceRecord"</td>
+      </tr>
+      <tr>
         <td>description: String</td>
         <td>Human-readable description.</td>
       </tr>
@@ -366,6 +370,8 @@ application.
       </tr>
     </table>
 
+The type field MUST be `"JSONServiceRecord"`. Mandating this string allows future record types *and* permits rapid rejection of byte arrays that lack this string before attempting JSON parsing.
+
 ### YARN Persistence policies
 
 The YARN Resource Manager integration integrates cleanup of service records
@@ -379,7 +385,6 @@ any use of the registry without the RM's participation.
 The attributes, `yarn:id` and `yarn:persistence` specify which records
 *and any child entries* may be deleted as the associated YARN components complete.
 
-
 The `yarn:id` field defines the application, attempt or container ID to match;
 the `yarn:persistence` attribute defines the trigger for record cleanup, and
 implicitly the type of the contents of the `yarn:id` field.
@@ -432,31 +437,32 @@ up according the lifecycle of that application.
     <td>Description</td>
   </tr>
   <tr>
-    <td>addresses: List[List[String]]</td>
-    <td>a list of address tuples whose format depends on the address type</td>
-  </tr>
-  <tr>
-    <td>addressType: String</td>
-    <td>format of the binding</td>
-  </tr>
+    <td>api: URI as String</td>
+    <td>API implemented at the end of the binding</td>
   <tr>
     <td>protocol: String</td>
     <td>Protocol. Examples:
 `http`, `https`, `hadoop-rpc`, `zookeeper`, `web`, `REST`, `SOAP`, ...</td>
   </tr>
   <tr>
-    <td>api: String</td>
-    <td>API implemented at the end of the binding</td>
+    <td>addressType: String</td>
+    <td>format of the binding</td>
   </tr>
+  </tr>
+    <tr>
+    <td>addresses: List[Map[String, String]]</td>
+    <td>a list of address maps</td>
+  </tr>
+
 </table>
 
 
 All string fields have a limit on size, to dissuade services from hiding
 complex JSON structures in the text description.
 
-### Field: Address Type
+#### Field `addressType`: Address Type
 
-The addressType field defines the string format of entries.
+The `addressType` field defines the string format of entries.
 
 Having separate types is that tools (such as a web viewer) can process binding
 strings without having to recognize the protocol.
@@ -467,43 +473,58 @@ strings without having to recognize the protocol.
     <td>binding format</td>
   </tr>
   <tr>
-    <td>`url`</td>
-    <td>`[URL]`</td>
+    <td>uri</td>
+    <td>uri:URI of endpoint</td>
   </tr>
   <tr>
-    <td>`hostname`</td>
-    <td>`[hostname]`</td>
+    <td>hostname</td>
+    <td>hostname: service host</td>
   </tr>
   <tr>
-    <td>`inetaddress`</td>
-    <td>`[hostname, port]`</td>
+    <td>inetaddress</td>
+    <td>hostname: service host, port: service port</td>
   </tr>
   <tr>
-    <td>`path`</td>
-    <td>`[/path/to/something]`</td>
+    <td>path</td>
+    <td>path: generic unix filesystem path</td>
   </tr>
   <tr>
-    <td>`zookeeper`</td>
-    <td>`[quorum-entry, path]`</td>
+    <td>zookeeper</td>
+    <td>hostname: service host, port: service port, path: ZK path</td>
   </tr>
 </table>
 
 
-An actual zookeeper binding consists of a list of `hostname:port` bindings –the
-quorum— and the path within. In the proposed schema, every quorum entry will be
-listed as a triple of `[hostname, port, path]`. Client applications do not
-expect the path to de be different across the quorum. The first entry in the
-list of quorum hosts MUST define the path to be used by all clients. Later
-entries SHOULD list the same path, though clients MUST ignore these.
+In the zookeeper binding, every entry represents a single node in quorum,
+the `hostname` and `port` fields defining the hostname of the ZK instance
+and the port on which it is listening. The `path` field lists zookeeper path
+for applications to use. For example, for HBase this would refer to the znode
+containing information about the HBase cluster.
+
+The path MUST be identical across all address elements in the `addresses` list.
+This ensures that any single address contains enough information to connect
+to the quorum and connect to the relevant znode.
 
 New Address types may be defined; if not standard please prefix with the
 character sequence `"x-"`.
 
-#### **Field: API**
+### Field `api`: API identifier
+
+The API field MUST contain a URI that identifies the specific API of an endpoint.
+These MUST be unique to an API to avoid confusion.
+
+The following strategies are suggested to provide unique URIs for an API
+
+1. The SOAP/WS-* convention of using the URL to where the WSDL defining the service
+2. A URL to the svn/git hosted document defining a REST API
+3. the `classpath` schema followed by a path to a class or package in an application.
+4. The `uuid` schema with a generated UUID.
+
+It is hoped that standard API URIs will be defined for common APIs. Two such non-normative APIs are used in this document
+
+* `http://` : A web site for humans
+* `classpath:javax.management.jmx`: and endpoint supporting the JMX management protocol (RMI-based)
 
-APIs may be unique to a service class, or may be common across by service
-classes. They MUST be given unique names. These MAY be based on service
-packages but MAY be derived from other naming schemes:
 
 ### Examples of Service Entries
 
@@ -524,12 +545,14 @@ overall application. It exports the URL to a load balancer.
 
     {
       "description" : "tomcat-based web application",
-      "registrationTime" : 1408638082444,
       "external" : [ {
-        "api" : "www",
+        "api" : "http://internal.example.org/restapis/scheduler/20141026v1",
         "addressType" : "uri",
-        "protocolType" : "REST",
-        "addresses" : [ [ "http://loadbalancer/" ] [ "http://loadbalancer2/" ] ]
+        "protocol" : "REST",
+        "addresses" : [
+         { "uri" : "http://loadbalancer/" },
+         { "uri" : "http://loadbalancer2/" }
+          ]
       } ],
       "internal" : [ ]
     }
@@ -545,21 +568,23 @@ will trigger the deletion of this entry
     /users/devteam/org-apache-tomcat/test1/components/container-1408631738011-0001-01-000001
 
     {
-      "registrationTime" : 1408638082445,
       "yarn:id" : "container_1408631738011_0001_01_000001",
-      "yarn:persistence" : "3",
-      "description" : null,
+      "yarn:persistence" : "container",
+      "description" : "",
       "external" : [ {
-        "api" : "www",
+        "api" : "http://internal.example.org/restapis/scheduler/20141026v1",
         "addressType" : "uri",
-        "protocolType" : "REST",
-        "addresses" : [ [ "http://rack4server3:43572" ] ]
+        "protocol" : "REST",
+        "addresses" : [{ "uri" : "rack4server3:43572" }  ]
       } ],
       "internal" : [ {
-        "api" : "jmx",
+        "api" : "classpath:javax.management.jmx",
         "addressType" : "host/port",
-        "protocolType" : "JMX",
-        "addresses" : [ [ "rack4server3", "43573" ] ]
+        "protocol" : "rmi",
+        "addresses" : [ {
+          "host" : "rack4server3",
+          "port" : "48551"
+        } ]
       } ]
     }
 
@@ -571,19 +596,22 @@ external endpoint, the JMX addresses as internal.
     {
       "registrationTime" : 1408638082445,
       "yarn:id" : "container_1408631738011_0001_01_000002",
-      "yarn:persistence" : "3",
+      "yarn:persistence" : "container",
       "description" : null,
       "external" : [ {
-        "api" : "www",
+        "api" : "http://internal.example.org/restapis/scheduler/20141026v1",
         "addressType" : "uri",
-        "protocolType" : "REST",
+        "protocol" : "REST",
         "addresses" : [ [ "http://rack1server28:35881" ] ]
       } ],
       "internal" : [ {
-        "api" : "jmx",
+        "api" : "classpath:javax.management.jmx",
         "addressType" : "host/port",
-        "protocolType" : "JMX",
-        "addresses" : [ [ "rack1server28", "35882" ] ]
+        "protocol" : "rmi",
+        "addresses" : [ {
+          "host" : "rack1server28",
+          "port" : "48551"
+        } ]
       } ]
     }
 
@@ -887,3 +915,106 @@ Implementations may throttle update operations.
 **Rate of Polling**
 
 Clients which poll the registry may be throttled.
+
+# Complete service record example
+
+Below is a (non-normative) example of a service record retrieved
+from a YARN application.
+
+
+    {
+      "type" : "JSONServiceRecord",
+      "description" : "Slider Application Master",
+      "yarn:persistence" : "application",
+      "yarn:id" : "application_1414052463672_0028",
+      "external" : [ {
+        "api" : "classpath:org.apache.slider.appmaster",
+        "addressType" : "host/port",
+        "protocol" : "hadoop/IPC",
+        "addresses" : [ {
+          "port" : "48551",
+          "host" : "nn.example.com"
+        } ]
+      }, {
+        "api" : "http://",
+        "addressType" : "uri",
+        "protocol" : "web",
+        "addresses" : [ {
+          "uri" : "http://nn.example.com:40743"
+        } ]
+      }, {
+        "api" : "classpath:org.apache.slider.management",
+        "addressType" : "uri",
+        "protocol" : "REST",
+        "addresses" : [ {
+          "uri" : "http://nn.example.com:40743/ws/v1/slider/mgmt"
+        } ]
+      }, {
+        "api" : "classpath:org.apache.slider.publisher",
+        "addressType" : "uri",
+        "protocol" : "REST",
+        "addresses" : [ {
+          "uri" : "http://nn.example.com:40743/ws/v1/slider/publisher"
+        } ]
+      }, {
+        "api" : "classpath:org.apache.slider.registry",
+        "addressType" : "uri",
+        "protocol" : "REST",
+        "addresses" : [ {
+          "uri" : "http://nn.example.com:40743/ws/v1/slider/registry"
+        } ]
+      }, {
+        "api" : "classpath:org.apache.slider.publisher.configurations",
+        "addressType" : "uri",
+        "protocol" : "REST",
+        "addresses" : [ {
+          "uri" : "http://nn.example.com:40743/ws/v1/slider/publisher/slider"
+        } ]
+      }, {
+        "api" : "classpath:org.apache.slider.publisher.exports",
+        "addressType" : "uri",
+        "protocol" : "REST",
+        "addresses" : [ {
+          "uri" : "http://nn.example.com:40743/ws/v1/slider/publisher/exports"
+        } ]
+      } ],
+      "internal" : [ {
+        "api" : "classpath:org.apache.slider.agents.secure",
+        "addressType" : "uri",
+        "protocol" : "REST",
+        "addresses" : [ {
+          "uri" : "https://nn.example.com:52705/ws/v1/slider/agents"
+        } ]
+      }, {
+        "api" : "classpath:org.apache.slider.agents.oneway",
+        "addressType" : "uri",
+        "protocol" : "REST",
+        "addresses" : [ {
+          "uri" : "https://nn.example.com:33425/ws/v1/slider/agents"
+        } ]
+      } ]
+    }
+
+It publishes a number of endpoints, both internal and external.
+
+External:
+
+1. The IPC hostname and port for client-AM communications
+1. URL to the AM's web UI
+1. A series of REST URLs under the web UI for specific application services.
+The details are irrelevant —note that they use an application-specific API
+value to ensure uniqueness.
+
+Internal:
+1. Two URLS to REST APIs offered by the AM for containers deployed by
+ the application itself.
+
+Python agents running in the containers retrieve the internal endpoint
+URLs to communicate with their AM. The record is resolved on container startup
+and cached until communications problems occur. At that point the registry is
+queried for the current record, then an attempt is made to reconnect to the AM.
+
+Here "connectivity" problems means both "low level socket/IO errors" and
+"failures in HTTPS authentication". The agents use two-way HTTPS authentication
+—if the AM fails and another application starts listening on the same ports
+it will trigger an authentication failure and hence service record reread.


[26/30] hadoop git commit: YARN-2505. Supported get/add/remove/change labels in RM REST API. Contributed by Craig Welch.

Posted by vi...@apache.org.
YARN-2505. Supported get/add/remove/change labels in RM REST API. 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/9a4e0d34
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9a4e0d34
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9a4e0d34

Branch: refs/heads/HDFS-EC
Commit: 9a4e0d343e9e891c10ef6682e7b2231a59e69ade
Parents: df36edf
Author: Zhijie Shen <zj...@apache.org>
Authored: Fri Nov 7 20:35:46 2014 -0800
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Nov 7 20:35:46 2014 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../apache/hadoop/yarn/util/ConverterUtils.java |   7 +
 .../hadoop/yarn/util/TestConverterUtils.java    |  14 +
 .../resourcemanager/webapp/RMWebServices.java   | 180 +++++++++-
 .../dao/ApplicationSubmissionContextInfo.java   |  23 ++
 .../webapp/dao/NodeLabelsInfo.java              |  52 +++
 .../webapp/dao/NodeToLabelsInfo.java            |  41 +++
 .../webapp/TestRMWebServicesNodeLabels.java     | 357 +++++++++++++++++++
 8 files changed, 676 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a4e0d34/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 1e6406a..748ffe0 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -198,6 +198,9 @@ Release 2.6.0 - UNRELEASED
     YARN-2632. Document NM Restart feature. (Junping Du and Vinod Kumar
     Vavilapalli via jlowe)
 
+    YARN-2505. Supported get/add/remove/change labels in RM REST API. (Craig Welch
+    via zjshen)
+
   IMPROVEMENTS
 
     YARN-2197. Add a link to YARN CHANGES.txt in the left side of doc

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a4e0d34/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ConverterUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ConverterUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ConverterUtils.java
index 27f7bc1..012d799 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ConverterUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ConverterUtils.java
@@ -151,6 +151,13 @@ public class ConverterUtils {
   public static String toString(ContainerId cId) {
     return cId == null ? null : cId.toString();
   }
+  
+  public static NodeId toNodeIdWithDefaultPort(String nodeIdStr) {
+    if (nodeIdStr.indexOf(":") < 0) {
+      return toNodeId(nodeIdStr + ":0");
+    }
+    return toNodeId(nodeIdStr);
+  }
 
   public static NodeId toNodeId(String nodeIdStr) {
     String[] parts = nodeIdStr.split(":");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a4e0d34/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestConverterUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestConverterUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestConverterUtils.java
index 824e6c0..7d53785 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestConverterUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestConverterUtils.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.yarn.api.TestContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.URL;
+import org.apache.hadoop.yarn.api.records.NodeId;
 import org.junit.Test;
 
 public class TestConverterUtils {
@@ -85,4 +86,17 @@ public class TestConverterUtils {
   public void testContainerIdNull() throws URISyntaxException {
     assertNull(ConverterUtils.toString((ContainerId)null));
   }  
+  
+  @Test
+  public void testNodeIdWithDefaultPort() throws URISyntaxException {
+    NodeId nid;
+    
+    nid = ConverterUtils.toNodeIdWithDefaultPort("node:10");
+    assertEquals(nid.getPort(), 10);
+    assertEquals(nid.getHost(), "node");
+    
+    nid = ConverterUtils.toNodeIdWithDefaultPort("node");
+    assertEquals(nid.getPort(), 0);
+    assertEquals(nid.getHost(), "node");
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a4e0d34/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 87c895a..cf0a83a 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
@@ -24,6 +24,7 @@ import java.security.AccessControlException;
 import java.nio.ByteBuffer;
 import java.security.Principal;
 import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.EnumSet;
@@ -133,6 +134,8 @@ 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.resourcemanager.webapp.dao.NodeLabelsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsInfo;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.webapp.BadRequestException;
@@ -715,6 +718,179 @@ public class RMWebServices {
 
     return Response.status(Status.OK).entity(ret).build();
   }
+  
+  @GET
+  @Path("/get-node-to-labels")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public NodeToLabelsInfo getNodeToLabels(@Context HttpServletRequest hsr) 
+    throws IOException {
+    init();
+
+    NodeToLabelsInfo ntl = new NodeToLabelsInfo();
+    HashMap<String, NodeLabelsInfo> ntlMap = ntl.getNodeToLabels();
+    Map<NodeId, Set<String>> nodeIdToLabels =   
+      rm.getRMContext().getNodeLabelManager().getNodeLabels();
+      
+    for (Map.Entry<NodeId, Set<String>> nitle : nodeIdToLabels.entrySet()) {
+      ntlMap.put(nitle.getKey().toString(), 
+        new NodeLabelsInfo(nitle.getValue()));
+    }
+
+    return ntl;
+  }
+  
+  @POST
+  @Path("/replace-node-to-labels")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public Response replaceLabelsOnNodes(
+    final NodeToLabelsInfo newNodeToLabels,
+    @Context HttpServletRequest hsr) 
+    throws IOException {
+    init();
+    
+    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";
+      throw new AuthorizationException(msg);
+    }
+    if (!rm.getRMContext().getNodeLabelManager().checkAccess(callerUGI)) {
+      String msg = "User " + callerUGI.getShortUserName() + " not authorized"
+        + " for post to .../replace-node-to-labels ";
+      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);
+
+    return Response.status(Status.OK).build();
+  }
+  
+  @GET
+  @Path("/get-node-labels")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public NodeLabelsInfo getClusterNodeLabels(@Context HttpServletRequest hsr) 
+    throws IOException {
+    init();
+
+    NodeLabelsInfo ret = 
+      new NodeLabelsInfo(rm.getRMContext().getNodeLabelManager()
+        .getClusterNodeLabels());
+
+    return ret;
+  }
+  
+  @POST
+  @Path("/add-node-labels")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public Response addToClusterNodeLabels(final NodeLabelsInfo newNodeLabels,
+      @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"
+        + " post to .../add-node-labels";
+      throw new AuthorizationException(msg);
+    }
+    if (!rm.getRMContext().getNodeLabelManager().checkAccess(callerUGI)) {
+      String msg = "User " + callerUGI.getShortUserName() + " not authorized"
+        + " for post to .../add-node-labels ";
+      throw new AuthorizationException(msg);
+    }
+    
+    rm.getRMContext().getNodeLabelManager()
+        .addToCluserNodeLabels(new HashSet<String>(
+          newNodeLabels.getNodeLabels()));
+            
+    return Response.status(Status.OK).build();
+
+  }
+  
+  @POST
+  @Path("/remove-node-labels")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public Response removeFromCluserNodeLabels(final NodeLabelsInfo oldNodeLabels,
+      @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"
+        + " post to .../remove-node-labels";
+      throw new AuthorizationException(msg);
+    }
+    if (!rm.getRMContext().getNodeLabelManager().checkAccess(callerUGI)) {
+      String msg = "User " + callerUGI.getShortUserName() + " not authorized"
+        + " for post to .../remove-node-labels ";
+      throw new AuthorizationException(msg);
+    }
+    
+    rm.getRMContext().getNodeLabelManager()
+        .removeFromClusterNodeLabels(new HashSet<String>(
+          oldNodeLabels.getNodeLabels()));
+            
+    return Response.status(Status.OK).build();
+
+  }
+  
+  @GET
+  @Path("/nodes/{nodeId}/get-labels")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public NodeLabelsInfo getLabelsOnNode(@Context HttpServletRequest hsr,
+                                  @PathParam("nodeId") String nodeId) 
+    throws IOException {
+    init();
+
+    NodeId nid = ConverterUtils.toNodeIdWithDefaultPort(nodeId);
+    return new NodeLabelsInfo(
+      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 {
@@ -965,7 +1141,9 @@ public class RMWebServices {
           newApp.getCancelTokensWhenComplete(), newApp.getMaxAppAttempts(),
           createAppSubmissionContextResource(newApp),
           newApp.getApplicationType(),
-          newApp.getKeepContainersAcrossApplicationAttempts());
+          newApp.getKeepContainersAcrossApplicationAttempts(),
+          newApp.getAppNodeLabelExpression(),
+          newApp.getAMContainerNodeLabelExpression());
     appContext.setApplicationTags(newApp.getApplicationTags());
 
     return appContext;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a4e0d34/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ApplicationSubmissionContextInfo.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/dao/ApplicationSubmissionContextInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ApplicationSubmissionContextInfo.java
index f7233e6..5278b3e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ApplicationSubmissionContextInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ApplicationSubmissionContextInfo.java
@@ -71,6 +71,12 @@ public class ApplicationSubmissionContextInfo {
   @XmlElementWrapper(name = "application-tags")
   @XmlElement(name = "tag")
   Set<String> tags;
+  
+  @XmlElement(name = "app-node-label-expression")
+  String appNodeLabelExpression;
+  
+  @XmlElement(name = "am-container-node-label-expression")
+  String amContainerNodeLabelExpression;
 
   public ApplicationSubmissionContextInfo() {
     applicationId = "";
@@ -83,6 +89,8 @@ public class ApplicationSubmissionContextInfo {
     keepContainers = false;
     applicationType = "";
     tags = new HashSet<String>();
+    appNodeLabelExpression = "";
+    amContainerNodeLabelExpression = "";
   }
 
   public String getApplicationId() {
@@ -132,6 +140,14 @@ public class ApplicationSubmissionContextInfo {
   public Set<String> getApplicationTags() {
     return tags;
   }
+  
+  public String getAppNodeLabelExpression() {
+    return appNodeLabelExpression;
+  }
+  
+  public String getAMContainerNodeLabelExpression() {
+    return amContainerNodeLabelExpression;
+  }
 
   public void setApplicationId(String applicationId) {
     this.applicationId = applicationId;
@@ -182,5 +198,12 @@ public class ApplicationSubmissionContextInfo {
   public void setApplicationTags(Set<String> tags) {
     this.tags = tags;
   }
+  
+  public void setAppNodeLabelExpression(String appNodeLabelExpression) {
+    this.appNodeLabelExpression = appNodeLabelExpression;
+  }
 
+  public void setAMContainerNodeLabelExpression(String nodeLabelExpression) {
+    this.amContainerNodeLabelExpression = nodeLabelExpression;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a4e0d34/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeLabelsInfo.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/dao/NodeLabelsInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeLabelsInfo.java
new file mode 100644
index 0000000..1cb895a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeLabelsInfo.java
@@ -0,0 +1,52 @@
+/**
+ * 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.webapp.dao;
+
+import java.util.*;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+@XmlRootElement(name = "nodeLabelsInfo")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class NodeLabelsInfo {
+
+  protected ArrayList<String> nodeLabels = new ArrayList<String>();
+
+  public NodeLabelsInfo() {
+  } // JAXB needs this
+  
+  public NodeLabelsInfo(ArrayList<String> nodeLabels) {
+   this.nodeLabels = nodeLabels; 
+  }
+  
+  public NodeLabelsInfo(Set<String> nodeLabelsSet) {
+   this.nodeLabels = new ArrayList<String>(nodeLabelsSet); 
+  }
+  
+  public ArrayList<String> getNodeLabels() {
+    return nodeLabels;
+  }
+  
+  public void setNodeLabels(ArrayList<String> nodeLabels) {
+    this.nodeLabels = nodeLabels; 
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a4e0d34/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeToLabelsInfo.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/dao/NodeToLabelsInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeToLabelsInfo.java
new file mode 100644
index 0000000..f2e6441
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeToLabelsInfo.java
@@ -0,0 +1,41 @@
+/**
+ * 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.webapp.dao;
+
+import java.util.*;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+@XmlRootElement(name = "nodeToLabelsInfo")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class NodeToLabelsInfo {
+
+  protected HashMap<String, NodeLabelsInfo> nodeToLabels = 
+    new HashMap<String, NodeLabelsInfo>();
+
+  public NodeToLabelsInfo() {
+  } // JAXB needs this
+  
+  public HashMap<String, NodeLabelsInfo> getNodeToLabels() {
+   return nodeToLabels; 
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a4e0d34/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
new file mode 100644
index 0000000..3c958f2
--- /dev/null
+++ 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
@@ -0,0 +1,357 @@
+/**
+ * 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.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;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeLabelsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsInfo;
+import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.codehaus.jettison.json.JSONArray;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
+import org.junit.Test;
+
+import com.google.inject.Guice;
+import com.google.inject.Injector;
+import com.google.inject.servlet.GuiceServletContextListener;
+import com.google.inject.servlet.ServletModule;
+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.guice.spi.container.servlet.GuiceContainer;
+import com.sun.jersey.test.framework.JerseyTest;
+import com.sun.jersey.test.framework.WebAppDescriptor;
+
+public class TestRMWebServicesNodeLabels extends JerseyTest {
+
+  private static final Log LOG = LogFactory
+      .getLog(TestRMWebServicesNodeLabels.class);
+
+  private static MockRM rm;
+  private YarnConfiguration conf;
+
+  private String userName;
+  private String notUserName;
+
+  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) {
+        throw new RuntimeException("Unable to get current user name "
+            + ioe.getMessage(), ioe);
+      }
+      notUserName = userName + "abc123";
+      conf = new YarnConfiguration();
+      conf.set(YarnConfiguration.YARN_ADMIN_ACL, userName);
+      rm = new MockRM(conf);
+      bind(ResourceManager.class).toInstance(rm);
+      bind(RMContext.class).toInstance(rm.getRMContext());
+      filter("/*").through(
+          TestRMWebServicesAppsModification.TestRMCustomAuthFilter.class);
+      serve("/*").with(GuiceContainer.class);
+    }
+  });
+
+  public class GuiceServletConfig extends GuiceServletContextListener {
+
+    @Override
+    protected Injector getInjector() {
+      return injector;
+    }
+  }
+
+  public TestRMWebServicesNodeLabels() {
+    super(new WebAppDescriptor.Builder(
+        "org.apache.hadoop.yarn.server.resourcemanager.webapp")
+        .contextListenerClass(GuiceServletConfig.class)
+        .filterClass(com.google.inject.servlet.GuiceFilter.class)
+        .contextPath("jersey-guice-filter").servletPath("/").build());
+  }
+
+  @Test
+  public void testNodeLabels() throws JSONException, Exception {
+    WebResource r = resource();
+
+    ClientResponse response;
+    JSONObject json;
+    JSONArray jarr;
+    String responseString;
+
+    // Add a label
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("add-node-labels").queryParam("user.name", userName)
+            .accept(MediaType.APPLICATION_JSON)
+            .entity("{\"nodeLabels\":\"a\"}", 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("a", json.getString("nodeLabels"));
+    
+    // Add another
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("add-node-labels").queryParam("user.name", userName)
+            .accept(MediaType.APPLICATION_JSON)
+            .entity("{\"nodeLabels\":\"b\"}", 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);
+    jarr = json.getJSONArray("nodeLabels");
+    assertEquals(2, jarr.length());
+    
+    // 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\": [\"a\", \"b\"]}",
+              MediaType.APPLICATION_JSON)
+            .post(ClientResponse.class);
+    LOG.info("posted node nodelabel");
+
+    // Verify
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("nodes").path("nid:0")
+            .path("get-labels").queryParam("user.name", userName)
+            .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
+    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    json = response.getEntity(JSONObject.class);
+    jarr = json.getJSONArray("nodeLabels");
+    assertEquals(2, jarr.length());
+    
+    // Replace
+    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\":\"a\"}", MediaType.APPLICATION_JSON)
+            .post(ClientResponse.class);
+    LOG.info("posted node nodelabel");
+    // Verify
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("nodes").path("nid:0")
+            .path("get-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("a", json.getString("nodeLabels"));
+            
+    // Replace labels using node-to-labels
+    NodeToLabelsInfo ntli = new NodeToLabelsInfo();
+    NodeLabelsInfo nli = new NodeLabelsInfo();
+    nli.getNodeLabels().add("a");
+    nli.getNodeLabels().add("b");
+    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
+    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(2, nli.getNodeLabels().size());
+    assertTrue(nli.getNodeLabels().contains("a"));
+    assertTrue(nli.getNodeLabels().contains("b"));
+    
+    // Remove all
+    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\"}", MediaType.APPLICATION_JSON)
+            .post(ClientResponse.class);
+    LOG.info("posted node nodelabel");
+    // Verify
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("nodes").path("nid:0")
+            .path("get-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("", json.getString("nodeLabels"));
+    
+    // Add a label back for auth tests
+    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\": \"a\"}",
+              MediaType.APPLICATION_JSON)
+            .post(ClientResponse.class);
+    LOG.info("posted node nodelabel");
+
+    // Verify
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("nodes").path("nid:0")
+            .path("get-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("a", json.getString("nodeLabels"));
+    
+    // Auth fail replace labels on node
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("nodes").path("nid:0")
+            .path("replace-labels")
+            .queryParam("user.name", notUserName)
+            .accept(MediaType.APPLICATION_JSON)
+            .entity("{\"nodeLabels\": [\"a\", \"b\"]}",
+              MediaType.APPLICATION_JSON)
+            .post(ClientResponse.class);
+    // Verify
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("nodes").path("nid:0")
+            .path("get-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("a", json.getString("nodeLabels"));
+    
+    // Fail to add a label with post
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("add-node-labels").queryParam("user.name", notUserName)
+            .accept(MediaType.APPLICATION_JSON)
+            .entity("{\"nodeLabels\":\"c\"}", 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);
+    jarr = json.getJSONArray("nodeLabels");
+    assertEquals(2, jarr.length());
+    
+    // Remove cluster label (succeed, we no longer need it)
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("remove-node-labels")
+            .queryParam("user.name", userName)
+            .accept(MediaType.APPLICATION_JSON)
+            .entity("{\"nodeLabels\":\"b\"}", 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("a", json.getString("nodeLabels"));
+    
+    
+    // Remove cluster label with post
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("remove-node-labels")
+            .queryParam("user.name", userName)
+            .accept(MediaType.APPLICATION_JSON)
+            .entity("{\"nodeLabels\":\"a\"}", 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());
+    String res = response.getEntity(String.class);
+    assertTrue(res.equals("null"));
+  }
+
+  @SuppressWarnings("rawtypes")
+  private String toJson(Object nsli, Class klass) throws Exception {
+    StringWriter sw = new StringWriter();
+    JSONJAXBContext ctx = new JSONJAXBContext(klass);
+    JSONMarshaller jm = ctx.createJSONMarshaller();
+    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);
+  }
+
+}


[13/30] hadoop git commit: HADOOP-11280. TestWinUtils#testChmod fails after removal of NO_PROPAGATE_INHERIT_ACE. Contributed by Chris Nauroth.

Posted by vi...@apache.org.
HADOOP-11280. TestWinUtils#testChmod fails after removal of NO_PROPAGATE_INHERIT_ACE. Contributed by Chris Nauroth.


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

Branch: refs/heads/HDFS-EC
Commit: 42bbe3794e4a8edef020e08111dbd7f1058938e2
Parents: d026f36
Author: cnauroth <cn...@apache.org>
Authored: Fri Nov 7 00:02:17 2014 -0800
Committer: cnauroth <cn...@apache.org>
Committed: Fri Nov 7 00:02:17 2014 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt                   | 3 +++
 .../src/test/java/org/apache/hadoop/util/TestWinUtils.java        | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/42bbe379/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 8587f12..0192560 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1083,6 +1083,9 @@ Release 2.6.0 - UNRELEASED
     HADOOP-11265. Credential and Key Shell Commands not available on Windows.
     (Larry McCay via cnauroth)
 
+    HADOOP-11280. TestWinUtils#testChmod fails after removal of
+    NO_PROPAGATE_INHERIT_ACE. (cnauroth)
+
 Release 2.5.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42bbe379/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestWinUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestWinUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestWinUtils.java
index 953039d..2d4e442 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestWinUtils.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestWinUtils.java
@@ -371,7 +371,7 @@ public class TestWinUtils {
     testChmodInternalR("a+rX", "rw-r--r--", "rwxr-xr-x");
 
     // Test a new file created in a chmod'ed directory has expected permission
-    testNewFileChmodInternal("-rwx------");
+    testNewFileChmodInternal("-rwxr-xr-x");
   }
 
   private void chown(String userGroup, File file) throws IOException {


[12/30] hadoop git commit: HDFS-7226. Update CHANGES.txt to indicate fix in 2.6.0.

Posted by vi...@apache.org.
HDFS-7226. Update CHANGES.txt to indicate fix in 2.6.0.


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

Branch: refs/heads/HDFS-EC
Commit: d026f3676278e24d7032dced5f14b52dec70b987
Parents: e7f1c04
Author: cnauroth <cn...@apache.org>
Authored: Thu Nov 6 23:10:21 2014 -0800
Committer: cnauroth <cn...@apache.org>
Committed: Thu Nov 6 23:10:21 2014 -0800

----------------------------------------------------------------------
 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/d026f367/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 0e64d69..8735d61 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -380,8 +380,6 @@ Release 2.7.0 - UNRELEASED
     HDFS-7252. small refinement to the use of isInAnEZ in FSNamesystem.
     (Yi Liu via vinayakumarb)
 
-    HDFS-7226. Fix TestDNFencing.testQueueingWithAppend. (Yongjun Zhang via jing9)
-
     HDFS-7277. Remove explicit dependency on netty 3.2 in BKJournal. (wheat9)
 
     HDFS-7232. Populate hostname in httpfs audit log (Zoran Dimitrijevic 
@@ -1409,6 +1407,8 @@ Release 2.6.0 - UNRELEASED
     dfs.namenode.secondary.http-address cannot be interpreted as a network
     address. (cnauroth)
 
+    HDFS-7226. Fix TestDNFencing.testQueueingWithAppend. (Yongjun Zhang via jing9)
+
 Release 2.5.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES


[20/30] hadoop git commit: YARN-2753. Fixed a bunch of bugs in the NodeLabelsManager classes. Contributed by Zhihai xu.

Posted by vi...@apache.org.
YARN-2753. Fixed a bunch of bugs in the NodeLabelsManager classes. 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/4cfd5bc7
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/4cfd5bc7
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/4cfd5bc7

Branch: refs/heads/HDFS-EC
Commit: 4cfd5bc7c18bb9a828f573b5c4d2b13fa28e732a
Parents: 57760c0
Author: Vinod Kumar Vavilapalli <vi...@apache.org>
Authored: Fri Nov 7 14:15:53 2014 -0800
Committer: Vinod Kumar Vavilapalli <vi...@apache.org>
Committed: Fri Nov 7 14:15:53 2014 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../nodelabels/CommonNodeLabelsManager.java     | 25 ++++++++++-------
 .../nodelabels/TestRMNodeLabelsManager.java     | 29 +++++++++++++++++++-
 3 files changed, 46 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4cfd5bc7/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 97b67b5..6e5f59a 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -903,6 +903,9 @@ Release 2.6.0 - UNRELEASED
     YARN-2803. MR distributed cache not working correctly on Windows after
     NodeManager privileged account changes. (Craig Welch via cnauroth)
 
+    YARN-2753. Fixed a bunch of bugs in the NodeLabelsManager classes. (Zhihai xu
+    via vinodkv)
+
 Release 2.5.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4cfd5bc7/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 ee1b945..1d86211 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
@@ -144,9 +144,7 @@ public class CommonNodeLabelsManager extends AbstractService {
 
     @Override
     public void handle(NodeLabelsStoreEvent event) {
-      if (isInState(STATE.STARTED)) {
-        handleStoreEvent(event);
-      }
+      handleStoreEvent(event);
     }
   }
   
@@ -256,7 +254,7 @@ public class CommonNodeLabelsManager extends AbstractService {
     if (null == labels || labels.isEmpty()) {
       return;
     }
-
+    Set<String> newLabels = new HashSet<String>();
     labels = normalizeLabels(labels);
 
     // do a check before actual adding them, will throw exception if any of them
@@ -266,11 +264,15 @@ public class CommonNodeLabelsManager extends AbstractService {
     }
 
     for (String label : labels) {
-      this.labelCollections.put(label, new Label());
+      // shouldn't overwrite it to avoid changing the Label.resource
+      if (this.labelCollections.get(label) == null) {
+        this.labelCollections.put(label, new Label());
+        newLabels.add(label);
+      }
     }
-    if (null != dispatcher) {
+    if (null != dispatcher && !newLabels.isEmpty()) {
       dispatcher.getEventHandler().handle(
-          new StoreNewClusterNodeLabels(labels));
+          new StoreNewClusterNodeLabels(newLabels));
     }
 
     LOG.info("Add labels: [" + StringUtils.join(labels.iterator(), ",") + "]");
@@ -453,12 +455,15 @@ public class CommonNodeLabelsManager extends AbstractService {
         LOG.error(msg);
         throw new IOException(msg);
       }
-      
-      if (labels == null || labels.isEmpty()) {
+
+      // the labels will never be null
+      if (labels.isEmpty()) {
         continue;
       }
 
-      if (!originalLabels.containsAll(labels)) {
+      // originalLabels may be null,
+      // because when a Node is created, Node.labels can be null.
+      if (originalLabels == null || !originalLabels.containsAll(labels)) {
         String msg =
             "Try to remove labels = [" + StringUtils.join(labels, ",")
                 + "], but not all labels contained by NM=" + nodeId;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4cfd5bc7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.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/TestRMNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java
index 0ea7456..ed675f3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java
@@ -74,6 +74,13 @@ public class TestRMNodeLabelsManager extends NodeLabelTestBase {
     Assert.assertEquals(mgr.getResourceByLabel("p1", null),
         Resources.add(SMALL_RESOURCE, LARGE_NODE));
 
+    // check add labels multiple times shouldn't overwrite
+    // original attributes on labels like resource
+    mgr.addToCluserNodeLabels(toSet("p1", "p4"));
+    Assert.assertEquals(mgr.getResourceByLabel("p1", null),
+        Resources.add(SMALL_RESOURCE, LARGE_NODE));
+    Assert.assertEquals(mgr.getResourceByLabel("p4", null), EMPTY_RESOURCE);
+
     // change the large NM to small, check if resource updated
     mgr.updateNodeResource(NodeId.newInstance("n1", 2), SMALL_RESOURCE);
     Assert.assertEquals(mgr.getResourceByLabel("p1", null),
@@ -374,7 +381,7 @@ public class TestRMNodeLabelsManager extends NodeLabelTestBase {
     Assert.assertEquals(clusterResource,
         mgr.getQueueResource("Q5", q5Label, clusterResource));
   }
-  
+
   @Test(timeout=5000)
   public void testGetLabelResourceWhenMultipleNMsExistingInSameHost() throws IOException {
     // active two NM to n1, one large and one small
@@ -401,4 +408,24 @@ public class TestRMNodeLabelsManager extends NodeLabelTestBase {
             mgr.getResourceByLabel("p1", null),
             Resources.multiply(SMALL_RESOURCE, 2));
   }
+
+  @Test(timeout = 5000)
+  public void testRemoveLabelsFromNode() throws Exception {
+    mgr.addToCluserNodeLabels(toSet("p1", "p2", "p3"));
+    mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("n1"), toSet("p1"),
+            toNodeId("n2"), toSet("p2"), toNodeId("n3"), toSet("p3")));
+    // active one NM to n1:1
+    mgr.activateNode(NodeId.newInstance("n1", 1), SMALL_RESOURCE);
+    try {
+      mgr.removeLabelsFromNode(ImmutableMap.of(toNodeId("n1:1"), toSet("p1")));
+      Assert.fail("removeLabelsFromNode should trigger IOException");
+    } catch (IOException e) {
+    }
+    mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("n1:1"), toSet("p1")));
+    try {
+      mgr.removeLabelsFromNode(ImmutableMap.of(toNodeId("n1:1"), toSet("p1")));
+    } catch (IOException e) {
+      Assert.fail("IOException from removeLabelsFromNode " + e);
+    }
+  }
 }


[15/30] hadoop git commit: YARN-2810. TestRMProxyUsersConf fails on Windows VMs. Contributed by Varun Vasudev

Posted by vi...@apache.org.
YARN-2810. TestRMProxyUsersConf fails on Windows VMs. Contributed by Varun Vasudev


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

Branch: refs/heads/HDFS-EC
Commit: 1e97f2f09464e871773188f642f3a01b744c580f
Parents: a565718
Author: Xuan <xg...@apache.org>
Authored: Fri Nov 7 09:44:43 2014 -0800
Committer: Xuan <xg...@apache.org>
Committed: Fri Nov 7 09:44:43 2014 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                             | 2 ++
 .../yarn/server/resourcemanager/TestRMProxyUsersConf.java   | 9 +++++----
 2 files changed, 7 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e97f2f0/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 40f54d4..111aaaa 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -892,6 +892,8 @@ Release 2.6.0 - UNRELEASED
     schedulers about previous finished attempts of a running appliation to avoid
     expectation mismatch w.r.t transferred containers. (Jian He via vinodkv)
 
+    YARN-2810. TestRMProxyUsersConf fails on Windows VMs. (Varun Vasudev via xgong)
+
 Release 2.5.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e97f2f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMProxyUsersConf.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/TestRMProxyUsersConf.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMProxyUsersConf.java
index 86a9fe4..f6d239d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMProxyUsersConf.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMProxyUsersConf.java
@@ -38,6 +38,7 @@ public class TestRMProxyUsersConf {
       UserGroupInformation.createUserForTesting("foo", new String[] { "foo_group" });
   private static final UserGroupInformation BAR_USER =
       UserGroupInformation.createUserForTesting("bar", new String[] { "bar_group" });
+  private final String ipAddress = "127.0.0.1";
 
   @Parameterized.Parameters
   public static Collection<Object[]> headers() {
@@ -51,13 +52,13 @@ public class TestRMProxyUsersConf {
     switch (round) {
       case 0:
         // hadoop.proxyuser prefix
-        conf.set("hadoop.proxyuser.foo.hosts", "localhost");
+        conf.set("hadoop.proxyuser.foo.hosts", ipAddress);
         conf.set("hadoop.proxyuser.foo.users", "bar");
         conf.set("hadoop.proxyuser.foo.groups", "bar_group");
         break;
       case 1:
         // yarn.resourcemanager.proxyuser prefix
-        conf.set("yarn.resourcemanager.proxyuser.foo.hosts", "localhost");
+        conf.set("yarn.resourcemanager.proxyuser.foo.hosts", ipAddress);
         conf.set("yarn.resourcemanager.proxyuser.foo.users", "bar");
         conf.set("yarn.resourcemanager.proxyuser.foo.groups", "bar_group");
         break;
@@ -67,7 +68,7 @@ public class TestRMProxyUsersConf {
         conf.set("hadoop.proxyuser.foo.hosts", "xyz");
         conf.set("hadoop.proxyuser.foo.users", "xyz");
         conf.set("hadoop.proxyuser.foo.groups", "xyz");
-        conf.set("yarn.resourcemanager.proxyuser.foo.hosts", "localhost");
+        conf.set("yarn.resourcemanager.proxyuser.foo.hosts", ipAddress);
         conf.set("yarn.resourcemanager.proxyuser.foo.users", "bar");
         conf.set("yarn.resourcemanager.proxyuser.foo.groups", "bar_group");
         break;
@@ -89,7 +90,7 @@ public class TestRMProxyUsersConf {
               BAR_USER.getShortUserName(), FOO_USER);
       try {
         ProxyUsers.getDefaultImpersonationProvider().authorize(proxyUser,
-            "localhost");
+            ipAddress);
       } catch (AuthorizationException e) {
         // Exception is not expected
         Assert.fail();


[25/30] hadoop git commit: YARN-2826. Fixed user-groups mappings' refresh bug caused by YARN-2826. Contributed by Wangda Tan.

Posted by vi...@apache.org.
YARN-2826. Fixed user-groups mappings' refresh bug caused by YARN-2826. 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/df36edf7
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/df36edf7
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/df36edf7

Branch: refs/heads/HDFS-EC
Commit: df36edf751202db00d8f43103d7120ec56d70a04
Parents: 4a114dd
Author: Vinod Kumar Vavilapalli <vi...@apache.org>
Authored: Fri Nov 7 19:43:55 2014 -0800
Committer: Vinod Kumar Vavilapalli <vi...@apache.org>
Committed: Fri Nov 7 19:43:55 2014 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../server/resourcemanager/ResourceManager.java | 32 ++++++++++----------
 2 files changed, 19 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/df36edf7/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index d4c8827..1e6406a 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -914,6 +914,9 @@ Release 2.6.0 - UNRELEASED
     YARN-2819. NPE in ATS Timeline Domains when upgrading from 2.4 to 2.6.
     (Zhijie Shen via xgong)
 
+    YARN-2826. Fixed user-groups mappings' refresh bug caused by YARN-2826.
+    (Wangda Tan via vinodkv)
+
 Release 2.5.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df36edf7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.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/ResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
index 4051054..e0840b6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
@@ -194,22 +194,6 @@ public class ResourceManager extends CompositeService implements Recoverable {
     this.conf = conf;
     this.rmContext = new RMContextImpl();
     
-    // Set HA configuration should be done before login
-    this.rmContext.setHAEnabled(HAUtil.isHAEnabled(this.conf));
-    if (this.rmContext.isHAEnabled()) {
-      HAUtil.verifyAndSetConfiguration(this.conf);
-    }
-    
-    // Set UGI and do login
-    // If security is enabled, use login user
-    // If security is not enabled, use current user
-    this.rmLoginUGI = UserGroupInformation.getCurrentUser();
-    try {
-      doSecureLogin();
-    } catch(IOException ie) {
-      throw new YarnRuntimeException("Failed to login", ie);
-    }
-    
     this.configurationProvider =
         ConfigurationProviderFactory.getConfigurationProvider(conf);
     this.configurationProvider.init(this.conf);
@@ -242,6 +226,22 @@ public class ResourceManager extends CompositeService implements Recoverable {
     }
 
     validateConfigs(this.conf);
+    
+    // Set HA configuration should be done before login
+    this.rmContext.setHAEnabled(HAUtil.isHAEnabled(this.conf));
+    if (this.rmContext.isHAEnabled()) {
+      HAUtil.verifyAndSetConfiguration(this.conf);
+    }
+    
+    // Set UGI and do login
+    // If security is enabled, use login user
+    // If security is not enabled, use current user
+    this.rmLoginUGI = UserGroupInformation.getCurrentUser();
+    try {
+      doSecureLogin();
+    } catch(IOException ie) {
+      throw new YarnRuntimeException("Failed to login", ie);
+    }
 
     // register the handlers for all AlwaysOn services using setupDispatcher().
     rmDispatcher = setupDispatcher();


[23/30] hadoop git commit: YARN-2632. Document NM Restart feature. Contributed by Junping Du and Vinod Kumar Vavilapalli

Posted by vi...@apache.org.
YARN-2632. Document NM Restart feature. Contributed by Junping Du and Vinod Kumar Vavilapalli


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

Branch: refs/heads/HDFS-EC
Commit: 1e215e8ba2e801eb26f16c307daee756d6b2ca66
Parents: c3d4750
Author: Jason Lowe <jl...@apache.org>
Authored: Fri Nov 7 23:40:22 2014 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Fri Nov 7 23:40:22 2014 +0000

----------------------------------------------------------------------
 hadoop-project/src/site/site.xml                |  1 +
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../src/site/apt/NodeManagerRestart.apt.vm      | 86 ++++++++++++++++++++
 3 files changed, 90 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e215e8b/hadoop-project/src/site/site.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/src/site/site.xml b/hadoop-project/src/site/site.xml
index e1d4c92..6a61a83 100644
--- a/hadoop-project/src/site/site.xml
+++ b/hadoop-project/src/site/site.xml
@@ -123,6 +123,7 @@
       <item name="Writing YARN Applications" href="hadoop-yarn/hadoop-yarn-site/WritingYarnApplications.html"/>
       <item name="YARN Commands" href="hadoop-yarn/hadoop-yarn-site/YarnCommands.html"/>
       <item name="Scheduler Load Simulator" href="hadoop-sls/SchedulerLoadSimulator.html"/>
+      <item name="NodeManager Restart" href="hadoop-yarn/hadoop-yarn-site/NodeManagerRestart.html"/>
     </menu>
 
     <menu name="YARN REST APIs" inherit="top">

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e215e8b/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index e4b116d..df7e3ea 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -195,6 +195,9 @@ Release 2.6.0 - UNRELEASED
     YARN-2647. Added a queue CLI for getting queue information. (Sunil Govind via
     vinodkv)
 
+    YARN-2632. Document NM Restart feature. (Junping Du and Vinod Kumar
+    Vavilapalli via jlowe)
+
   IMPROVEMENTS
 
     YARN-2197. Add a link to YARN CHANGES.txt in the left side of doc

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e215e8b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/NodeManagerRestart.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/NodeManagerRestart.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/NodeManagerRestart.apt.vm
new file mode 100644
index 0000000..ba03f4e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/NodeManagerRestart.apt.vm
@@ -0,0 +1,86 @@
+~~ Licensed 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. See accompanying LICENSE file.
+
+  ---
+  NodeManager Restart
+  ---
+  ---
+  ${maven.build.timestamp}
+
+NodeManager Restart
+
+* Introduction
+
+  This document gives an overview of NodeManager (NM) restart, a feature that
+  enables the NodeManager to be restarted without losing 
+  the active containers running on the node. At a high level, the NM stores any 
+  necessary state to a local state-store as it processes container-management
+  requests. When the NM restarts, it recovers by first loading state for
+  various subsystems and then letting those subsystems perform recovery using
+  the loaded state.
+
+* Enabling NM Restart
+
+  [[1]] To enable NM Restart functionality, set the following property in <<conf/yarn-site.xml>> to true:
+
+*--------------------------------------+--------------------------------------+
+|| Property                            || Value                                |
+*--------------------------------------+--------------------------------------+
+| <<<yarn.nodemanager.recovery.enabled>>> | |
+| | <<<true>>>, (default value is set to false) |
+*--------------------------------------+--------------------------------------+ 
+
+  [[2]] Configure a path to the local file-system directory where the
+  NodeManager can save its run state
+
+*--------------------------------------+--------------------------------------+
+|| Property                            || Description                        |
+*--------------------------------------+--------------------------------------+
+| <<<yarn.nodemanager.recovery.dir>>> | |
+| | The local filesystem directory in which the node manager will store state |
+| | when recovery is enabled.  |
+| | The default value is set to |
+| | <<<${hadoop.tmp.dir}/yarn-nm-recovery>>>. |
+*--------------------------------------+--------------------------------------+ 
+
+  [[3]] Configure a valid RPC address for the NodeManager
+  
+*--------------------------------------+--------------------------------------+
+|| Property                            || Description                        |
+*--------------------------------------+--------------------------------------+
+| <<<yarn.nodemanager.address>>> | |
+| |   Ephemeral ports (port 0, which is default) cannot be used for the |
+| | NodeManager's RPC server specified via yarn.nodemanager.address as it can |
+| | make NM use different ports before and after a restart. This will break any |
+| | previously running clients that were communicating with the NM before |
+| | restart. Explicitly setting yarn.nodemanager.address to an address with |
+| | specific port number (for e.g 0.0.0.0:45454) is a precondition for enabling |
+| | NM restart. |
+*--------------------------------------+--------------------------------------+
+
+  [[4]] Auxiliary services
+  
+  NodeManagers in a YARN cluster can be configured to run auxiliary services.
+  For a completely functional NM restart, YARN relies on any auxiliary service
+  configured to also support recovery. This usually includes (1) avoiding usage
+  of ephemeral ports so that previously running clients (in this case, usually
+  containers) are not disrupted after restart and (2) having the auxiliary
+  service itself support recoverability by reloading any previous state when
+  NodeManager restarts and reinitializes the auxiliary service.
+  
+  A simple example for the above is the auxiliary service 'ShuffleHandler' for
+  MapReduce (MR). ShuffleHandler respects the above two requirements already,
+  so users/admins don't have do anything for it to support NM restart: (1) The
+  configuration property <<mapreduce.shuffle.port>> controls which port the
+  ShuffleHandler on a NodeManager host binds to, and it defaults to a
+  non-ephemeral port. (2) The ShuffleHandler service also already supports
+  recovery of previous state after NM restarts.
\ No newline at end of file


[14/30] hadoop git commit: YARN-2823. Fixed ResourceManager app-attempt state machine to inform schedulers about previous finished attempts of a running appliation to avoid expectation mismatch w.r.t transferred containers. Contributed by Jian He.

Posted by vi...@apache.org.
YARN-2823. Fixed ResourceManager app-attempt state machine to inform schedulers about previous finished attempts of a running appliation to avoid expectation mismatch w.r.t transferred containers. Contributed by Jian He.


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

Branch: refs/heads/HDFS-EC
Commit: a5657182a7accebe08cd86e46b4cdeb163d4d1f2
Parents: 42bbe37
Author: Vinod Kumar Vavilapalli <vi...@apache.org>
Authored: Fri Nov 7 09:28:36 2014 -0800
Committer: Vinod Kumar Vavilapalli <vi...@apache.org>
Committed: Fri Nov 7 09:28:36 2014 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                 | 4 ++++
 .../server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java  | 4 ++++
 .../hadoop/yarn/server/resourcemanager/TestRMRestart.java       | 5 ++++-
 3 files changed, 12 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5657182/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 9dffb22..40f54d4 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -888,6 +888,10 @@ Release 2.6.0 - UNRELEASED
     YARN-2744. Fixed CapacityScheduler to validate node-labels correctly against
     queues. (Wangda Tan via vinodkv)
 
+    YARN-2823. Fixed ResourceManager app-attempt state machine to inform
+    schedulers about previous finished attempts of a running appliation to avoid
+    expectation mismatch w.r.t transferred containers. (Jian He via vinodkv)
+
 Release 2.5.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5657182/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.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/rmapp/attempt/RMAppAttemptImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
index ae11b07..d3fe151 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
@@ -1021,6 +1021,10 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
         // state but application is not in final state.
         if (rmApp.getCurrentAppAttempt() == appAttempt
             && !RMAppImpl.isAppInFinalState(rmApp)) {
+          // Add the previous finished attempt to scheduler synchronously so
+          // that scheduler knows the previous attempt.
+          appAttempt.scheduler.handle(new AppAttemptAddedSchedulerEvent(
+            appAttempt.getAppAttemptId(), false, true));
           (new BaseFinalTransition(appAttempt.recoveredFinalState)).transition(
               appAttempt, event);
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5657182/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.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/TestRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
index 9502eba..a9683f1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
@@ -440,7 +440,10 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     nm1.registerNode();
 
     // create app and launch the AM
-    RMApp app0 = rm1.submitApp(200);
+    RMApp app0 =
+        rm1.submitApp(200, "name", "user",
+          new HashMap<ApplicationAccessType, String>(), false, "default", -1,
+          null, "MAPREDUCE", true, true);
     MockAM am0 = launchAM(app0, rm1, nm1);
 
     // fail the AM by sending CONTAINER_FINISHED event without registering.


[21/30] hadoop git commit: HADOOP-11282. Skip NFS TestShellBasedIdMapping tests that are irrelevant on Windows. Contributed by Chris Nauroth.

Posted by vi...@apache.org.
HADOOP-11282. Skip NFS TestShellBasedIdMapping tests that are irrelevant on Windows. Contributed by Chris Nauroth.


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

Branch: refs/heads/HDFS-EC
Commit: 68db5b3736287ef218acc861368c6b95b72fa2f4
Parents: 4cfd5bc
Author: cnauroth <cn...@apache.org>
Authored: Fri Nov 7 14:30:29 2014 -0800
Committer: cnauroth <cn...@apache.org>
Committed: Fri Nov 7 14:30:29 2014 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt                 | 3 +++
 .../org/apache/hadoop/security/TestShellBasedIdMapping.java     | 5 +++++
 2 files changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/68db5b37/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 0192560..550b009 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1086,6 +1086,9 @@ Release 2.6.0 - UNRELEASED
     HADOOP-11280. TestWinUtils#testChmod fails after removal of
     NO_PROPAGATE_INHERIT_ACE. (cnauroth)
 
+    HADOOP-11282. Skip NFS TestShellBasedIdMapping tests that are irrelevant on
+    Windows. (cnauroth)
+
 Release 2.5.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/68db5b37/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestShellBasedIdMapping.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestShellBasedIdMapping.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestShellBasedIdMapping.java
index a3857e3..808c3fd 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestShellBasedIdMapping.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestShellBasedIdMapping.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.security;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assume.assumeTrue;
 
 import java.io.File;
 import java.io.FileOutputStream;
@@ -27,6 +28,7 @@ import java.io.OutputStream;
 import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.security.ShellBasedIdMapping.PassThroughMap;
 import org.apache.hadoop.security.ShellBasedIdMapping.StaticMapping;
 import org.junit.Test;
@@ -73,6 +75,7 @@ public class TestShellBasedIdMapping {
   
   @Test
   public void testStaticMapping() throws IOException {
+    assumeTrue(!Shell.WINDOWS);
     Map<Integer, Integer> uidStaticMap = new PassThroughMap<Integer>();
     Map<Integer, Integer> gidStaticMap = new PassThroughMap<Integer>();
     
@@ -113,6 +116,7 @@ public class TestShellBasedIdMapping {
 
   @Test
   public void testDuplicates() throws IOException {
+    assumeTrue(!Shell.WINDOWS);
     String GET_ALL_USERS_CMD = "echo \"root:x:0:0:root:/root:/bin/bash\n"
         + "hdfs:x:11501:10787:Grid Distributed File System:/home/hdfs:/bin/bash\n"
         + "hdfs:x:11502:10788:Grid Distributed File System:/home/hdfs:/bin/bash\n"
@@ -152,6 +156,7 @@ public class TestShellBasedIdMapping {
 
   @Test
   public void testIdOutOfIntegerRange() throws IOException {
+    assumeTrue(!Shell.WINDOWS);
     String GET_ALL_USERS_CMD = "echo \""
         + "nfsnobody:x:4294967294:4294967294:Anonymous NFS User:/var/lib/nfs:/sbin/nologin\n"
         + "nfsnobody1:x:4294967295:4294967295:Anonymous NFS User:/var/lib/nfs1:/sbin/nologin\n"