Prechádzať zdrojové kódy

Merging r1519884 through r1520449 from trunk to branch HDFS-2832

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2832@1520454 13f79535-47bb-0310-9956-ffa450edef68
Arpit Agarwal 12 rokov pred
rodič
commit
bbce64c8c5
30 zmenil súbory, kde vykonal 583 pridanie a 90 odobranie
  1. 12 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 8 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
  3. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Stat.java
  4. 3 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
  5. 21 7
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RetryCache.java
  6. 3 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/VersionInfo.java
  7. 16 7
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java
  8. 21 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestStat.java
  9. 31 29
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java
  10. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  11. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
  12. 54 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
  13. 4 0
      hadoop-mapreduce-project/CHANGES.txt
  14. 42 22
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/client/MRClientService.java
  15. 85 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRClientService.java
  16. 7 0
      hadoop-yarn-project/CHANGES.txt
  17. 48 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/AuxiliaryServiceHelper.java
  18. 5 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
  19. 13 2
      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
  20. 6 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncher.java
  21. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/DummyContainerManager.java
  22. 13 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
  23. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java
  24. 34 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java
  25. 5 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
  26. 7 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
  27. 9 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.java
  28. 9 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSSchedulerNode.java
  29. 8 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
  30. 109 0
      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

+ 12 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -330,6 +330,9 @@ Release 2.3.0 - UNRELEASED
     HADOOP-9889. Refresh the Krb5 configuration when creating a new kdc in
     HADOOP-9889. Refresh the Krb5 configuration when creating a new kdc in
     Hadoop-MiniKDC (Wei Yan via Sandy Ryza)
     Hadoop-MiniKDC (Wei Yan via Sandy Ryza)
 
 
+    HADOOP-9915.  o.a.h.fs.Stat support on Mac OS X  (Binglin Chang via Colin
+    Patrick McCabe)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     HADOOP-9748. Reduce blocking on UGI.ensureInitialized (daryn)
     HADOOP-9748. Reduce blocking on UGI.ensureInitialized (daryn)
@@ -360,6 +363,8 @@ Release 2.3.0 - UNRELEASED
     HADOOP-9909. org.apache.hadoop.fs.Stat should permit other LANG.
     HADOOP-9909. org.apache.hadoop.fs.Stat should permit other LANG.
     (Shinichi Yamashita via Andrew Wang)
     (Shinichi Yamashita via Andrew Wang)
 
 
+    HADOOP-9908. Fix NPE when versioninfo properties file is missing (todd)
+
 Release 2.1.1-beta - UNRELEASED
 Release 2.1.1-beta - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -406,6 +411,8 @@ Release 2.1.1-beta - UNRELEASED
 
 
   BUG FIXES
   BUG FIXES
 
 
+    HADOOP-9916. Fix race in ipc.Client retry. (Binglin Chang via llu)
+
     HADOOP-9768. chown and chgrp reject users and groups with spaces on platforms
     HADOOP-9768. chown and chgrp reject users and groups with spaces on platforms
     where spaces are otherwise acceptable. (cnauroth)
     where spaces are otherwise acceptable. (cnauroth)
 
 
@@ -444,6 +451,11 @@ Release 2.1.1-beta - UNRELEASED
     HADOOP-9774. RawLocalFileSystem.listStatus() return absolute paths when
     HADOOP-9774. RawLocalFileSystem.listStatus() return absolute paths when
     input path is relative on Windows. (Shanyu Zhao via ivanmi)
     input path is relative on Windows. (Shanyu Zhao via ivanmi)
 
 
+    HADOOP-9924. FileUtil.createJarWithClassPath() does not generate relative
+    classpath correctly. (Shanyu Zhao via ivanmi)
+
+    HADOOP-9932. Improper synchronization in RetryCache. (kihwal)
+
 Release 2.1.0-beta - 2013-08-22
 Release 2.1.0-beta - 2013-08-22
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 8 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java

@@ -1252,7 +1252,14 @@ public class FileUtil {
         }
         }
       } else {
       } else {
         // Append just this entry
         // Append just this entry
-        String classPathEntryUrl = new File(classPathEntry).toURI().toURL()
+        File fileCpEntry = null;
+        if(!new Path(classPathEntry).isAbsolute()) {
+          fileCpEntry = new File(workingDir, classPathEntry);
+        }
+        else {
+          fileCpEntry = new File(classPathEntry);
+        }
+        String classPathEntryUrl = fileCpEntry.toURI().toURL()
           .toExternalForm();
           .toExternalForm();
 
 
         // File.toURI only appends trailing '/' if it can determine that it is a
         // File.toURI only appends trailing '/' if it can determine that it is a

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Stat.java

@@ -80,7 +80,7 @@ public class Stat extends Shell {
    * @return
    * @return
    */
    */
   public static boolean isAvailable() {
   public static boolean isAvailable() {
-    if (Shell.LINUX || Shell.FREEBSD) {
+    if (Shell.LINUX || Shell.FREEBSD || Shell.MAC) {
       return true;
       return true;
     }
     }
     return false;
     return false;
@@ -100,7 +100,7 @@ public class Stat extends Shell {
     if (Shell.LINUX) {
     if (Shell.LINUX) {
       return new String[] {
       return new String[] {
           "stat", derefFlag + "c", "%s,%F,%Y,%X,%a,%U,%G,%N", path.toString() };
           "stat", derefFlag + "c", "%s,%F,%Y,%X,%a,%U,%G,%N", path.toString() };
-    } else if (Shell.FREEBSD) {
+    } else if (Shell.FREEBSD || Shell.MAC) {
       return new String[] {
       return new String[] {
           "stat", derefFlag + "f", "%z,%HT,%m,%a,%Op,%Su,%Sg,`link' -> `%Y'",
           "stat", derefFlag + "f", "%z,%HT,%m,%a,%Op,%Su,%Sg,`link' -> `%Y'",
           path.toString() };
           path.toString() };

+ 3 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java

@@ -1063,8 +1063,8 @@ public class Client {
         if (status == RpcStatusProto.SUCCESS) {
         if (status == RpcStatusProto.SUCCESS) {
           Writable value = ReflectionUtils.newInstance(valueClass, conf);
           Writable value = ReflectionUtils.newInstance(valueClass, conf);
           value.readFields(in);                 // read value
           value.readFields(in);                 // read value
-          call.setRpcResponse(value);
           calls.remove(callId);
           calls.remove(callId);
+          call.setRpcResponse(value);
           
           
           // verify that length was correct
           // verify that length was correct
           // only for ProtobufEngine where len can be verified easily
           // only for ProtobufEngine where len can be verified easily
@@ -1098,8 +1098,8 @@ public class Client {
                   new RemoteException(exceptionClassName, errorMsg) :
                   new RemoteException(exceptionClassName, errorMsg) :
               new RemoteException(exceptionClassName, errorMsg, erCode));
               new RemoteException(exceptionClassName, errorMsg, erCode));
           if (status == RpcStatusProto.ERROR) {
           if (status == RpcStatusProto.ERROR) {
-            call.setException(re);
             calls.remove(callId);
             calls.remove(callId);
+            call.setException(re);
           } else if (status == RpcStatusProto.FATAL) {
           } else if (status == RpcStatusProto.FATAL) {
             // Close the connection
             // Close the connection
             markClosed(re);
             markClosed(re);
@@ -1166,8 +1166,8 @@ public class Client {
       Iterator<Entry<Integer, Call>> itor = calls.entrySet().iterator() ;
       Iterator<Entry<Integer, Call>> itor = calls.entrySet().iterator() ;
       while (itor.hasNext()) {
       while (itor.hasNext()) {
         Call c = itor.next().getValue(); 
         Call c = itor.next().getValue(); 
+        itor.remove();
         c.setException(closeException); // local exception
         c.setException(closeException); // local exception
-        itor.remove();         
       }
       }
     }
     }
   }
   }

+ 21 - 7
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RetryCache.java

@@ -76,6 +76,12 @@ public class RetryCache {
       this.expirationTime = expirationTime;
       this.expirationTime = expirationTime;
     }
     }
 
 
+    CacheEntry(byte[] clientId, int callId, long expirationTime,
+        boolean success) {
+      this(clientId, callId, expirationTime);
+      this.state = success ? SUCCESS : FAILED;
+    }
+
     private static int hashCode(long value) {
     private static int hashCode(long value) {
       return (int)(value ^ (value >>> 32));
       return (int)(value ^ (value >>> 32));
     }
     }
@@ -147,6 +153,12 @@ public class RetryCache {
       this.payload = payload;
       this.payload = payload;
     }
     }
 
 
+    CacheEntryWithPayload(byte[] clientId, int callId, Object payload,
+        long expirationTime, boolean success) {
+     super(clientId, callId, expirationTime, success);
+     this.payload = payload;
+   }
+
     /** Override equals to avoid findbugs warnings */
     /** Override equals to avoid findbugs warnings */
     @Override
     @Override
     public boolean equals(Object obj) {
     public boolean equals(Object obj) {
@@ -253,18 +265,20 @@ public class RetryCache {
    */
    */
   public void addCacheEntry(byte[] clientId, int callId) {
   public void addCacheEntry(byte[] clientId, int callId) {
     CacheEntry newEntry = new CacheEntry(clientId, callId, System.nanoTime()
     CacheEntry newEntry = new CacheEntry(clientId, callId, System.nanoTime()
-        + expirationTime);
-    newEntry.completed(true);
-    set.put(newEntry);
+        + expirationTime, true);
+    synchronized(this) {
+      set.put(newEntry);
+    }
   }
   }
   
   
   public void addCacheEntryWithPayload(byte[] clientId, int callId,
   public void addCacheEntryWithPayload(byte[] clientId, int callId,
       Object payload) {
       Object payload) {
-    CacheEntry newEntry = new CacheEntryWithPayload(clientId, callId, payload,
-        System.nanoTime() + expirationTime);
     // since the entry is loaded from editlog, we can assume it succeeded.    
     // since the entry is loaded from editlog, we can assume it succeeded.    
-    newEntry.completed(true);
-    set.put(newEntry);
+    CacheEntry newEntry = new CacheEntryWithPayload(clientId, callId, payload,
+        System.nanoTime() + expirationTime, true);
+    synchronized(this) {
+      set.put(newEntry);
+    }
   }
   }
 
 
   private static CacheEntry newEntry(long expirationTime) {
   private static CacheEntry newEntry(long expirationTime) {

+ 3 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/VersionInfo.java

@@ -48,6 +48,9 @@ public class VersionInfo {
     try {
     try {
       InputStream is = Thread.currentThread().getContextClassLoader()
       InputStream is = Thread.currentThread().getContextClassLoader()
         .getResourceAsStream(versionInfoFile);
         .getResourceAsStream(versionInfoFile);
+      if (is == null) {
+        throw new IOException("Resource not found");
+      }
       info.load(is);
       info.load(is);
     } catch (IOException ex) {
     } catch (IOException ex) {
       LogFactory.getLog(getClass()).warn("Could not read '" + 
       LogFactory.getLog(getClass()).warn("Could not read '" + 

+ 16 - 7
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java

@@ -782,14 +782,23 @@ public class TestFileUtil {
             expectedClassPaths.add(wildcardMatch.toURI().toURL()
             expectedClassPaths.add(wildcardMatch.toURI().toURL()
               .toExternalForm());
               .toExternalForm());
           }
           }
-        } else if (nonExistentSubdir.equals(classPath)) {
-          // expect to maintain trailing path separator if present in input, even
-          // if directory doesn't exist yet
-          expectedClassPaths.add(new File(classPath).toURI().toURL()
-            .toExternalForm() + Path.SEPARATOR);
         } else {
         } else {
-          expectedClassPaths.add(new File(classPath).toURI().toURL()
-            .toExternalForm());
+          File fileCp = null;
+          if(!new Path(classPath).isAbsolute()) {
+            fileCp = new File(tmp, classPath);
+          }
+          else {
+            fileCp = new File(classPath);
+          }
+          if (nonExistentSubdir.equals(classPath)) {
+            // expect to maintain trailing path separator if present in input, even
+            // if directory doesn't exist yet
+            expectedClassPaths.add(fileCp.toURI().toURL()
+              .toExternalForm() + Path.SEPARATOR);
+          } else {
+            expectedClassPaths.add(fileCp.toURI().toURL()
+              .toExternalForm());
+          }
         }
         }
       }
       }
       List<String> actualClassPaths = Arrays.asList(classPathAttr.split(" "));
       List<String> actualClassPaths = Arrays.asList(classPathAttr.split(" "));

+ 21 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestStat.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.fs;
 
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.fail;
 import static org.junit.Assert.fail;
 
 
 import java.io.BufferedReader;
 import java.io.BufferedReader;
@@ -26,10 +27,11 @@ import java.io.FileNotFoundException;
 import java.io.StringReader;
 import java.io.StringReader;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.junit.Assume;
 import org.junit.BeforeClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.Test;
 
 
-public class TestStat {
+public class TestStat extends FileSystemTestHelper {
 
 
   private static Stat stat;
   private static Stat stat;
 
 
@@ -113,6 +115,7 @@ public class TestStat {
 
 
   @Test(timeout=10000)
   @Test(timeout=10000)
   public void testStatFileNotFound() throws Exception {
   public void testStatFileNotFound() throws Exception {
+    Assume.assumeTrue(Stat.isAvailable());
     try {
     try {
       stat.getFileStatus();
       stat.getFileStatus();
       fail("Expected FileNotFoundException");
       fail("Expected FileNotFoundException");
@@ -125,4 +128,21 @@ public class TestStat {
   public void testStatEnvironment() throws Exception {
   public void testStatEnvironment() throws Exception {
     assertEquals(stat.getEnvironment("LANG"), "C");
     assertEquals(stat.getEnvironment("LANG"), "C");
   }
   }
+
+  @Test(timeout=10000)
+  public void testStat() throws Exception {
+    Assume.assumeTrue(Stat.isAvailable());
+    FileSystem fs = FileSystem.getLocal(new Configuration());
+    Path testDir = new Path(getTestRootPath(fs), "teststat");
+    fs.mkdirs(testDir);
+    Path sub1 = new Path(testDir, "sub1");
+    Path sub2 = new Path(testDir, "sub2");
+    fs.mkdirs(sub1);
+    fs.createSymlink(sub1, sub2, false);
+    FileStatus stat1 = new Stat(sub1, 4096l, false, fs).getFileStatus();
+    FileStatus stat2 = new Stat(sub2, 0, false, fs).getFileStatus();
+    assertTrue(stat1.isDirectory());
+    assertFalse(stat2.isDirectory());
+    fs.delete(testDir, true);
+  }
 }
 }

+ 31 - 29
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java

@@ -216,13 +216,13 @@ public class TestIPC {
     }
     }
   }
   }
   
   
-  @Test
+  @Test(timeout=60000)
   public void testSerial() throws IOException, InterruptedException {
   public void testSerial() throws IOException, InterruptedException {
-    testSerial(3, false, 2, 5, 100);
-    testSerial(3, true, 2, 5, 10);
+    internalTestSerial(3, false, 2, 5, 100);
+    internalTestSerial(3, true, 2, 5, 10);
   }
   }
 
 
-  public void testSerial(int handlerCount, boolean handlerSleep, 
+  public void internalTestSerial(int handlerCount, boolean handlerSleep,
                          int clientCount, int callerCount, int callCount)
                          int clientCount, int callerCount, int callCount)
     throws IOException, InterruptedException {
     throws IOException, InterruptedException {
     Server server = new TestServer(handlerCount, handlerSleep);
     Server server = new TestServer(handlerCount, handlerSleep);
@@ -249,7 +249,7 @@ public class TestIPC {
     server.stop();
     server.stop();
   }
   }
 	
 	
-  @Test
+  @Test(timeout=60000)
   public void testStandAloneClient() throws IOException {
   public void testStandAloneClient() throws IOException {
     Client client = new Client(LongWritable.class, conf);
     Client client = new Client(LongWritable.class, conf);
     InetSocketAddress address = new InetSocketAddress("127.0.0.1", 10);
     InetSocketAddress address = new InetSocketAddress("127.0.0.1", 10);
@@ -383,7 +383,7 @@ public class TestIPC {
     }
     }
   }
   }
 
 
-  @Test
+  @Test(timeout=60000)
   public void testIOEOnClientWriteParam() throws Exception {
   public void testIOEOnClientWriteParam() throws Exception {
     doErrorTest(IOEOnWriteWritable.class,
     doErrorTest(IOEOnWriteWritable.class,
         LongWritable.class,
         LongWritable.class,
@@ -391,7 +391,7 @@ public class TestIPC {
         LongWritable.class);
         LongWritable.class);
   }
   }
   
   
-  @Test
+  @Test(timeout=60000)
   public void testRTEOnClientWriteParam() throws Exception {
   public void testRTEOnClientWriteParam() throws Exception {
     doErrorTest(RTEOnWriteWritable.class,
     doErrorTest(RTEOnWriteWritable.class,
         LongWritable.class,
         LongWritable.class,
@@ -399,7 +399,7 @@ public class TestIPC {
         LongWritable.class);
         LongWritable.class);
   }
   }
 
 
-  @Test
+  @Test(timeout=60000)
   public void testIOEOnServerReadParam() throws Exception {
   public void testIOEOnServerReadParam() throws Exception {
     doErrorTest(LongWritable.class,
     doErrorTest(LongWritable.class,
         IOEOnReadWritable.class,
         IOEOnReadWritable.class,
@@ -407,7 +407,7 @@ public class TestIPC {
         LongWritable.class);
         LongWritable.class);
   }
   }
   
   
-  @Test
+  @Test(timeout=60000)
   public void testRTEOnServerReadParam() throws Exception {
   public void testRTEOnServerReadParam() throws Exception {
     doErrorTest(LongWritable.class,
     doErrorTest(LongWritable.class,
         RTEOnReadWritable.class,
         RTEOnReadWritable.class,
@@ -416,7 +416,7 @@ public class TestIPC {
   }
   }
 
 
   
   
-  @Test
+  @Test(timeout=60000)
   public void testIOEOnServerWriteResponse() throws Exception {
   public void testIOEOnServerWriteResponse() throws Exception {
     doErrorTest(LongWritable.class,
     doErrorTest(LongWritable.class,
         LongWritable.class,
         LongWritable.class,
@@ -424,7 +424,7 @@ public class TestIPC {
         LongWritable.class);
         LongWritable.class);
   }
   }
   
   
-  @Test
+  @Test(timeout=60000)
   public void testRTEOnServerWriteResponse() throws Exception {
   public void testRTEOnServerWriteResponse() throws Exception {
     doErrorTest(LongWritable.class,
     doErrorTest(LongWritable.class,
         LongWritable.class,
         LongWritable.class,
@@ -432,7 +432,7 @@ public class TestIPC {
         LongWritable.class);
         LongWritable.class);
   }
   }
   
   
-  @Test
+  @Test(timeout=60000)
   public void testIOEOnClientReadResponse() throws Exception {
   public void testIOEOnClientReadResponse() throws Exception {
     doErrorTest(LongWritable.class,
     doErrorTest(LongWritable.class,
         LongWritable.class,
         LongWritable.class,
@@ -440,7 +440,7 @@ public class TestIPC {
         IOEOnReadWritable.class);
         IOEOnReadWritable.class);
   }
   }
   
   
-  @Test
+  @Test(timeout=60000)
   public void testRTEOnClientReadResponse() throws Exception {
   public void testRTEOnClientReadResponse() throws Exception {
     doErrorTest(LongWritable.class,
     doErrorTest(LongWritable.class,
         LongWritable.class,
         LongWritable.class,
@@ -453,7 +453,7 @@ public class TestIPC {
    * that a ping should have been sent. This is a reproducer for a
    * that a ping should have been sent. This is a reproducer for a
    * deadlock seen in one iteration of HADOOP-6762.
    * deadlock seen in one iteration of HADOOP-6762.
    */
    */
-  @Test
+  @Test(timeout=60000)
   public void testIOEOnWriteAfterPingClient() throws Exception {
   public void testIOEOnWriteAfterPingClient() throws Exception {
     // start server
     // start server
     Client.setPingInterval(conf, 100);
     Client.setPingInterval(conf, 100);
@@ -481,7 +481,7 @@ public class TestIPC {
    * Test that, if the socket factory throws an IOE, it properly propagates
    * Test that, if the socket factory throws an IOE, it properly propagates
    * to the client.
    * to the client.
    */
    */
-  @Test
+  @Test(timeout=60000)
   public void testSocketFactoryException() throws IOException {
   public void testSocketFactoryException() throws IOException {
     SocketFactory mockFactory = mock(SocketFactory.class);
     SocketFactory mockFactory = mock(SocketFactory.class);
     doThrow(new IOException("Injected fault")).when(mockFactory).createSocket();
     doThrow(new IOException("Injected fault")).when(mockFactory).createSocket();
@@ -503,7 +503,7 @@ public class TestIPC {
    * failure is handled properly. This is a regression test for
    * failure is handled properly. This is a regression test for
    * HADOOP-7428.
    * HADOOP-7428.
    */
    */
-  @Test
+  @Test(timeout=60000)
   public void testRTEDuringConnectionSetup() throws IOException {
   public void testRTEDuringConnectionSetup() throws IOException {
     // Set up a socket factory which returns sockets which
     // Set up a socket factory which returns sockets which
     // throw an RTE when setSoTimeout is called.
     // throw an RTE when setSoTimeout is called.
@@ -544,7 +544,7 @@ public class TestIPC {
     }
     }
   }
   }
   
   
-  @Test
+  @Test(timeout=60000)
   public void testIpcTimeout() throws IOException {
   public void testIpcTimeout() throws IOException {
     // start server
     // start server
     Server server = new TestServer(1, true);
     Server server = new TestServer(1, true);
@@ -566,7 +566,7 @@ public class TestIPC {
         addr, null, null, 3*PING_INTERVAL+MIN_SLEEP_TIME, conf);
         addr, null, null, 3*PING_INTERVAL+MIN_SLEEP_TIME, conf);
   }
   }
 
 
-  @Test
+  @Test(timeout=60000)
   public void testIpcConnectTimeout() throws IOException {
   public void testIpcConnectTimeout() throws IOException {
     // start server
     // start server
     Server server = new TestServer(1, true);
     Server server = new TestServer(1, true);
@@ -670,31 +670,31 @@ public class TestIPC {
     return FD_DIR.list().length;
     return FD_DIR.list().length;
   }
   }
 
 
-  @Test
+  @Test(timeout=60000)
   public void testIpcFromHadoop_0_18_13() throws IOException {
   public void testIpcFromHadoop_0_18_13() throws IOException {
     doIpcVersionTest(NetworkTraces.HADOOP_0_18_3_RPC_DUMP,
     doIpcVersionTest(NetworkTraces.HADOOP_0_18_3_RPC_DUMP,
         NetworkTraces.RESPONSE_TO_HADOOP_0_18_3_RPC);
         NetworkTraces.RESPONSE_TO_HADOOP_0_18_3_RPC);
   }
   }
   
   
-  @Test
+  @Test(timeout=60000)
   public void testIpcFromHadoop0_20_3() throws IOException {
   public void testIpcFromHadoop0_20_3() throws IOException {
     doIpcVersionTest(NetworkTraces.HADOOP_0_20_3_RPC_DUMP,
     doIpcVersionTest(NetworkTraces.HADOOP_0_20_3_RPC_DUMP,
         NetworkTraces.RESPONSE_TO_HADOOP_0_20_3_RPC);
         NetworkTraces.RESPONSE_TO_HADOOP_0_20_3_RPC);
   }
   }
   
   
-  @Test
+  @Test(timeout=60000)
   public void testIpcFromHadoop0_21_0() throws IOException {
   public void testIpcFromHadoop0_21_0() throws IOException {
     doIpcVersionTest(NetworkTraces.HADOOP_0_21_0_RPC_DUMP,
     doIpcVersionTest(NetworkTraces.HADOOP_0_21_0_RPC_DUMP,
         NetworkTraces.RESPONSE_TO_HADOOP_0_21_0_RPC);
         NetworkTraces.RESPONSE_TO_HADOOP_0_21_0_RPC);
   }
   }
   
   
-  @Test
+  @Test(timeout=60000)
   public void testHttpGetResponse() throws IOException {
   public void testHttpGetResponse() throws IOException {
     doIpcVersionTest("GET / HTTP/1.0\r\n\r\n".getBytes(),
     doIpcVersionTest("GET / HTTP/1.0\r\n\r\n".getBytes(),
         Server.RECEIVED_HTTP_REQ_RESPONSE.getBytes());
         Server.RECEIVED_HTTP_REQ_RESPONSE.getBytes());
   }
   }
   
   
-  @Test
+  @Test(timeout=60000)
   public void testConnectionRetriesOnSocketTimeoutExceptions() throws IOException {
   public void testConnectionRetriesOnSocketTimeoutExceptions() throws IOException {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     // set max retries to 0
     // set max retries to 0
@@ -720,7 +720,7 @@ public class TestIPC {
    * (1) the rpc server uses the call id/retry provided by the rpc client, and
    * (1) the rpc server uses the call id/retry provided by the rpc client, and
    * (2) the rpc client receives the same call id/retry from the rpc server.
    * (2) the rpc client receives the same call id/retry from the rpc server.
    */
    */
-  @Test
+  @Test(timeout=60000)
   public void testCallIdAndRetry() throws IOException {
   public void testCallIdAndRetry() throws IOException {
     final CallInfo info = new CallInfo();
     final CallInfo info = new CallInfo();
 
 
@@ -772,7 +772,7 @@ public class TestIPC {
   /**
   /**
    * Test the retry count while used in a retry proxy.
    * Test the retry count while used in a retry proxy.
    */
    */
-  @Test
+  @Test(timeout=60000)
   public void testRetryProxy() throws IOException {
   public void testRetryProxy() throws IOException {
     final Client client = new Client(LongWritable.class, conf);
     final Client client = new Client(LongWritable.class, conf);
     
     
@@ -785,7 +785,9 @@ public class TestIPC {
       }
       }
     };
     };
 
 
-    final int totalRetry = 256;
+    // try more times, so it is easier to find race condition bug
+    // 10000 times runs about 6s on a core i7 machine
+    final int totalRetry = 10000;
     DummyProtocol proxy = (DummyProtocol) Proxy.newProxyInstance(
     DummyProtocol proxy = (DummyProtocol) Proxy.newProxyInstance(
         DummyProtocol.class.getClassLoader(),
         DummyProtocol.class.getClassLoader(),
         new Class[] { DummyProtocol.class }, new TestInvocationHandler(client,
         new Class[] { DummyProtocol.class }, new TestInvocationHandler(client,
@@ -807,7 +809,7 @@ public class TestIPC {
   /**
   /**
    * Test if the rpc server gets the default retry count (0) from client.
    * Test if the rpc server gets the default retry count (0) from client.
    */
    */
-  @Test
+  @Test(timeout=60000)
   public void testInitialCallRetryCount() throws IOException {
   public void testInitialCallRetryCount() throws IOException {
     // Override client to store the call id
     // Override client to store the call id
     final Client client = new Client(LongWritable.class, conf);
     final Client client = new Client(LongWritable.class, conf);
@@ -838,7 +840,7 @@ public class TestIPC {
   /**
   /**
    * Test if the rpc server gets the retry count from client.
    * Test if the rpc server gets the retry count from client.
    */
    */
-  @Test
+  @Test(timeout=60000)
   public void testCallRetryCount() throws IOException {
   public void testCallRetryCount() throws IOException {
     final int retryCount = 255;
     final int retryCount = 255;
     // Override client to store the call id
     // Override client to store the call id
@@ -873,7 +875,7 @@ public class TestIPC {
    * even if multiple threads are using the same client.
    * even if multiple threads are using the same client.
  * @throws InterruptedException 
  * @throws InterruptedException 
    */
    */
-  @Test
+  @Test(timeout=60000)
   public void testUniqueSequentialCallIds() 
   public void testUniqueSequentialCallIds() 
       throws IOException, InterruptedException {
       throws IOException, InterruptedException {
     int serverThreads = 10, callerCount = 100, perCallerCallCount = 100;
     int serverThreads = 10, callerCount = 100, perCallerCallCount = 100;

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

@@ -415,6 +415,9 @@ Release 2.1.1-beta - UNRELEASED
     HDFS-5140. Too many safemode monitor threads being created in the standby 
     HDFS-5140. Too many safemode monitor threads being created in the standby 
     namenode causing it to fail with out of memory error. (jing9)
     namenode causing it to fail with out of memory error. (jing9)
 
 
+    HDFS-5159. Secondary NameNode fails to checkpoint if error occurs
+    downloading edits on first checkpoint. (atm)
+
 Release 2.1.0-beta - 2013-08-22
 Release 2.1.0-beta - 2013-08-22
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java

@@ -429,10 +429,8 @@ public class SecondaryNameNode implements Runnable {
             dstImage.getStorage().cTime = sig.cTime;
             dstImage.getStorage().cTime = sig.cTime;
 
 
             // get fsimage
             // get fsimage
-            boolean downloadImage = true;
             if (sig.mostRecentCheckpointTxId ==
             if (sig.mostRecentCheckpointTxId ==
                 dstImage.getStorage().getMostRecentCheckpointTxId()) {
                 dstImage.getStorage().getMostRecentCheckpointTxId()) {
-              downloadImage = false;
               LOG.info("Image has not changed. Will not download image.");
               LOG.info("Image has not changed. Will not download image.");
             } else {
             } else {
               LOG.info("Image has changed. Downloading updated image from NN.");
               LOG.info("Image has changed. Downloading updated image from NN.");
@@ -448,7 +446,9 @@ public class SecondaryNameNode implements Runnable {
                   nnHostPort, log, dstImage.getStorage());
                   nnHostPort, log, dstImage.getStorage());
             }
             }
         
         
-            return Boolean.valueOf(downloadImage);
+            // true if we haven't loaded all the transactions represented by the
+            // downloaded fsimage.
+            return dstImage.getLastAppliedTxId() < sig.mostRecentCheckpointTxId;
           }
           }
         });
         });
         return b.booleanValue();
         return b.booleanValue();

+ 54 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java

@@ -39,7 +39,6 @@ import java.util.Collection;
 import java.util.List;
 import java.util.List;
 
 
 import org.apache.commons.cli.ParseException;
 import org.apache.commons.cli.ParseException;
-import org.apache.commons.io.filefilter.FileFilterUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.commons.logging.impl.Log4JLogger;
@@ -1224,7 +1223,6 @@ public class TestCheckpoint {
   }
   }
   
   
   /* Test case to test CheckpointSignature */
   /* Test case to test CheckpointSignature */
-  @SuppressWarnings("deprecation")
   @Test
   @Test
   public void testCheckpointSignature() throws IOException {
   public void testCheckpointSignature() throws IOException {
 
 
@@ -1562,12 +1560,65 @@ public class TestCheckpoint {
       Mockito.reset(faultInjector);
       Mockito.reset(faultInjector);
     }
     }
   }
   }
+  
+  /**
+   * Test that a fault while downloading edits the first time after the 2NN
+   * starts up does not prevent future checkpointing.
+   */
+  @Test(timeout = 30000)
+  public void testEditFailureOnFirstCheckpoint() throws IOException {
+    Configuration conf = new HdfsConfiguration();
+    SecondaryNameNode secondary = null;
+    MiniDFSCluster cluster = null;
+    FileSystem fs = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDatanodes)
+          .build();
+      cluster.waitActive();
+      fs = cluster.getFileSystem();
+      fs.mkdirs(new Path("test-file-1"));
+      
+      // Make sure the on-disk fsimage on the NN has txid > 0.
+      FSNamesystem fsns = cluster.getNamesystem();
+      fsns.enterSafeMode(false);
+      fsns.saveNamespace();
+      fsns.leaveSafeMode();
+      
+      secondary = startSecondaryNameNode(conf);
+
+      // Cause edit rename to fail during next checkpoint
+      Mockito.doThrow(new IOException("Injecting failure before edit rename"))
+          .when(faultInjector).beforeEditsRename();
+      
+      try {
+        secondary.doCheckpoint();
+        fail("Fault injection failed.");
+      } catch (IOException ioe) {
+        GenericTestUtils.assertExceptionContains(
+            "Injecting failure before edit rename", ioe);
+      }
+      Mockito.reset(faultInjector);
+      
+      // Next checkpoint should succeed
+      secondary.doCheckpoint();
+    } finally {
+      if (secondary != null) {
+        secondary.shutdown();
+      }
+      if (fs != null) {
+        fs.close();
+      }
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+      Mockito.reset(faultInjector);
+    }
+  }
 
 
   /**
   /**
    * Test that the secondary namenode correctly deletes temporary edits
    * Test that the secondary namenode correctly deletes temporary edits
    * on startup.
    * on startup.
    */
    */
-
   @Test(timeout = 30000)
   @Test(timeout = 30000)
   public void testDeleteTemporaryEditsOnStartup() throws IOException {
   public void testDeleteTemporaryEditsOnStartup() throws IOException {
     Configuration conf = new HdfsConfiguration();
     Configuration conf = new HdfsConfiguration();
@@ -1943,7 +1994,6 @@ public class TestCheckpoint {
    * Test that, if a storage directory is failed when a checkpoint occurs,
    * Test that, if a storage directory is failed when a checkpoint occurs,
    * the non-failed storage directory receives the checkpoint.
    * the non-failed storage directory receives the checkpoint.
    */
    */
-  @SuppressWarnings("deprecation")
   @Test
   @Test
   public void testCheckpointWithFailedStorageDir() throws Exception {
   public void testCheckpointWithFailedStorageDir() throws Exception {
     MiniDFSCluster cluster = null;
     MiniDFSCluster cluster = null;
@@ -2006,7 +2056,6 @@ public class TestCheckpoint {
    * should function correctly.
    * should function correctly.
    * @throws Exception
    * @throws Exception
    */
    */
-  @SuppressWarnings("deprecation")
   @Test
   @Test
   public void testCheckpointWithSeparateDirsAfterNameFails() throws Exception {
   public void testCheckpointWithSeparateDirsAfterNameFails() throws Exception {
     MiniDFSCluster cluster = null;
     MiniDFSCluster cluster = null;

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

@@ -251,6 +251,8 @@ Release 2.1.1-beta - UNRELEASED
     commands to reboot, so that client can continue to track the overall job.
     commands to reboot, so that client can continue to track the overall job.
     (Jian He via vinodkv)
     (Jian He via vinodkv)
 
 
+    MAPREDUCE-5475. MRClientService does not verify ACLs properly (jlowe)
+
 Release 2.1.0-beta - 2013-08-22
 Release 2.1.0-beta - 2013-08-22
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -1329,6 +1331,8 @@ Release 0.23.10 - UNRELEASED
     MAPREDUCE-5001. LocalJobRunner has race condition resulting in job
     MAPREDUCE-5001. LocalJobRunner has race condition resulting in job
     failures (Sandy Ryza via jlowe)
     failures (Sandy Ryza via jlowe)
 
 
+    MAPREDUCE-5475. MRClientService does not verify ACLs properly (jlowe)
+
 Release 0.23.9 - 2013-07-08
 Release 0.23.9 - 2013-07-08
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 42 - 22
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/client/MRClientService.java

@@ -28,6 +28,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.TypeConverter;
 import org.apache.hadoop.mapreduce.TypeConverter;
 import org.apache.hadoop.mapreduce.v2.api.MRClientProtocol;
 import org.apache.hadoop.mapreduce.v2.api.MRClientProtocol;
@@ -78,6 +79,8 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType;
 import org.apache.hadoop.mapreduce.v2.app.security.authorize.MRAMPolicyProvider;
 import org.apache.hadoop.mapreduce.v2.app.security.authorize.MRAMPolicyProvider;
 import org.apache.hadoop.mapreduce.v2.app.webapp.AMWebApp;
 import org.apache.hadoop.mapreduce.v2.app.webapp.AMWebApp;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.PolicyProvider;
 import org.apache.hadoop.security.authorize.PolicyProvider;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factories.RecordFactory;
@@ -175,16 +178,22 @@ public class MRClientService extends AbstractService
       return getBindAddress();
       return getBindAddress();
     }
     }
     
     
-    private Job verifyAndGetJob(JobId jobID, 
-        boolean modifyAccess) throws IOException {
+    private Job verifyAndGetJob(JobId jobID,
+        JobACL accessType) throws IOException {
       Job job = appContext.getJob(jobID);
       Job job = appContext.getJob(jobID);
+      UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+      if (!job.checkAccess(ugi, accessType)) {
+        throw new AccessControlException("User " + ugi.getShortUserName()
+            + " cannot perform operation " + accessType.name() + " on "
+            + jobID);
+      }
       return job;
       return job;
     }
     }
  
  
     private Task verifyAndGetTask(TaskId taskID, 
     private Task verifyAndGetTask(TaskId taskID, 
-        boolean modifyAccess) throws IOException {
+        JobACL accessType) throws IOException {
       Task task = verifyAndGetJob(taskID.getJobId(), 
       Task task = verifyAndGetJob(taskID.getJobId(), 
-          modifyAccess).getTask(taskID);
+          accessType).getTask(taskID);
       if (task == null) {
       if (task == null) {
         throw new IOException("Unknown Task " + taskID);
         throw new IOException("Unknown Task " + taskID);
       }
       }
@@ -192,9 +201,9 @@ public class MRClientService extends AbstractService
     }
     }
 
 
     private TaskAttempt verifyAndGetAttempt(TaskAttemptId attemptID, 
     private TaskAttempt verifyAndGetAttempt(TaskAttemptId attemptID, 
-        boolean modifyAccess) throws IOException {
+        JobACL accessType) throws IOException {
       TaskAttempt attempt = verifyAndGetTask(attemptID.getTaskId(), 
       TaskAttempt attempt = verifyAndGetTask(attemptID.getTaskId(), 
-          modifyAccess).getAttempt(attemptID);
+          accessType).getAttempt(attemptID);
       if (attempt == null) {
       if (attempt == null) {
         throw new IOException("Unknown TaskAttempt " + attemptID);
         throw new IOException("Unknown TaskAttempt " + attemptID);
       }
       }
@@ -205,7 +214,7 @@ public class MRClientService extends AbstractService
     public GetCountersResponse getCounters(GetCountersRequest request) 
     public GetCountersResponse getCounters(GetCountersRequest request) 
       throws IOException {
       throws IOException {
       JobId jobId = request.getJobId();
       JobId jobId = request.getJobId();
-      Job job = verifyAndGetJob(jobId, false);
+      Job job = verifyAndGetJob(jobId, JobACL.VIEW_JOB);
       GetCountersResponse response =
       GetCountersResponse response =
         recordFactory.newRecordInstance(GetCountersResponse.class);
         recordFactory.newRecordInstance(GetCountersResponse.class);
       response.setCounters(TypeConverter.toYarn(job.getAllCounters()));
       response.setCounters(TypeConverter.toYarn(job.getAllCounters()));
@@ -216,7 +225,7 @@ public class MRClientService extends AbstractService
     public GetJobReportResponse getJobReport(GetJobReportRequest request) 
     public GetJobReportResponse getJobReport(GetJobReportRequest request) 
       throws IOException {
       throws IOException {
       JobId jobId = request.getJobId();
       JobId jobId = request.getJobId();
-      Job job = verifyAndGetJob(jobId, false);
+      Job job = verifyAndGetJob(jobId, JobACL.VIEW_JOB);
       GetJobReportResponse response = 
       GetJobReportResponse response = 
         recordFactory.newRecordInstance(GetJobReportResponse.class);
         recordFactory.newRecordInstance(GetJobReportResponse.class);
       if (job != null) {
       if (job != null) {
@@ -235,7 +244,7 @@ public class MRClientService extends AbstractService
       GetTaskAttemptReportResponse response =
       GetTaskAttemptReportResponse response =
         recordFactory.newRecordInstance(GetTaskAttemptReportResponse.class);
         recordFactory.newRecordInstance(GetTaskAttemptReportResponse.class);
       response.setTaskAttemptReport(
       response.setTaskAttemptReport(
-          verifyAndGetAttempt(taskAttemptId, false).getReport());
+          verifyAndGetAttempt(taskAttemptId, JobACL.VIEW_JOB).getReport());
       return response;
       return response;
     }
     }
 
 
@@ -245,7 +254,8 @@ public class MRClientService extends AbstractService
       TaskId taskId = request.getTaskId();
       TaskId taskId = request.getTaskId();
       GetTaskReportResponse response = 
       GetTaskReportResponse response = 
         recordFactory.newRecordInstance(GetTaskReportResponse.class);
         recordFactory.newRecordInstance(GetTaskReportResponse.class);
-      response.setTaskReport(verifyAndGetTask(taskId, false).getReport());
+      response.setTaskReport(
+          verifyAndGetTask(taskId, JobACL.VIEW_JOB).getReport());
       return response;
       return response;
     }
     }
 
 
@@ -256,7 +266,7 @@ public class MRClientService extends AbstractService
       JobId jobId = request.getJobId();
       JobId jobId = request.getJobId();
       int fromEventId = request.getFromEventId();
       int fromEventId = request.getFromEventId();
       int maxEvents = request.getMaxEvents();
       int maxEvents = request.getMaxEvents();
-      Job job = verifyAndGetJob(jobId, false);
+      Job job = verifyAndGetJob(jobId, JobACL.VIEW_JOB);
       
       
       GetTaskAttemptCompletionEventsResponse response = 
       GetTaskAttemptCompletionEventsResponse response = 
         recordFactory.newRecordInstance(GetTaskAttemptCompletionEventsResponse.class);
         recordFactory.newRecordInstance(GetTaskAttemptCompletionEventsResponse.class);
@@ -270,9 +280,11 @@ public class MRClientService extends AbstractService
     public KillJobResponse killJob(KillJobRequest request) 
     public KillJobResponse killJob(KillJobRequest request) 
       throws IOException {
       throws IOException {
       JobId jobId = request.getJobId();
       JobId jobId = request.getJobId();
-      String message = "Kill Job received from client " + jobId;
+      UserGroupInformation callerUGI = UserGroupInformation.getCurrentUser();
+      String message = "Kill job " + jobId + " received from " + callerUGI
+          + " at " + Server.getRemoteAddress();
       LOG.info(message);
       LOG.info(message);
-  	  verifyAndGetJob(jobId, true);
+      verifyAndGetJob(jobId, JobACL.MODIFY_JOB);
       appContext.getEventHandler().handle(
       appContext.getEventHandler().handle(
           new JobDiagnosticsUpdateEvent(jobId, message));
           new JobDiagnosticsUpdateEvent(jobId, message));
       appContext.getEventHandler().handle(
       appContext.getEventHandler().handle(
@@ -287,9 +299,11 @@ public class MRClientService extends AbstractService
     public KillTaskResponse killTask(KillTaskRequest request) 
     public KillTaskResponse killTask(KillTaskRequest request) 
       throws IOException {
       throws IOException {
       TaskId taskId = request.getTaskId();
       TaskId taskId = request.getTaskId();
-      String message = "Kill task received from client " + taskId;
+      UserGroupInformation callerUGI = UserGroupInformation.getCurrentUser();
+      String message = "Kill task " + taskId + " received from " + callerUGI
+          + " at " + Server.getRemoteAddress();
       LOG.info(message);
       LOG.info(message);
-      verifyAndGetTask(taskId, true);
+      verifyAndGetTask(taskId, JobACL.MODIFY_JOB);
       appContext.getEventHandler().handle(
       appContext.getEventHandler().handle(
           new TaskEvent(taskId, TaskEventType.T_KILL));
           new TaskEvent(taskId, TaskEventType.T_KILL));
       KillTaskResponse response = 
       KillTaskResponse response = 
@@ -302,9 +316,12 @@ public class MRClientService extends AbstractService
     public KillTaskAttemptResponse killTaskAttempt(
     public KillTaskAttemptResponse killTaskAttempt(
         KillTaskAttemptRequest request) throws IOException {
         KillTaskAttemptRequest request) throws IOException {
       TaskAttemptId taskAttemptId = request.getTaskAttemptId();
       TaskAttemptId taskAttemptId = request.getTaskAttemptId();
-      String message = "Kill task attempt received from client " + taskAttemptId;
+      UserGroupInformation callerUGI = UserGroupInformation.getCurrentUser();
+      String message = "Kill task attempt " + taskAttemptId
+          + " received from " + callerUGI + " at "
+          + Server.getRemoteAddress();
       LOG.info(message);
       LOG.info(message);
-      verifyAndGetAttempt(taskAttemptId, true);
+      verifyAndGetAttempt(taskAttemptId, JobACL.MODIFY_JOB);
       appContext.getEventHandler().handle(
       appContext.getEventHandler().handle(
           new TaskAttemptDiagnosticsUpdateEvent(taskAttemptId, message));
           new TaskAttemptDiagnosticsUpdateEvent(taskAttemptId, message));
       appContext.getEventHandler().handle(
       appContext.getEventHandler().handle(
@@ -322,8 +339,8 @@ public class MRClientService extends AbstractService
       
       
       GetDiagnosticsResponse response = 
       GetDiagnosticsResponse response = 
         recordFactory.newRecordInstance(GetDiagnosticsResponse.class);
         recordFactory.newRecordInstance(GetDiagnosticsResponse.class);
-      response.addAllDiagnostics(
-          verifyAndGetAttempt(taskAttemptId, false).getDiagnostics());
+      response.addAllDiagnostics(verifyAndGetAttempt(taskAttemptId,
+          JobACL.VIEW_JOB).getDiagnostics());
       return response;
       return response;
     }
     }
 
 
@@ -332,9 +349,12 @@ public class MRClientService extends AbstractService
     public FailTaskAttemptResponse failTaskAttempt(
     public FailTaskAttemptResponse failTaskAttempt(
         FailTaskAttemptRequest request) throws IOException {
         FailTaskAttemptRequest request) throws IOException {
       TaskAttemptId taskAttemptId = request.getTaskAttemptId();
       TaskAttemptId taskAttemptId = request.getTaskAttemptId();
-      String message = "Fail task attempt received from client " + taskAttemptId;
+      UserGroupInformation callerUGI = UserGroupInformation.getCurrentUser();
+      String message = "Fail task attempt " + taskAttemptId
+          + " received from " + callerUGI + " at "
+          + Server.getRemoteAddress();
       LOG.info(message);
       LOG.info(message);
-      verifyAndGetAttempt(taskAttemptId, true);
+      verifyAndGetAttempt(taskAttemptId, JobACL.MODIFY_JOB);
       appContext.getEventHandler().handle(
       appContext.getEventHandler().handle(
           new TaskAttemptDiagnosticsUpdateEvent(taskAttemptId, message));
           new TaskAttemptDiagnosticsUpdateEvent(taskAttemptId, message));
       appContext.getEventHandler().handle(
       appContext.getEventHandler().handle(
@@ -356,7 +376,7 @@ public class MRClientService extends AbstractService
       GetTaskReportsResponse response = 
       GetTaskReportsResponse response = 
         recordFactory.newRecordInstance(GetTaskReportsResponse.class);
         recordFactory.newRecordInstance(GetTaskReportsResponse.class);
       
       
-      Job job = verifyAndGetJob(jobId, false);
+      Job job = verifyAndGetJob(jobId, JobACL.VIEW_JOB);
       Collection<Task> tasks = job.getTasks(taskType).values();
       Collection<Task> tasks = job.getTasks(taskType).values();
       LOG.info("Getting task report for " + taskType + "   " + jobId
       LOG.info("Getting task report for " + taskType + "   " + jobId
           + ". Report-size will be " + tasks.size());
           + ". Report-size will be " + tasks.size());

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

@@ -18,13 +18,20 @@
 
 
 package org.apache.hadoop.mapreduce.v2.app;
 package org.apache.hadoop.mapreduce.v2.app;
 
 
+import static org.junit.Assert.fail;
+
+import java.security.PrivilegedExceptionAction;
 import java.util.Iterator;
 import java.util.Iterator;
 import java.util.List;
 import java.util.List;
 
 
 import junit.framework.Assert;
 import junit.framework.Assert;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.JobACL;
+import org.apache.hadoop.mapreduce.MRConfig;
+import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.v2.api.MRClientProtocol;
 import org.apache.hadoop.mapreduce.v2.api.MRClientProtocol;
+import org.apache.hadoop.mapreduce.v2.api.protocolrecords.FailTaskAttemptRequest;
 import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetCountersRequest;
 import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetCountersRequest;
 import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetDiagnosticsRequest;
 import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetDiagnosticsRequest;
 import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetJobReportRequest;
 import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetJobReportRequest;
@@ -32,6 +39,9 @@ import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetTaskAttemptCompleti
 import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetTaskAttemptReportRequest;
 import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetTaskAttemptReportRequest;
 import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetTaskReportRequest;
 import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetTaskReportRequest;
 import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetTaskReportsRequest;
 import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetTaskReportsRequest;
+import org.apache.hadoop.mapreduce.v2.api.protocolrecords.KillJobRequest;
+import org.apache.hadoop.mapreduce.v2.api.protocolrecords.KillTaskAttemptRequest;
+import org.apache.hadoop.mapreduce.v2.api.protocolrecords.KillTaskRequest;
 import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
 import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
 import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
 import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
 import org.apache.hadoop.mapreduce.v2.api.records.JobState;
 import org.apache.hadoop.mapreduce.v2.api.records.JobState;
@@ -51,6 +61,8 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent.TaskAttemptStatus;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent.TaskAttemptStatus;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
@@ -169,6 +181,79 @@ public class TestMRClientService {
     app.waitForState(job, JobState.SUCCEEDED);
     app.waitForState(job, JobState.SUCCEEDED);
   }
   }
 
 
+  @Test
+  public void testViewAclOnlyCannotModify() throws Exception {
+    final MRAppWithClientService app = new MRAppWithClientService(1, 0, false);
+    final Configuration conf = new Configuration();
+    conf.setBoolean(MRConfig.MR_ACLS_ENABLED, true);
+    conf.set(MRJobConfig.JOB_ACL_VIEW_JOB, "viewonlyuser");
+    Job job = app.submit(conf);
+    app.waitForState(job, JobState.RUNNING);
+    Assert.assertEquals("Num tasks not correct", 1, job.getTasks().size());
+    Iterator<Task> it = job.getTasks().values().iterator();
+    Task task = it.next();
+    app.waitForState(task, TaskState.RUNNING);
+    TaskAttempt attempt = task.getAttempts().values().iterator().next();
+    app.waitForState(attempt, TaskAttemptState.RUNNING);
+
+    UserGroupInformation viewOnlyUser =
+        UserGroupInformation.createUserForTesting(
+            "viewonlyuser", new String[] {});
+    Assert.assertTrue("viewonlyuser cannot view job",
+        job.checkAccess(viewOnlyUser, JobACL.VIEW_JOB));
+    Assert.assertFalse("viewonlyuser can modify job",
+        job.checkAccess(viewOnlyUser, JobACL.MODIFY_JOB));
+    MRClientProtocol client = viewOnlyUser.doAs(
+        new PrivilegedExceptionAction<MRClientProtocol>() {
+          @Override
+          public MRClientProtocol run() throws Exception {
+            YarnRPC rpc = YarnRPC.create(conf);
+            return (MRClientProtocol) rpc.getProxy(MRClientProtocol.class,
+                app.clientService.getBindAddress(), conf);
+          }
+        });
+
+    KillJobRequest killJobRequest = recordFactory.newRecordInstance(
+        KillJobRequest.class);
+    killJobRequest.setJobId(app.getJobId());
+    try {
+      client.killJob(killJobRequest);
+      fail("viewonlyuser killed job");
+    } catch (AccessControlException e) {
+      // pass
+    }
+
+    KillTaskRequest killTaskRequest = recordFactory.newRecordInstance(
+        KillTaskRequest.class);
+    killTaskRequest.setTaskId(task.getID());
+    try {
+      client.killTask(killTaskRequest);
+      fail("viewonlyuser killed task");
+    } catch (AccessControlException e) {
+      // pass
+    }
+
+    KillTaskAttemptRequest killTaskAttemptRequest =
+        recordFactory.newRecordInstance(KillTaskAttemptRequest.class);
+    killTaskAttemptRequest.setTaskAttemptId(attempt.getID());
+    try {
+      client.killTaskAttempt(killTaskAttemptRequest);
+      fail("viewonlyuser killed task attempt");
+    } catch (AccessControlException e) {
+      // pass
+    }
+
+    FailTaskAttemptRequest failTaskAttemptRequest =
+        recordFactory.newRecordInstance(FailTaskAttemptRequest.class);
+    failTaskAttemptRequest.setTaskAttemptId(attempt.getID());
+    try {
+      client.failTaskAttempt(failTaskAttemptRequest);
+      fail("viewonlyuser killed task attempt");
+    } catch (AccessControlException e) {
+      // pass
+    }
+  }
+
   private void verifyJobReport(JobReport jr) {
   private void verifyJobReport(JobReport jr) {
     Assert.assertNotNull("JobReport is null", jr);
     Assert.assertNotNull("JobReport is null", jr);
     List<AMInfo> amInfos = jr.getAMInfos();
     List<AMInfo> amInfos = jr.getAMInfos();

+ 7 - 0
hadoop-yarn-project/CHANGES.txt

@@ -86,6 +86,9 @@ Release 2.1.1-beta - UNRELEASED
     applications together with running apps by default, following up YARN-1074.
     applications together with running apps by default, following up YARN-1074.
     (Xuan Gong via vinodkv)
     (Xuan Gong via vinodkv)
 
 
+    YARN-1065. NM should provide AuxillaryService data to the container (Xuan
+    Gong via bikas)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES
@@ -149,6 +152,10 @@ Release 2.1.1-beta - UNRELEASED
     YARN-1077. Fixed TestContainerLaunch test failure on Windows. (Chuan Liu via
     YARN-1077. Fixed TestContainerLaunch test failure on Windows. (Chuan Liu via
     vinodkv)
     vinodkv)
 
 
+    YARN-957. Fixed a bug in CapacityScheduler because of which requests that
+    need more than a node's total capability were incorrectly allocated on that
+    node causing apps to hang. (Omkar Vinit Joshi via vinodkv)
+
 Release 2.1.0-beta - 2013-08-22
 Release 2.1.0-beta - 2013-08-22
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 48 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/AuxiliaryServiceHelper.java

@@ -0,0 +1,48 @@
+/**
+* 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.util;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+
+import org.apache.commons.codec.binary.Base64;
+
+
+public class AuxiliaryServiceHelper {
+
+  public final static String NM_AUX_SERVICE = "NM_AUX_SERVICE_";
+
+  public static ByteBuffer getServiceDataFromEnv(String serviceName,
+      Map<String, String> env) {
+    byte[] metaData =
+        Base64.decodeBase64(env.get(getPrefixServiceName(serviceName)));
+    return ByteBuffer.wrap(metaData);
+  }
+
+  public static void setServiceDataIntoEnv(String serviceName,
+      ByteBuffer metaData, Map<String, String> env) {
+    byte[] byteData = metaData.array();
+    env.put(getPrefixServiceName(serviceName),
+        Base64.encodeBase64String(byteData));
+  }
+
+  private static String getPrefixServiceName(String serviceName) {
+    return NM_AUX_SERVICE + serviceName;
+  }
+}

+ 5 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java

@@ -216,7 +216,7 @@ public class ContainerManagerImpl extends CompositeService implements
 
 
   protected ContainersLauncher createContainersLauncher(Context context,
   protected ContainersLauncher createContainersLauncher(Context context,
       ContainerExecutor exec) {
       ContainerExecutor exec) {
-    return new ContainersLauncher(context, this.dispatcher, exec, dirsHandler);
+    return new ContainersLauncher(context, this.dispatcher, exec, dirsHandler, this);
   }
   }
 
 
   @Override
   @Override
@@ -410,7 +410,7 @@ public class ContainerManagerImpl extends CompositeService implements
       }
       }
     }
     }
 
 
-    return StartContainersResponse.newInstance(auxiliaryServices.getMetaData(),
+    return StartContainersResponse.newInstance(getAuxServiceMetaData(),
       succeededContainers, failedContainers);
       succeededContainers, failedContainers);
   }
   }
 
 
@@ -759,4 +759,7 @@ public class ContainerManagerImpl extends CompositeService implements
     return this.context;
     return this.context;
   }
   }
 
 
+  public Map<String, ByteBuffer> getAuxServiceMetaData() {
+    return this.auxiliaryServices.getMetaData();
+  }
 }
 }

+ 13 - 2
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

@@ -26,6 +26,7 @@ import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.io.OutputStream;
 import java.io.PrintStream;
 import java.io.PrintStream;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashMap;
@@ -60,6 +61,7 @@ import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.ExitCode;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.Signal;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.Signal;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService;
 import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerDiagnosticsUpdateEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerDiagnosticsUpdateEvent;
@@ -70,6 +72,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.Cont
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
 import org.apache.hadoop.yarn.server.nodemanager.util.ProcessIdFileReader;
 import org.apache.hadoop.yarn.server.nodemanager.util.ProcessIdFileReader;
 import org.apache.hadoop.yarn.util.Apps;
 import org.apache.hadoop.yarn.util.Apps;
+import org.apache.hadoop.yarn.util.AuxiliaryServiceHelper;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 
 
 public class ContainerLaunch implements Callable<Integer> {
 public class ContainerLaunch implements Callable<Integer> {
@@ -88,6 +91,7 @@ public class ContainerLaunch implements Callable<Integer> {
   private final Container container;
   private final Container container;
   private final Configuration conf;
   private final Configuration conf;
   private final Context context;
   private final Context context;
+  private final ContainerManagerImpl containerManager;
   
   
   private volatile AtomicBoolean shouldLaunchContainer = new AtomicBoolean(false);
   private volatile AtomicBoolean shouldLaunchContainer = new AtomicBoolean(false);
   private volatile AtomicBoolean completed = new AtomicBoolean(false);
   private volatile AtomicBoolean completed = new AtomicBoolean(false);
@@ -101,7 +105,8 @@ public class ContainerLaunch implements Callable<Integer> {
 
 
   public ContainerLaunch(Context context, Configuration configuration,
   public ContainerLaunch(Context context, Configuration configuration,
       Dispatcher dispatcher, ContainerExecutor exec, Application app,
       Dispatcher dispatcher, ContainerExecutor exec, Application app,
-      Container container, LocalDirsHandlerService dirsHandler) {
+      Container container, LocalDirsHandlerService dirsHandler,
+      ContainerManagerImpl containerManager) {
     this.context = context;
     this.context = context;
     this.conf = configuration;
     this.conf = configuration;
     this.app = app;
     this.app = app;
@@ -109,6 +114,7 @@ public class ContainerLaunch implements Callable<Integer> {
     this.container = container;
     this.container = container;
     this.dispatcher = dispatcher;
     this.dispatcher = dispatcher;
     this.dirsHandler = dirsHandler;
     this.dirsHandler = dirsHandler;
+    this.containerManager = containerManager;
     this.sleepDelayBeforeSigKill =
     this.sleepDelayBeforeSigKill =
         conf.getLong(YarnConfiguration.NM_SLEEP_DELAY_BEFORE_SIGKILL_MS,
         conf.getLong(YarnConfiguration.NM_SLEEP_DELAY_BEFORE_SIGKILL_MS,
             YarnConfiguration.DEFAULT_NM_SLEEP_DELAY_BEFORE_SIGKILL_MS);
             YarnConfiguration.DEFAULT_NM_SLEEP_DELAY_BEFORE_SIGKILL_MS);
@@ -227,7 +233,6 @@ public class ContainerLaunch implements Callable<Integer> {
             ApplicationConstants.CONTAINER_TOKEN_FILE_ENV_NAME, 
             ApplicationConstants.CONTAINER_TOKEN_FILE_ENV_NAME, 
             new Path(containerWorkDir, 
             new Path(containerWorkDir, 
                 FINAL_CONTAINER_TOKENS_FILE).toUri().getPath());
                 FINAL_CONTAINER_TOKENS_FILE).toUri().getPath());
-
         // Sanitize the container's environment
         // Sanitize the container's environment
         sanitizeEnv(environment, containerWorkDir, appDirs, containerLogDirs,
         sanitizeEnv(environment, containerWorkDir, appDirs, containerLogDirs,
           localResources);
           localResources);
@@ -680,6 +685,12 @@ public class ContainerLaunch implements Callable<Integer> {
         environment.put(Environment.CLASSPATH.name(), classPathJar);
         environment.put(Environment.CLASSPATH.name(), classPathJar);
       }
       }
     }
     }
+    // put AuxiliaryService data to environment
+    for (Map.Entry<String, ByteBuffer> meta : containerManager
+        .getAuxServiceMetaData().entrySet()) {
+      AuxiliaryServiceHelper.setServiceDataIntoEnv(
+          meta.getKey(), meta.getValue(), environment);
+    }
   }
   }
     
     
   static void writeLaunchEnv(OutputStream out,
   static void writeLaunchEnv(OutputStream out,

+ 6 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncher.java

@@ -41,6 +41,7 @@ import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.ExitCode;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.ExitCode;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService;
 import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEventType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEventType;
@@ -65,6 +66,7 @@ public class ContainersLauncher extends AbstractService
   private final Context context;
   private final Context context;
   private final ContainerExecutor exec;
   private final ContainerExecutor exec;
   private final Dispatcher dispatcher;
   private final Dispatcher dispatcher;
+  private final ContainerManagerImpl containerManager;
 
 
   private LocalDirsHandlerService dirsHandler;
   private LocalDirsHandlerService dirsHandler;
   @VisibleForTesting
   @VisibleForTesting
@@ -89,12 +91,14 @@ public class ContainersLauncher extends AbstractService
 
 
 
 
   public ContainersLauncher(Context context, Dispatcher dispatcher,
   public ContainersLauncher(Context context, Dispatcher dispatcher,
-      ContainerExecutor exec, LocalDirsHandlerService dirsHandler) {
+      ContainerExecutor exec, LocalDirsHandlerService dirsHandler,
+      ContainerManagerImpl containerManager) {
     super("containers-launcher");
     super("containers-launcher");
     this.exec = exec;
     this.exec = exec;
     this.context = context;
     this.context = context;
     this.dispatcher = dispatcher;
     this.dispatcher = dispatcher;
     this.dirsHandler = dirsHandler;
     this.dirsHandler = dirsHandler;
+    this.containerManager = containerManager;
   }
   }
 
 
   @Override
   @Override
@@ -128,7 +132,7 @@ public class ContainersLauncher extends AbstractService
 
 
         ContainerLaunch launch =
         ContainerLaunch launch =
             new ContainerLaunch(context, getConfig(), dispatcher, exec, app,
             new ContainerLaunch(context, getConfig(), dispatcher, exec, app,
-              event.getContainer(), dirsHandler);
+              event.getContainer(), dirsHandler, containerManager);
         running.put(containerId,
         running.put(containerId,
             new RunningContainer(containerLauncher.submit(launch), 
             new RunningContainer(containerLauncher.submit(launch), 
                 launch));
                 launch));

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/DummyContainerManager.java

@@ -145,7 +145,7 @@ public class DummyContainerManager extends ContainerManagerImpl {
   protected ContainersLauncher createContainersLauncher(Context context,
   protected ContainersLauncher createContainersLauncher(Context context,
       ContainerExecutor exec) {
       ContainerExecutor exec) {
     return new ContainersLauncher(context, super.dispatcher, exec,
     return new ContainersLauncher(context, super.dispatcher, exec,
-                                  super.dirsHandler) {
+                                  super.dirsHandler, this) {
       @Override
       @Override
       public void handle(ContainersLauncherEvent event) {
       public void handle(ContainersLauncherEvent event) {
         Container container = event.getContainer();
         Container container = event.getContainer();

+ 13 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java

@@ -20,8 +20,11 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager;
 
 
 import java.io.File;
 import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
 import java.util.List;
+import java.util.Map;
 
 
 import junit.framework.Assert;
 import junit.framework.Assert;
 
 
@@ -211,6 +214,16 @@ public abstract class BaseContainerManagerTest {
             NMTokenIdentifier nmTokenIdentifier) throws InvalidToken {
             NMTokenIdentifier nmTokenIdentifier) throws InvalidToken {
           // Do nothing
           // Do nothing
         }
         }
+
+      @Override
+      public Map<String, ByteBuffer> getAuxServiceMetaData() {
+        Map<String, ByteBuffer> serviceData = new HashMap<String, ByteBuffer>();
+        serviceData.put("AuxService1",
+            ByteBuffer.wrap("AuxServiceMetaData1".getBytes()));
+        serviceData.put("AuxService2",
+            ByteBuffer.wrap("AuxServiceMetaData2".getBytes()));
+        return serviceData;
+      }
     };
     };
   }
   }
 
 

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java

@@ -650,7 +650,7 @@ public class TestContainer {
       Context context = mock(Context.class);
       Context context = mock(Context.class);
       when(context.getApplications()).thenReturn(
       when(context.getApplications()).thenReturn(
           new ConcurrentHashMap<ApplicationId, Application>());
           new ConcurrentHashMap<ApplicationId, Application>());
-      launcher = new ContainersLauncher(context, dispatcher, null, null);
+      launcher = new ContainersLauncher(context, dispatcher, null, null, null);
       // create a mock ExecutorService, which will not really launch
       // create a mock ExecutorService, which will not really launch
       // ContainerLaunch at all.
       // ContainerLaunch at all.
       launcher.containerLauncher = mock(ExecutorService.class);
       launcher.containerLauncher = mock(ExecutorService.class);

+ 34 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.spy;
 
 
 import java.io.BufferedReader;
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.File;
@@ -28,6 +29,7 @@ import java.io.FileOutputStream;
 import java.io.FileReader;
 import java.io.FileReader;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.io.PrintWriter;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.Collections;
@@ -37,6 +39,7 @@ import java.util.Map;
 
 
 import junit.framework.Assert;
 import junit.framework.Assert;
 
 
+import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
@@ -70,11 +73,13 @@ import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.ExitCode;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.ExitCode;
 import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEventType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEventType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerExitEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerExitEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
+import org.apache.hadoop.yarn.util.AuxiliaryServiceHelper;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.LinuxResourceCalculatorPlugin;
 import org.apache.hadoop.yarn.util.LinuxResourceCalculatorPlugin;
 import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin;
 import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin;
@@ -381,6 +386,12 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
     	  + processStartFile);
     	  + processStartFile);
       fileWriter.println("@echo " + Environment.HOME.$() + ">> "
       fileWriter.println("@echo " + Environment.HOME.$() + ">> "
           + processStartFile);
           + processStartFile);
+      for (String serviceName : containerManager.getAuxServiceMetaData()
+          .keySet()) {
+        fileWriter.println("@echo" + AuxiliaryServiceHelper.NM_AUX_SERVICE
+            + serviceName + " >> "
+            + processStartFile);
+      }
       fileWriter.println("@echo " + cId + ">> " + processStartFile);
       fileWriter.println("@echo " + cId + ">> " + processStartFile);
       fileWriter.println("@ping -n 100 127.0.0.1 >nul");
       fileWriter.println("@ping -n 100 127.0.0.1 >nul");
     } else {
     } else {
@@ -403,6 +414,12 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
           + processStartFile);
           + processStartFile);
       fileWriter.write("\necho $" + Environment.HOME.name() + " >> "
       fileWriter.write("\necho $" + Environment.HOME.name() + " >> "
           + processStartFile);
           + processStartFile);
+      for (String serviceName : containerManager.getAuxServiceMetaData()
+          .keySet()) {
+        fileWriter.write("\necho $" + AuxiliaryServiceHelper.NM_AUX_SERVICE
+            + serviceName + " >> "
+            + processStartFile);
+      }
       fileWriter.write("\necho $$ >> " + processStartFile);
       fileWriter.write("\necho $$ >> " + processStartFile);
       fileWriter.write("\nexec sleep 100");
       fileWriter.write("\nexec sleep 100");
     }
     }
@@ -487,6 +504,12 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
               YarnConfiguration.DEFAULT_NM_USER_HOME_DIR),
               YarnConfiguration.DEFAULT_NM_USER_HOME_DIR),
         reader.readLine());
         reader.readLine());
 
 
+    for (String serviceName : containerManager.getAuxServiceMetaData().keySet()) {
+      Assert.assertEquals(
+          containerManager.getAuxServiceMetaData().get(serviceName),
+          ByteBuffer.wrap(Base64.decodeBase64(reader.readLine().getBytes())));
+    }
+
     Assert.assertEquals(cId.toString(), containerLaunchContext
     Assert.assertEquals(cId.toString(), containerLaunchContext
         .getEnvironment().get(Environment.CONTAINER_ID.name()));
         .getEnvironment().get(Environment.CONTAINER_ID.name()));
     Assert.assertEquals(context.getNodeId().getHost(), containerLaunchContext
     Assert.assertEquals(context.getNodeId().getHost(), containerLaunchContext
@@ -557,6 +580,16 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
       DefaultContainerExecutor.containerIsAlive(pid));
       DefaultContainerExecutor.containerIsAlive(pid));
   }
   }
 
 
+  @Test (timeout = 5000)
+  public void testAuxiliaryServiceHelper() throws Exception {
+    Map<String, String> env = new HashMap<String, String>();
+    String serviceName = "testAuxiliaryService";
+    ByteBuffer bb = ByteBuffer.wrap("testAuxiliaryService".getBytes());
+    AuxiliaryServiceHelper.setServiceDataIntoEnv(serviceName, bb, env);
+    Assert.assertEquals(bb,
+        AuxiliaryServiceHelper.getServiceDataFromEnv(serviceName, env));
+  }
+
   @Test
   @Test
   public void testDelayedKill() throws Exception {
   public void testDelayedKill() throws Exception {
     containerManager.start();
     containerManager.start();
@@ -703,7 +736,7 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
     };
     };
     when(dispatcher.getEventHandler()).thenReturn(eventHandler);
     when(dispatcher.getEventHandler()).thenReturn(eventHandler);
     ContainerLaunch launch = new ContainerLaunch(context, new Configuration(),
     ContainerLaunch launch = new ContainerLaunch(context, new Configuration(),
-        dispatcher, exec, null, container, dirsHandler);
+        dispatcher, exec, null, container, dirsHandler, containerManager);
     launch.call();
     launch.call();
   }
   }
 
 

+ 5 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java

@@ -67,4 +67,9 @@ public abstract class SchedulerNode {
    */
    */
   public abstract int getNumContainers();
   public abstract int getNumContainers();
 
 
+  /**
+   * Get total resources on the node.
+   * @return total resources on the node.
+   */
+  public abstract Resource getTotalResource();
 }
 }

+ 7 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java

@@ -1308,9 +1308,15 @@ public class LeafQueue implements CSQueue {
         + " request=" + request + " type=" + type);
         + " request=" + request + " type=" + type);
     }
     }
     Resource capability = request.getCapability();
     Resource capability = request.getCapability();
-
     Resource available = node.getAvailableResource();
     Resource available = node.getAvailableResource();
+    Resource totalResource = node.getTotalResource();
 
 
+    if (!Resources.fitsIn(capability, totalResource)) {
+      LOG.warn("Node : " + node.getNodeID()
+          + " does not have sufficient resource for request : " + request
+          + " node total capability : " + node.getTotalResource());
+      return Resources.none();
+    }
     assert Resources.greaterThan(
     assert Resources.greaterThan(
         resourceCalculator, clusterResource, available, Resources.none());
         resourceCalculator, clusterResource, available, Resources.none());
 
 

+ 9 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.java

@@ -49,6 +49,7 @@ public class FiCaSchedulerNode extends SchedulerNode {
 
 
   private Resource availableResource = recordFactory.newRecordInstance(Resource.class);
   private Resource availableResource = recordFactory.newRecordInstance(Resource.class);
   private Resource usedResource = recordFactory.newRecordInstance(Resource.class);
   private Resource usedResource = recordFactory.newRecordInstance(Resource.class);
+  private Resource totalResourceCapability;
 
 
   private volatile int numContainers;
   private volatile int numContainers;
 
 
@@ -65,6 +66,9 @@ public class FiCaSchedulerNode extends SchedulerNode {
     this.rmNode = node;
     this.rmNode = node;
     this.availableResource.setMemory(node.getTotalCapability().getMemory());
     this.availableResource.setMemory(node.getTotalCapability().getMemory());
     this.availableResource.setVirtualCores(node.getTotalCapability().getVirtualCores());
     this.availableResource.setVirtualCores(node.getTotalCapability().getVirtualCores());
+    totalResourceCapability =
+        Resource.newInstance(node.getTotalCapability().getMemory(), node
+            .getTotalCapability().getVirtualCores());
     if (usePortForNodeName) {
     if (usePortForNodeName) {
       nodeName = rmNode.getHostName() + ":" + node.getNodeID().getPort();
       nodeName = rmNode.getHostName() + ":" + node.getNodeID().getPort();
     } else {
     } else {
@@ -126,6 +130,11 @@ public class FiCaSchedulerNode extends SchedulerNode {
     return this.usedResource;
     return this.usedResource;
   }
   }
 
 
+  @Override
+  public Resource getTotalResource() {
+    return this.totalResourceCapability;
+  }
+
   private synchronized boolean isValidContainer(Container c) {    
   private synchronized boolean isValidContainer(Container c) {    
     if (launchedContainers.containsKey(c.getId()))
     if (launchedContainers.containsKey(c.getId()))
       return true;
       return true;

+ 9 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSSchedulerNode.java

@@ -52,6 +52,7 @@ public class FSSchedulerNode extends SchedulerNode {
 
 
   private Resource availableResource;
   private Resource availableResource;
   private Resource usedResource = recordFactory.newRecordInstance(Resource.class);
   private Resource usedResource = recordFactory.newRecordInstance(Resource.class);
+  private Resource totalResourceCapability;
 
 
   private volatile int numContainers;
   private volatile int numContainers;
 
 
@@ -68,6 +69,9 @@ public class FSSchedulerNode extends SchedulerNode {
   public FSSchedulerNode(RMNode node, boolean usePortForNodeName) {
   public FSSchedulerNode(RMNode node, boolean usePortForNodeName) {
     this.rmNode = node;
     this.rmNode = node;
     this.availableResource = Resources.clone(node.getTotalCapability());
     this.availableResource = Resources.clone(node.getTotalCapability());
+    totalResourceCapability =
+        Resource.newInstance(node.getTotalCapability().getMemory(), node
+            .getTotalCapability().getVirtualCores());
     if (usePortForNodeName) {
     if (usePortForNodeName) {
       nodeName = rmNode.getHostName() + ":" + node.getNodeID().getPort();
       nodeName = rmNode.getHostName() + ":" + node.getNodeID().getPort();
     } else {
     } else {
@@ -173,6 +177,11 @@ public class FSSchedulerNode extends SchedulerNode {
     Resources.subtractFrom(usedResource, resource);
     Resources.subtractFrom(usedResource, resource);
   }
   }
 
 
+  @Override
+  public Resource getTotalResource() {
+    return this.totalResourceCapability;
+  }
+
   private synchronized void deductAvailableResource(Resource resource) {
   private synchronized void deductAvailableResource(Resource resource) {
     if (resource == null) {
     if (resource == null) {
       LOG.error("Invalid deduction of null resource for "
       LOG.error("Invalid deduction of null resource for "

+ 8 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java

@@ -232,6 +232,14 @@ public class MockRM extends ResourceManager {
     return nm;
     return nm;
   }
   }
 
 
+  public MockNM registerNode(String nodeIdStr, int memory, int vCores)
+      throws Exception {
+    MockNM nm =
+        new MockNM(nodeIdStr, memory, vCores, getResourceTrackerService());
+    nm.registerNode();
+    return nm;
+  }
+
   public void sendNodeStarted(MockNM nm) throws Exception {
   public void sendNodeStarted(MockNM nm) throws Exception {
     RMNodeImpl node = (RMNodeImpl) getRMContext().getRMNodes().get(
     RMNodeImpl node = (RMNodeImpl) getRMContext().getRMNodes().get(
         nm.getNodeId());
         nm.getNodeId());

+ 109 - 0
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

@@ -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.yarn.server.resourcemanager.scheduler.capacity;
+
+import junit.framework.Assert;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
+import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
+import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
+import org.apache.hadoop.yarn.server.resourcemanager.TestFifoScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
+import org.junit.Test;
+
+
+public class TestContainerAllocation {
+
+  private static final Log LOG = LogFactory.getLog(TestFifoScheduler.class);
+
+  private final int GB = 1024;
+
+  @Test(timeout = 3000000)
+  public void testExcessReservationThanNodeManagerCapacity() throws Exception {
+    YarnConfiguration conf = new YarnConfiguration();
+    MockRM rm = new MockRM(conf);
+    rm.start();
+
+    // Register node1
+    MockNM nm1 = rm.registerNode("127.0.0.1:1234", 2 * GB, 4);
+    MockNM nm2 = rm.registerNode("127.0.0.1:2234", 3 * GB, 4);
+
+    nm1.nodeHeartbeat(true);
+    nm2.nodeHeartbeat(true);
+
+    // wait..
+    int waitCount = 20;
+    int size = rm.getRMContext().getRMNodes().size();
+    while ((size = rm.getRMContext().getRMNodes().size()) != 2
+        && waitCount-- > 0) {
+      LOG.info("Waiting for node managers to register : " + size);
+      Thread.sleep(100);
+    }
+    Assert.assertEquals(2, rm.getRMContext().getRMNodes().size());
+    // Submit an application
+    RMApp app1 = rm.submitApp(128);
+
+    // kick the scheduling
+    nm1.nodeHeartbeat(true);
+    RMAppAttempt attempt1 = app1.getCurrentAppAttempt();
+    MockAM am1 = rm.sendAMLaunched(attempt1.getAppAttemptId());
+    am1.registerAppAttempt();
+
+    LOG.info("sending container requests ");
+    am1.addRequests(new String[] {"*"}, 3 * GB, 1, 1);
+    AllocateResponse alloc1Response = am1.schedule(); // send the request
+
+    // kick the scheduler
+    nm1.nodeHeartbeat(true);
+    int waitCounter = 20;
+    LOG.info("heartbeating nm1");
+    while (alloc1Response.getAllocatedContainers().size() < 1
+        && waitCounter-- > 0) {
+      LOG.info("Waiting for containers to be created for app 1...");
+      Thread.sleep(500);
+      alloc1Response = am1.schedule();
+    }
+    LOG.info("received container : "
+        + alloc1Response.getAllocatedContainers().size());
+
+    // No container should be allocated.
+    // Internally it should not been reserved.
+    Assert.assertTrue(alloc1Response.getAllocatedContainers().size() == 0);
+
+    LOG.info("heartbeating nm2");
+    waitCounter = 20;
+    nm2.nodeHeartbeat(true);
+    while (alloc1Response.getAllocatedContainers().size() < 1
+        && waitCounter-- > 0) {
+      LOG.info("Waiting for containers to be created for app 1...");
+      Thread.sleep(500);
+      alloc1Response = am1.schedule();
+    }
+    LOG.info("received container : "
+        + alloc1Response.getAllocatedContainers().size());
+    Assert.assertTrue(alloc1Response.getAllocatedContainers().size() == 1);
+
+    rm.stop();
+  }
+}